diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java index ab247e97ef54..640c944315cc 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java @@ -730,19 +730,6 @@ public void setBatchTimeInterval(int batchTimeInterval) { } } - /** - * Set GroupTransactionEvents for this GatewaySender. - * - * Care must be taken to set this consistently across all gateway senders in the cluster and only - * when safe to do so. - * - * @since Geode 1.15 - * - */ - public void setGroupTransactionEvents(boolean groupTransactionEvents) { - // TODO jbarrett remove this - } - /** * Set GatewayEventFilters for this GatewaySender. * diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java index 6198de920d4d..f8aadbeed889 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java @@ -32,7 +32,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; -import java.util.function.Predicate; import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; import org.apache.logging.log4j.Logger; @@ -55,7 +54,6 @@ import org.apache.geode.cache.RegionAttributes; import org.apache.geode.cache.RegionDestroyedException; import org.apache.geode.cache.RegionShortcut; -import org.apache.geode.cache.TransactionId; import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueImpl; import org.apache.geode.distributed.internal.DistributionManager; import org.apache.geode.distributed.internal.InternalDistributedSystem; @@ -1506,41 +1504,6 @@ protected Object peekAhead(PartitionedRegion prQ, int bucketId) throws CacheExce // finished with peeked object. } - // TODO jbarrett move this - protected List peekEventsWithTransactionId(PartitionedRegion prQ, int bucketId, - TransactionId transactionId) throws CacheException { - List objects; - BucketRegionQueue brq = getBucketRegionQueueByBucketId(prQ, bucketId); - - try { - Predicate hasTransactionIdPredicate = - getHasTransactionIdPredicate(transactionId); - Predicate isLastEventInTransactionPredicate = - getIsLastEventInTransactionPredicate(); - objects = - brq.getElementsMatching(hasTransactionIdPredicate, isLastEventInTransactionPredicate); - } catch (BucketRegionQueueUnavailableException e) { - // BucketRegionQueue unavailable. Can be due to the BucketRegionQueue being destroyed. - return Collections.emptyList(); - } - - return objects; // OFFHEAP: ok since callers are careful to do destroys on region queue after - // finished with peeked objects. - } - - // TODO jbarrett move this - @VisibleForTesting - public static Predicate getIsLastEventInTransactionPredicate() { - return GatewaySenderEventImpl::isLastEventInTransaction; - } - - // TODO jbarrett move this - @VisibleForTesting - public static Predicate getHasTransactionIdPredicate( - TransactionId transactionId) { - return x -> transactionId.equals(x.getTransactionId()); - } - protected BucketRegionQueue getBucketRegionQueueByBucketId(final PartitionedRegion prQ, final int bucketId) { return (BucketRegionQueue) prQ.getDataStore().getLocalBucketById(bucketId); @@ -2014,7 +1977,7 @@ public int size(PartitionedRegion pr, int bucketId) throws ForceReattemptExcepti throw new RuntimeException("This method(size)is not supported by ParallelGatewaySenderQueue"); } - static class MetaRegionFactory { + public static class MetaRegionFactory { ParallelGatewaySenderQueueMetaRegion newMetataRegion(InternalCache cache, final String prQName, final RegionAttributes ra, AbstractGatewaySender sender) { ParallelGatewaySenderQueueMetaRegion meta = diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java index 5b2a04019d26..c01a349455e4 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java @@ -18,17 +18,14 @@ import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast; import java.util.ArrayList; -import java.util.Collections; import java.util.Deque; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Predicate; import org.apache.logging.log4j.Logger; @@ -107,23 +104,7 @@ public class SerialGatewaySenderQueue implements RegionQueue { */ private final AtomicLong lastPeekedId = new AtomicLong(-1); - protected final Deque peekedIds = new LinkedBlockingDeque(); - - /** - * Contains the set of peekedIds that were peeked to complete a transaction - * inside a batch when groupTransactionEvents is set. - */ - protected final Set extraPeekedIds = ConcurrentHashMap.newKeySet(); - - /** - * Contains the set of peekedIds that were peeked to complete a transaction - * inside a batch when groupTransactionEvents is set and whose event has been - * removed from the queue because an ack has been received from the receiver. - * Elements from this set are deleted when the event with the previous id - * is removed. - */ - private final Set extraPeekedIdsRemovedButPreviousIdNotRemoved = - ConcurrentHashMap.newKeySet(); + protected final Deque peekedIds = new LinkedBlockingDeque<>(); /** * The name of the Region backing this queue @@ -133,7 +114,7 @@ public class SerialGatewaySenderQueue implements RegionQueue { /** * The Region backing this queue */ - private Region> region; + protected Region> region; /** * The name of the DiskStore to overflow this queue @@ -181,9 +162,9 @@ public class SerialGatewaySenderQueue implements RegionQueue { private static final boolean NO_ACK = Boolean.getBoolean(GeodeGlossary.GEMFIRE_PREFIX + "gateway-queue-no-ack"); - private volatile long lastDispatchedKey = -1; + protected volatile long lastDispatchedKey = -1; - private volatile long lastDestroyedKey = -1; + protected volatile long lastDestroyedKey = -1; public static final int DEFAULT_MESSAGE_SYNC_INTERVAL = 1; @@ -305,13 +286,9 @@ public synchronized void remove() throws CacheException { } boolean wasEmpty = lastDispatchedKey == lastDestroyedKey; Long key = peekedIds.remove(); - boolean isExtraPeekedId = extraPeekedIds.contains(key); - if (!isExtraPeekedId) { - updateHeadKey(key); - lastDispatchedKey = key; - } else { - extraPeekedIdsRemovedButPreviousIdNotRemoved.add(key); - } + updateHeadKey(key); + lastDispatchedKey = key; + removeIndex(key); // Remove the entry at that key with a callback arg signifying it is // a WAN queue so that AbstractRegionEntry.destroy can get the value @@ -330,19 +307,6 @@ public synchronized void remove() throws CacheException { } } - // For those extraPeekedIds removed that are consecutive to lastDispatchedKey: - // - Update lastDispatchedKey with them so that they are removed - // by the batch removal thread. - // - Update the head key with them. - // - Remove them from extraPeekedIds. - long tmpKey = lastDispatchedKey; - while (extraPeekedIdsRemovedButPreviousIdNotRemoved.contains(tmpKey = inc(tmpKey))) { - extraPeekedIdsRemovedButPreviousIdNotRemoved.remove(tmpKey); - extraPeekedIds.remove(tmpKey); - updateHeadKey(tmpKey); - lastDispatchedKey = tmpKey; - } - if (wasEmpty) { synchronized (this) { notifyAll(); @@ -562,7 +526,7 @@ private void removeOldEntry(Conflatable object, Long tailKey) throws CacheExcept /** * Does a get that gets the value without fault values in from disk. */ - private AsyncEvent optimalGet(Long k) { + protected AsyncEvent optimalGet(Long k) { // Get the object at that key (to remove the index). LocalRegion lr = (LocalRegion) region; Object o = null; @@ -584,7 +548,7 @@ private void removeOldEntry(Conflatable object, Long tailKey) throws CacheExcept /* * this must be invoked under synchronization */ - private void removeIndex(Long qkey) { + protected void removeIndex(Long qkey) { // Determine whether conflation is enabled for this queue and object if (enableConflation) { // only call get after checking enableConflation for bug 40508 @@ -627,7 +591,7 @@ private boolean before(long a, long b) { return a < b ^ a - b > (MAXIMUM_KEY / 2); } - private long inc(long value) { + protected long inc(long value) { long val = value + 1; val = val == MAXIMUM_KEY ? 0 : val; return val; @@ -639,7 +603,6 @@ private long inc(long value) { */ public void resetLastPeeked() { peekedIds.clear(); - extraPeekedIds.clear(); lastPeekedId.set(-1); } @@ -679,7 +642,7 @@ public static class KeyAndEventPair { public final long key; public final AsyncEvent event; - KeyAndEventPair(Long key, AsyncEvent event) { + public KeyAndEventPair(Long key, AsyncEvent event) { this.key = key; this.event = event; } @@ -708,7 +671,7 @@ public KeyAndEventPair peekAhead() throws CacheException { // does not save anything since GatewayBatchOp needs to GatewayEventImpl // in object form. while (before(currentKey, getTailKey())) { - if (!extraPeekedIds.contains(currentKey)) { + if (!skipPeekedKey(currentKey)) { object = getObjectInSerialSenderQueue(currentKey); if (object != null) { break; @@ -733,54 +696,23 @@ public KeyAndEventPair peekAhead() throws CacheException { return null; } + protected boolean skipPeekedKey(Long currentKey) { + return false; + } + protected void incrementEventsNotQueueConflated() { if (stats != null) { stats.incEventsNotQueuedConflated(); } } - - /** - * 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. - */ - protected List getElementsMatching(Predicate condition, - Predicate stopCondition, - long lastKey) { - GatewaySenderEventImpl event; - List elementsMatching = new ArrayList<>(); - - long currentKey = lastKey; - - while ((currentKey = inc(currentKey)) != getTailKey()) { - if (extraPeekedIds.contains(currentKey)) { - continue; - } - event = (GatewaySenderEventImpl) optimalGet(currentKey); - if (event == null) { - continue; - } - - if (condition.test(event)) { - elementsMatching.add(new KeyAndEventPair(currentKey, event)); - - if (stopCondition.test(event)) { - break; - } - } - } - - return elementsMatching; - } - /** * Returns the value of the tail key. The tail key points to an empty where the next queue entry * will be stored. * * @return the value of the tail key */ - private long getTailKey() throws CacheException { + protected long getTailKey() throws CacheException { long tlKey; // Test whether tailKey = -1. If so, the queue has just been created. // Go into the region to get the value of TAIL_KEY. If it is null, then @@ -908,7 +840,7 @@ private long getHeadKey() throws CacheException { * Increments the value of the head key by one. * */ - private void updateHeadKey(long destroyedKey) throws CacheException { + protected void updateHeadKey(long destroyedKey) throws CacheException { headKey = inc(destroyedKey); if (logger.isTraceEnabled()) { logger.trace("{}: Incremented HEAD_KEY for region {} to {}", this, region.getName(), @@ -1199,17 +1131,12 @@ public void shutdown() { } @VisibleForTesting - long getLastPeekedId() { + public long getLastPeekedId() { return lastPeekedId.get(); } - @VisibleForTesting - Set getExtraPeekedIds() { - return Collections.unmodifiableSet(extraPeekedIds); - } - public static class SerialGatewaySenderQueueMetaRegion extends DistributedRegion { - final AbstractGatewaySender sender; + AbstractGatewaySender sender; protected SerialGatewaySenderQueueMetaRegion(String regionName, RegionAttributes attrs, LocalRegion parentRegion, InternalCache cache, AbstractGatewaySender sender, @@ -1312,7 +1239,7 @@ public boolean virtualPut(EntryEventImpl event, boolean ifNew, boolean ifOld, } } - static class MetaRegionFactory { + protected static class MetaRegionFactory { SerialGatewaySenderQueueMetaRegion newMetaRegion(InternalCache cache, final String regionName, final RegionAttributes ra, diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionQueueJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionQueueJUnitTest.java index 6643695ea085..581f261dfa49 100644 --- a/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionQueueJUnitTest.java +++ b/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionQueueJUnitTest.java @@ -39,7 +39,6 @@ import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl; import org.apache.geode.internal.cache.wan.GatewaySenderStats; import org.apache.geode.internal.cache.wan.parallel.ParallelGatewaySenderHelper; -import org.apache.geode.internal.cache.wan.parallel.ParallelGatewaySenderQueue; import org.apache.geode.internal.statistics.DummyStatisticsFactory; import org.apache.geode.test.fake.Fakes; @@ -169,9 +168,9 @@ public void testGetElementsMatchingWithParallelGatewaySenderQueuePredicatesAndSo bucketRegionQueue.addToQueue(8L, event7); Predicate hasTransactionIdPredicate = - ParallelGatewaySenderQueue.getHasTransactionIdPredicate(tx1); + x -> tx1.equals(x.getTransactionId()); Predicate isLastEventInTransactionPredicate = - ParallelGatewaySenderQueue.getIsLastEventInTransactionPredicate(); + GatewaySenderEventImpl::isLastEventInTransaction; List objects = bucketRegionQueue.getElementsMatching(hasTransactionIdPredicate, isLastEventInTransactionPredicate); @@ -183,8 +182,7 @@ public void testGetElementsMatchingWithParallelGatewaySenderQueuePredicatesAndSo assertEquals(1, objects.size()); assertEquals(objects, Arrays.asList(event7)); - hasTransactionIdPredicate = - ParallelGatewaySenderQueue.getHasTransactionIdPredicate(tx2); + hasTransactionIdPredicate = x -> tx2.equals(x.getTransactionId()); objects = bucketRegionQueue.getElementsMatching(hasTransactionIdPredicate, isLastEventInTransactionPredicate); assertEquals(2, objects.size()); diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/AlterGatewaySenderFunction.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/AlterGatewaySenderFunction.java index 1169cb2f624f..77ae264d9311 100644 --- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/AlterGatewaySenderFunction.java +++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/AlterGatewaySenderFunction.java @@ -95,12 +95,6 @@ private GatewaySender alterGatewaySender(Cache cache, ((AbstractGatewaySender) gateway).setBatchTimeInterval(batchTimeInterval); } - Boolean groupTransactionEvents = gatewaySenderCreateArgs.mustGroupTransactionEvents(); - if (groupTransactionEvents != null) { - ((AbstractGatewaySender) gateway) - .setGroupTransactionEvents(groupTransactionEvents); - } - List gatewayEventFilters = gatewaySenderCreateArgs.getGatewayEventFilter(); if (gatewayEventFilters != null) { List filters = new ArrayList<>(); diff --git a/geode-wan-txgrouping/build.gradle b/geode-wan-txgrouping/build.gradle index 796045e25aa6..12378559a28c 100644 --- a/geode-wan-txgrouping/build.gradle +++ b/geode-wan-txgrouping/build.gradle @@ -27,10 +27,12 @@ dependencies { compileOnly('org.jetbrains:annotations') - implementation(project(':geode-wan')) implementation(project(':geode-logging')) + implementation(project(':geode-membership')) implementation(project(':geode-serialization')) + implementation(project(':geode-tcp-server')) implementation(project(':geode-core')) + implementation(project(':geode-wan')) // test @@ -39,4 +41,15 @@ dependencies { testImplementation('junit:junit') testImplementation('org.mockito:mockito-core') + integrationTestImplementation(project(':geode-junit')) + integrationTestImplementation('org.assertj:assertj-core') + integrationTestImplementation('junit:junit') + + distributedTestImplementation(project(':geode-dunit')) + distributedTestImplementation(project(':geode-junit')) + + distributedTestImplementation('org.awaitility:awaitility') + distributedTestImplementation('junit:junit') + distributedTestImplementation('org.assertj:assertj-core') + distributedTestImplementation('pl.pragmatists:JUnitParams') } diff --git a/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/TxGroupingBaseDUnitTest.java b/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/TxGroupingBaseDUnitTest.java new file mode 100644 index 000000000000..4bf839eaa223 --- /dev/null +++ b/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/TxGroupingBaseDUnitTest.java @@ -0,0 +1,531 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.internal.cache.wan.txgrouping; + +import static org.apache.geode.cache.Region.SEPARATOR; +import static org.apache.geode.distributed.ConfigurationProperties.DISTRIBUTED_SYSTEM_ID; +import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS; +import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT; +import static org.apache.geode.distributed.ConfigurationProperties.REMOTE_LOCATORS; +import static org.apache.geode.distributed.ConfigurationProperties.START_LOCATOR; +import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts; +import static org.apache.geode.test.awaitility.GeodeAwaitility.await; +import static org.apache.geode.test.dunit.VM.getVM; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import org.apache.geode.cache.CacheTransactionManager; +import org.apache.geode.cache.DiskStore; +import org.apache.geode.cache.DiskStoreFactory; +import org.apache.geode.cache.PartitionAttributesFactory; +import org.apache.geode.cache.Region; +import org.apache.geode.cache.RegionFactory; +import org.apache.geode.cache.RegionShortcut; +import org.apache.geode.cache.TransactionException; +import org.apache.geode.cache.wan.GatewayReceiver; +import org.apache.geode.cache.wan.GatewayReceiverFactory; +import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.cache.wan.GatewaySenderFactory; +import org.apache.geode.distributed.LocatorLauncher; +import org.apache.geode.distributed.ServerLauncher; +import org.apache.geode.internal.cache.CacheServerImpl; +import org.apache.geode.internal.cache.CustomerIDPartitionResolver; +import org.apache.geode.internal.cache.PartitionedRegion; +import org.apache.geode.internal.cache.RegionQueue; +import org.apache.geode.internal.cache.execute.data.OrderId; +import org.apache.geode.internal.cache.tier.sockets.CacheServerStats; +import org.apache.geode.internal.cache.wan.AbstractGatewaySender; +import org.apache.geode.internal.cache.wan.GatewayReceiverStats; +import org.apache.geode.internal.cache.wan.GatewaySenderStats; +import org.apache.geode.internal.cache.wan.InternalGatewaySenderFactory; +import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySenderQueue; +import org.apache.geode.test.dunit.Invoke; +import org.apache.geode.test.dunit.VM; +import org.apache.geode.test.dunit.rules.CacheRule; +import org.apache.geode.test.dunit.rules.DistributedRule; +import org.apache.geode.test.junit.categories.WanTest; +import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@Category({WanTest.class}) +@RunWith(GeodeParamsRunner.class) +public class TxGroupingBaseDUnitTest implements Serializable { + + protected static final String REGION_NAME = "TheRegion"; + + protected final String shipmentRegionName = "ShipmentsRegion"; + protected final String customerRegionName = "CustomersRegion"; + protected final String orderRegionName = "OrdersRegion"; + + protected static LocatorLauncher locatorLauncher; + protected static ServerLauncher serverLauncher; + + protected VM londonLocatorVM; + protected VM newYorkLocatorVM; + protected VM newYorkServerVM; + protected VM londonServer1VM; + protected VM londonServer2VM; + protected VM londonServer3VM; + protected VM londonServer4VM; + protected VM[] londonServersVM; + + protected String newYorkName; + + protected int londonId; + protected int newYorkId; + + protected int londonLocatorPort; + protected int newYorkLocatorPort; + + protected int newYorkReceiverPort; + + @Rule + public DistributedRule distributedRule = new DistributedRule(); + + @Rule + public CacheRule cacheRule = new CacheRule(); + + @Rule + public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder(); + + private static List dispatcherThreads = new ArrayList<>(Arrays.asList(1, 3, 5)); + // this will be set for each test method run with one of the values from above list + private static int numDispatcherThreadsForTheRun = 1; + + @Before + public void setUp() { + londonLocatorVM = getVM(0); + newYorkLocatorVM = getVM(1); + newYorkServerVM = getVM(2); + londonServer1VM = getVM(3); + londonServer2VM = getVM(4); + londonServer3VM = getVM(5); + londonServer4VM = getVM(6); + londonServersVM = new VM[] {londonServer1VM, londonServer2VM, londonServer3VM, londonServer4VM}; + + newYorkName = "ny"; + + londonId = 1; + newYorkId = 2; + + int[] ports = getRandomAvailableTCPPorts(3); + londonLocatorPort = ports[0]; + newYorkLocatorPort = ports[1]; + newYorkReceiverPort = ports[2]; + + newYorkLocatorVM.invoke("start New York locator", () -> { + Properties config = createLocatorConfig(newYorkId, newYorkLocatorPort, londonLocatorPort); + cacheRule.createCache(config); + }); + + londonLocatorVM.invoke("start London locator", () -> { + Properties config = createLocatorConfig(londonId, londonLocatorPort, newYorkLocatorPort); + cacheRule.createCache(config); + }); + Collections.shuffle(dispatcherThreads); + int dispatcherThreadsNo = dispatcherThreads.get(0); + Invoke.invokeInEveryVM(() -> setNumDispatcherThreadsForTheRun(dispatcherThreadsNo)); + + } + + @After + public void tearDown() { + newYorkServerVM.invoke(() -> { + if (serverLauncher != null) { + serverLauncher.stop(); + serverLauncher = null; + } + }); + + for (VM server : londonServersVM) { + server.invoke(() -> { + if (serverLauncher != null) { + serverLauncher.stop(); + serverLauncher = null; + } + }); + } + + newYorkLocatorVM.invoke(() -> { + if (locatorLauncher != null) { + locatorLauncher.stop(); + locatorLauncher = null; + } + }); + + londonLocatorVM.invoke(() -> { + if (locatorLauncher != null) { + locatorLauncher.stop(); + locatorLauncher = null; + } + }); + } + + protected Properties createLocatorConfig(int systemId, int locatorPort, int remoteLocatorPort) { + Properties config = new Properties(); + config.setProperty(MCAST_PORT, "0"); + config.setProperty(DISTRIBUTED_SYSTEM_ID, String.valueOf(systemId)); + config.setProperty(LOCATORS, "localhost[" + locatorPort + ']'); + config.setProperty(REMOTE_LOCATORS, "localhost[" + remoteLocatorPort + ']'); + config.setProperty(START_LOCATOR, + "localhost[" + locatorPort + "],server=true,peer=true,hostname-for-clients=localhost"); + return config; + } + + protected void startServerWithSender(int systemId, int locatorPort, int remoteSystemId, + String remoteName, boolean isParallel, boolean groupTransactionEvents, int batchSize) + throws IOException { + startServerWithSender(systemId, locatorPort, remoteSystemId, remoteName, isParallel, + groupTransactionEvents, batchSize, 0); + } + + protected void startServerWithSender(int systemId, int locatorPort, int remoteSystemId, + String remoteName, boolean isParallel, boolean groupTransactionEvents, int batchSize, + int dispatcherThreads) throws IOException { + cacheRule.createCache(createServerConfig(locatorPort)); + + String uniqueName = "server-" + systemId; + File[] dirs = new File[] {temporaryFolder.newFolder(uniqueName)}; + + GatewaySenderFactory senderFactory = createGatewaySenderFactory(dirs, uniqueName); + senderFactory.setParallel(isParallel); + senderFactory.setGroupTransactionEvents(groupTransactionEvents); + senderFactory.setBatchSize(batchSize); + if (dispatcherThreads > 0) { + senderFactory.setDispatcherThreads(dispatcherThreads); + } + GatewaySender sender = senderFactory.create(remoteName, remoteSystemId); + sender.start(); + } + + protected void startServerWithReceiver(int locatorPort, + int receiverPort) throws IOException { + startServerWithReceiver(locatorPort, receiverPort, true); + } + + protected void startServerWithReceiver(int locatorPort, + int receiverPort, boolean start) throws IOException { + cacheRule.createCache(createServerConfig(locatorPort)); + + GatewayReceiverFactory receiverFactory = createGatewayReceiverFactory(receiverPort); + GatewayReceiver receiver = receiverFactory.create(); + if (start) { + receiver.start(); + } + } + + protected void startReceiver() throws IOException { + cacheRule.getCache().getGatewayReceivers().iterator().next().start(); + } + + protected GatewayReceiverFactory createGatewayReceiverFactory(int receiverPort) { + GatewayReceiverFactory receiverFactory = cacheRule.getCache().createGatewayReceiverFactory(); + + receiverFactory.setStartPort(receiverPort); + receiverFactory.setEndPort(receiverPort); + receiverFactory.setManualStart(true); + return receiverFactory; + } + + protected Properties createServerConfig(int locatorPort) { + Properties config = new Properties(); + config.setProperty(MCAST_PORT, "0"); + config.setProperty(LOCATORS, "localhost[" + locatorPort + ']'); + return config; + } + + protected GatewaySenderFactory createGatewaySenderFactory(File[] dirs, String diskStoreName) { + InternalGatewaySenderFactory senderFactory = + (InternalGatewaySenderFactory) cacheRule.getCache().createGatewaySenderFactory(); + + senderFactory.setMaximumQueueMemory(100); + senderFactory.setBatchSize(10); + senderFactory.setBatchConflationEnabled(false); + senderFactory.setManualStart(true); + senderFactory.setDispatcherThreads(numDispatcherThreadsForTheRun); + senderFactory.setOrderPolicy(GatewaySender.DEFAULT_ORDER_POLICY); + + DiskStoreFactory dsf = cacheRule.getCache().createDiskStoreFactory(); + DiskStore store = dsf.setDiskDirs(dirs).create(diskStoreName); + senderFactory.setDiskStoreName(store.getName()); + + return senderFactory; + } + + protected boolean isRunning(GatewaySender sender) { + return sender != null && sender.isRunning(); + } + + protected void validateRegionSize(String regionName, final int regionSize) { + final Region r = cacheRule.getCache().getRegion(SEPARATOR + regionName); + assertNotNull(r); + if (regionSize != r.keySet().size()) { + await().untilAsserted(() -> assertThat(r.keySet().size()).isEqualTo(regionSize)); + } + } + + protected List getSenderStats(String senderId, final int expectedQueueSize) { + AbstractGatewaySender sender = + (AbstractGatewaySender) cacheRule.getCache().getGatewaySender(senderId); + GatewaySenderStats statistics = sender.getStatistics(); + if (expectedQueueSize != -1) { + final RegionQueue regionQueue; + regionQueue = sender.getQueues().toArray(new RegionQueue[1])[0]; + if (sender.isParallel()) { + ConcurrentParallelGatewaySenderQueue parallelGatewaySenderQueue = + (ConcurrentParallelGatewaySenderQueue) regionQueue; + PartitionedRegion pr = + parallelGatewaySenderQueue.getRegions().toArray(new PartitionedRegion[1])[0]; + } + await() + .untilAsserted(() -> assertThat(regionQueue.size()).isEqualTo(expectedQueueSize)); + } + ArrayList stats = new ArrayList<>(); + stats.add(statistics.getEventQueueSize()); + stats.add(statistics.getEventsReceived()); + stats.add(statistics.getEventsQueued()); + stats.add(statistics.getEventsDistributed()); + stats.add(statistics.getBatchesDistributed()); + stats.add(statistics.getBatchesRedistributed()); + stats.add(statistics.getEventsFiltered()); + stats.add(statistics.getEventsNotQueuedConflated()); + stats.add(statistics.getEventsConflatedFromBatches()); + stats.add(statistics.getConflationIndexesMapSize()); + stats.add(statistics.getSecondaryEventQueueSize()); + stats.add(statistics.getEventsProcessedByPQRM()); + stats.add(statistics.getEventsExceedingAlertThreshold()); + stats.add((int) statistics.getBatchesWithIncompleteTransactions()); + return stats; + } + + protected GatewaySender getGatewaySender(String senderId) { + Set senders = cacheRule.getCache().getGatewaySenders(); + GatewaySender sender = null; + for (GatewaySender s : senders) { + if (s.getId().equals(senderId)) { + sender = s; + break; + } + } + return sender; + } + + protected void doPutsInsideTransactions(String regionName, Map keyValues, + int eventsPerTransaction) { + Region r = cacheRule.getCache().getRegion(Region.SEPARATOR + regionName); + assertNotNull(r); + int eventInTransaction = 0; + CacheTransactionManager cacheTransactionManager = + cacheRule.getCache().getCacheTransactionManager(); + for (Object key : keyValues.keySet()) { + if (eventInTransaction == 0) { + cacheTransactionManager.begin(); + } + r.put(key, keyValues.get(key)); + if (++eventInTransaction == eventsPerTransaction) { + cacheTransactionManager.commit(); + eventInTransaction = 0; + } + } + if (eventInTransaction != 0) { + cacheTransactionManager.commit(); + } + } + + protected void checkGatewayReceiverStats(int processBatches, int eventsReceived, + int creates) { + checkGatewayReceiverStats(processBatches, eventsReceived, creates, false); + } + + protected void checkGatewayReceiverStats(int processBatches, int eventsReceived, + int creates, boolean isExact) { + Set gatewayReceivers = cacheRule.getCache().getGatewayReceivers(); + GatewayReceiver receiver = gatewayReceivers.iterator().next(); + CacheServerStats stats = ((CacheServerImpl) receiver.getServer()).getAcceptor().getStats(); + + assertThat(stats).isInstanceOf(GatewayReceiverStats.class); + GatewayReceiverStats gatewayReceiverStats = (GatewayReceiverStats) stats; + if (isExact) { + assertThat(gatewayReceiverStats.getProcessBatchRequests()).isEqualTo(processBatches); + } else { + assertThat(gatewayReceiverStats.getProcessBatchRequests()) + .isGreaterThanOrEqualTo(processBatches); + } + assertThat(eventsReceived).isEqualTo(gatewayReceiverStats.getEventsReceived()); + assertThat(creates).isEqualTo(gatewayReceiverStats.getCreateRequest()); + } + + protected void doTxPutsWithRetryIfError(String regionName, final long putsPerTransaction, + final long transactions, long offset) { + Region r = cacheRule.getCache().getRegion(Region.SEPARATOR + regionName); + long keyOffset = offset * ((putsPerTransaction + (10 * transactions)) * 100); + long j; + CacheTransactionManager mgr = cacheRule.getCache().getCacheTransactionManager(); + for (int i = 0; i < transactions; i++) { + boolean done = false; + do { + try { + mgr.begin(); + for (j = 0; j < putsPerTransaction; j++) { + long key = keyOffset + ((j + (10L * i)) * 100); + String value = "Value_" + key; + r.put(key, value); + } + mgr.commit(); + done = true; + } catch (TransactionException ignore) { + } catch (IllegalStateException ignore) { + try { + mgr.rollback(); + } catch (Exception ignored) { + } + } + } while (!done); + } + } + + public void createCustomerOrderShipmentPartitionedRegion(String senderId) { + RegionFactory fact = + cacheRule.getCache().createRegionFactory(RegionShortcut.PARTITION); + if (senderId != null) { + fact.addGatewaySenderId(senderId); + } + + PartitionAttributesFactory paf = new PartitionAttributesFactory(); + paf.setPartitionResolver(new CustomerIDPartitionResolver("CustomerIDPartitionResolver")); + fact.setPartitionAttributes(paf.create()); + fact.create(customerRegionName); + + paf = new PartitionAttributesFactory(); + paf.setColocatedWith(customerRegionName) + .setPartitionResolver(new CustomerIDPartitionResolver("CustomerIDPartitionResolver")); + fact = cacheRule.getCache().createRegionFactory(RegionShortcut.PARTITION); + if (senderId != null) { + fact.addGatewaySenderId(senderId); + } + fact.setPartitionAttributes(paf.create()); + fact.create(orderRegionName); + + paf = new PartitionAttributesFactory(); + paf.setColocatedWith(orderRegionName) + .setPartitionResolver(new CustomerIDPartitionResolver("CustomerIDPartitionResolver")); + fact = cacheRule.getCache().createRegionFactory(RegionShortcut.PARTITION); + if (senderId != null) { + fact.addGatewaySenderId(senderId); + } + fact.setPartitionAttributes(paf.create()); + fact.create(shipmentRegionName); + } + + public void doOrderAndShipmentPutsInsideTransactions(Map keyValues, + int eventsPerTransaction) { + Region orderRegion = cacheRule.getCache().getRegion(orderRegionName); + Region shipmentRegion = cacheRule.getCache().getRegion(shipmentRegionName); + assertNotNull(orderRegion); + assertNotNull(shipmentRegion); + int eventInTransaction = 0; + CacheTransactionManager cacheTransactionManager = + cacheRule.getCache().getCacheTransactionManager(); + for (Object key : keyValues.keySet()) { + if (eventInTransaction == 0) { + cacheTransactionManager.begin(); + } + Region r; + if (key instanceof OrderId) { + r = orderRegion; + } else { + r = shipmentRegion; + } + r.put(key, keyValues.get(key)); + if (++eventInTransaction == eventsPerTransaction) { + cacheTransactionManager.commit(); + eventInTransaction = 0; + } + } + if (eventInTransaction != 0) { + cacheTransactionManager.commit(); + } + } + + protected Integer getRegionSize(String regionName) { + final Region r = cacheRule.getCache().getRegion(SEPARATOR + regionName); + return r.keySet().size(); + } + + protected void checkGatewayReceiverStatsHA(int processBatches, int eventsReceived, + int creates) { + Set gatewayReceivers = cacheRule.getCache().getGatewayReceivers(); + GatewayReceiver receiver = gatewayReceivers.iterator().next(); + CacheServerStats stats = ((CacheServerImpl) receiver.getServer()).getAcceptor().getStats(); + assertThat(stats).isInstanceOf(GatewayReceiverStats.class); + + GatewayReceiverStats gatewayReceiverStats = (GatewayReceiverStats) stats; + assertThat(gatewayReceiverStats.getProcessBatchRequests()) + .isGreaterThanOrEqualTo(processBatches); + assertThat(gatewayReceiverStats.getEventsReceived()).isGreaterThanOrEqualTo(eventsReceived); + assertThat(gatewayReceiverStats.getCreateRequest()).isGreaterThanOrEqualTo(creates); + } + + protected void putGivenKeyValues(String regionName, Map keyValues) { + Region r = cacheRule.getCache().getRegion(SEPARATOR + regionName); + assertNotNull(r); + for (Object key : keyValues.keySet()) { + r.put(key, keyValues.get(key)); + } + } + + protected void checkConflatedStats(String senderId, final int eventsConflated) { + GatewaySenderStats statistics = getGatewaySenderStats(senderId); + assertEquals(eventsConflated, statistics.getEventsNotQueuedConflated()); + } + + protected GatewaySenderStats getGatewaySenderStats(String senderId) { + GatewaySender sender = cacheRule.getCache().getGatewaySender(senderId); + return ((AbstractGatewaySender) sender).getStatistics(); + } + + protected void validateGatewaySenderQueueAllBucketsDrained(final String senderId) { + GatewaySender sender = getGatewaySender(senderId); + final AbstractGatewaySender abstractSender = (AbstractGatewaySender) sender; + await().untilAsserted(() -> { + assertThat(abstractSender.getEventQueueSize()).isEqualTo(0); + }); + await().untilAsserted(() -> { + assertThat(abstractSender.getSecondaryEventQueueSize()).isEqualTo(0); + }); + } + + public static void setNumDispatcherThreadsForTheRun(int numThreads) { + numDispatcherThreadsForTheRun = numThreads; + } +} diff --git a/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/TxGroupingPartitionedRegionDUnitTest.java b/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/TxGroupingPartitionedRegionDUnitTest.java new file mode 100644 index 000000000000..09d8d487c987 --- /dev/null +++ b/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/TxGroupingPartitionedRegionDUnitTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.internal.cache.wan.txgrouping; + +import static org.apache.geode.test.awaitility.GeodeAwaitility.await; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.HashMap; +import java.util.Map; + +import junitparams.Parameters; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.internal.cache.execute.data.CustId; +import org.apache.geode.internal.cache.execute.data.Order; +import org.apache.geode.internal.cache.execute.data.OrderId; +import org.apache.geode.internal.cache.execute.data.Shipment; +import org.apache.geode.internal.cache.execute.data.ShipmentId; +import org.apache.geode.test.dunit.AsyncInvocation; +import org.apache.geode.test.dunit.VM; +import org.apache.geode.test.junit.categories.WanTest; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@Category({WanTest.class}) +@RunWith(GeodeParamsRunner.class) +public class TxGroupingPartitionedRegionDUnitTest extends TxGroupingBaseDUnitTest { + @Test + @Parameters({"true", "false"}) + public void testPartitionedRegionPropagationWithGroupTransactionEventsAndMixOfEventsInAndNotInTransactions( + boolean isParallel) + throws Exception { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort, true); + createCustomerOrderShipmentPartitionedRegion(null); + }); + + int batchSize = 10; + for (VM server : londonServersVM) { + server.invoke("create London server " + server.getId(), () -> { + startServerWithSender(server.getId(), londonLocatorPort, newYorkId, newYorkName, isParallel, + true, + batchSize, isParallel ? 2 : 1); + createCustomerOrderShipmentPartitionedRegion(newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + } + + int customers = 4; + + int transactionsPerCustomer = 1000; + final Map keyValuesInTransactions = new HashMap<>(); + 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); + ShipmentId shipmentId1 = new ShipmentId(i, orderId); + ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); + ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); + keyValuesInTransactions.put(orderId, new Order()); + keyValuesInTransactions.put(shipmentId1, new Shipment()); + keyValuesInTransactions.put(shipmentId2, new Shipment()); + keyValuesInTransactions.put(shipmentId3, new Shipment()); + } + } + + int ordersPerCustomerNotInTransactions = 1000; + + final Map keyValuesNotInTransactions = new HashMap<>(); + for (int custId = 0; custId < customers; custId++) { + for (int i = 0; i < ordersPerCustomerNotInTransactions; i++) { + CustId custIdObject = new CustId(custId); + OrderId orderId = new OrderId(i + transactionsPerCustomer * customers, custIdObject); + keyValuesNotInTransactions.put(orderId, new Order()); + } + } + + // eventsPerTransaction is 1 (orders) + 3 (shipments) + int eventsPerTransaction = 4; + AsyncInvocation putsInTransactionsInvocation = + londonServer1VM.invokeAsync( + () -> doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions, + eventsPerTransaction)); + + AsyncInvocation putsNotInTransactionsInvocation = + londonServer2VM.invokeAsync( + () -> putGivenKeyValues(orderRegionName, keyValuesNotInTransactions)); + + putsInTransactionsInvocation.await(); + putsNotInTransactionsInvocation.await(); + + int entries = + ordersPerCustomerNotInTransactions * customers + transactionsPerCustomer * customers; + + for (VM londonServer : londonServersVM) { + londonServer.invoke(() -> validateRegionSize(orderRegionName, entries)); + } + + newYorkServerVM.invoke(() -> validateRegionSize(orderRegionName, entries)); + + for (VM londonServer : londonServersVM) { + londonServer.invoke(() -> checkConflatedStats(newYorkName, 0)); + } + + for (VM londonServer : londonServersVM) { + londonServer.invoke(() -> validateGatewaySenderQueueAllBucketsDrained(newYorkName)); + } + } +} diff --git a/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/parallel/TxGroupingParallelDUnitTest.java b/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/parallel/TxGroupingParallelDUnitTest.java new file mode 100644 index 000000000000..10b9458c7f56 --- /dev/null +++ b/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/parallel/TxGroupingParallelDUnitTest.java @@ -0,0 +1,506 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.internal.cache.wan.txgrouping.parallel; + +import static org.apache.geode.test.awaitility.GeodeAwaitility.await; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import junitparams.Parameters; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import org.apache.geode.cache.PartitionAttributesFactory; +import org.apache.geode.cache.RegionDestroyedException; +import org.apache.geode.cache.RegionFactory; +import org.apache.geode.cache.RegionShortcut; +import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.internal.cache.ForceReattemptException; +import org.apache.geode.internal.cache.execute.data.CustId; +import org.apache.geode.internal.cache.execute.data.Customer; +import org.apache.geode.internal.cache.execute.data.Order; +import org.apache.geode.internal.cache.execute.data.OrderId; +import org.apache.geode.internal.cache.execute.data.Shipment; +import org.apache.geode.internal.cache.execute.data.ShipmentId; +import org.apache.geode.internal.cache.wan.AbstractGatewaySender; +import org.apache.geode.internal.cache.wan.txgrouping.TxGroupingBaseDUnitTest; +import org.apache.geode.internal.util.ArrayUtils; +import org.apache.geode.test.dunit.AsyncInvocation; +import org.apache.geode.test.dunit.IgnoredException; +import org.apache.geode.test.dunit.VM; +import org.apache.geode.test.junit.categories.WanTest; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@Category({WanTest.class}) +@RunWith(GeodeParamsRunner.class) +public class TxGroupingParallelDUnitTest extends TxGroupingBaseDUnitTest { + @Test + @Parameters({"true", "false"}) + public void testPRParallelPropagationWithVsWithoutGroupTransactionEvents( + boolean groupTransactionEvents) { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort); + createCustomerOrderShipmentPartitionedRegion(null); + }); + + for (VM server : londonServersVM) { + server.invoke("create London server " + server.getId(), () -> { + startServerWithSender(server.getId(), londonLocatorPort, newYorkId, newYorkName, true, + groupTransactionEvents, + 10); + createCustomerOrderShipmentPartitionedRegion(newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + } + + final Map custKeyValue = new HashMap<>(); + int intCustId = 1; + CustId custId = new CustId(intCustId); + custKeyValue.put(custId, new Customer()); + londonServer1VM.invoke(() -> putGivenKeyValues(customerRegionName, custKeyValue)); + + int transactions = 3; + final Map keyValues = new HashMap<>(); + for (int i = 0; i < transactions; i++) { + OrderId orderId = new OrderId(i, custId); + ShipmentId shipmentId1 = new ShipmentId(i, orderId); + ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); + ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); + keyValues.put(orderId, new Order()); + keyValues.put(shipmentId1, new Shipment()); + keyValues.put(shipmentId2, new Shipment()); + keyValues.put(shipmentId3, new Shipment()); + } + + // 3 transactions of 4 events each are sent so that the batch would + // initially contain the first 2 transactions complete and the first + // 2 events of the last transaction (10 entries). + // If --group-transaction-events is configured in the senders, the remaining + // 2 events of the last transaction are added to the batch which makes + // that only one batch of 12 events is sent. + // If --group-transaction-events is not configured in the senders, the + // remaining 2 events of the last transaction are added to the second batch + // which makes that 2 batches will be sent, one with 10 events and + // one with 2. + int eventsPerTransaction = 4; + londonServer1VM.invoke(() -> doOrderAndShipmentPutsInsideTransactions(keyValues, + eventsPerTransaction)); + + int entries = (transactions * eventsPerTransaction) + 1; + + londonServer1VM.invoke(() -> validateRegionSize(customerRegionName, 1)); + londonServer1VM.invoke(() -> validateRegionSize(orderRegionName, transactions)); + londonServer1VM.invoke(() -> validateRegionSize(shipmentRegionName, transactions * 3)); + + List senderStatsLondonServers = getSenderStats(newYorkName, 0, londonServersVM); + + int expectedBatchesSent = groupTransactionEvents ? 1 : 2; + // queue size: + assertThat(senderStatsLondonServers.get(0)).isEqualTo(0); + // eventsReceived: + assertThat(senderStatsLondonServers.get(1)).isEqualTo(entries); + // events queued: + assertThat(senderStatsLondonServers.get(2)).isEqualTo(entries); + // events distributed: + assertThat(senderStatsLondonServers.get(3)).isEqualTo(entries); + // batches distributed: + assertThat(senderStatsLondonServers.get(4)).isEqualTo(expectedBatchesSent); + // batches redistributed: + assertThat(senderStatsLondonServers.get(5)).isEqualTo(0); + // events not queued conflated: + assertThat(senderStatsLondonServers.get(7)).isEqualTo(0); + // batches with incomplete transactions: + assertThat(senderStatsLondonServers.get(13)).isEqualTo(0); + } + + @Test + @Parameters({"true", "false"}) + public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions_SeveralClients( + boolean isBatchesRedistributed) { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort, !isBatchesRedistributed); + createCustomerOrderShipmentPartitionedRegion(null); + }); + + int batchSize = 10; + for (VM server : londonServersVM) { + server.invoke("create London server " + server.getId(), () -> { + startServerWithSender(server.getId(), londonLocatorPort, newYorkId, newYorkName, true, true, + batchSize); + createCustomerOrderShipmentPartitionedRegion(newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + } + + int clients = 4; + int transactions = 300; + // batchSize is 10. Each transaction will contain 1 order + 3 shipments = 4 events. + // As a result, all batches will contain extra events to complete the + // transactions it will deliver. + int shipmentsPerTransaction = 3; + + final List> customerData = new ArrayList<>(clients); + for (int intCustId = 0; intCustId < clients; intCustId++) { + final Map custKeyValue = new HashMap<>(); + CustId custId = new CustId(intCustId); + custKeyValue.put(custId, new Customer()); + customerData.add(new HashMap<>()); + londonServer1VM.invoke(() -> putGivenKeyValues(customerRegionName, custKeyValue)); + + for (int i = 0; i < transactions; i++) { + OrderId orderId = new OrderId(i, custId); + customerData.get(intCustId).put(orderId, new Order()); + for (int j = 0; j < shipmentsPerTransaction; j++) { + customerData.get(intCustId).put(new ShipmentId(i + j, orderId), new Shipment()); + } + } + } + + List> asyncInvocations = new ArrayList<>(clients); + + int eventsPerTransaction = shipmentsPerTransaction + 1; + for (int i = 0; i < clients; i++) { + final int intCustId = i; + AsyncInvocation asyncInvocation = + londonServer1VM.invokeAsync(() -> doOrderAndShipmentPutsInsideTransactions( + customerData.get(intCustId), + eventsPerTransaction)); + asyncInvocations.add(asyncInvocation); + } + + try { + for (AsyncInvocation asyncInvocation : asyncInvocations) { + asyncInvocation.await(); + } + } catch (InterruptedException e) { + fail("Interrupted"); + } + + londonServer1VM.invoke(() -> validateRegionSize(customerRegionName, clients)); + londonServer1VM.invoke(() -> validateRegionSize(orderRegionName, transactions * clients)); + londonServer1VM.invoke(() -> validateRegionSize(shipmentRegionName, + transactions * shipmentsPerTransaction * clients)); + + if (isBatchesRedistributed) { + // wait for batches to be redistributed and then start the receiver + londonServer1VM.invoke(() -> await() + .until(() -> getSenderStats(newYorkName, -1).get(5) > 0)); + newYorkServerVM.invoke("start New York receiver", this::startReceiver); + } + + // Check that all entries have been written in the receiver + newYorkServerVM.invoke( + () -> validateRegionSize(customerRegionName, clients)); + newYorkServerVM.invoke( + () -> validateRegionSize(orderRegionName, transactions * clients)); + newYorkServerVM.invoke( + () -> validateRegionSize(shipmentRegionName, + shipmentsPerTransaction * transactions * clients)); + + checkQueuesAreEmptyAndOnlyCompleteTransactionsAreReplicated(newYorkName, + isBatchesRedistributed); + } + + @Test + public void testPRParallelPropagationWithGroupTransactionEventsWithIncompleteTransactionsWhenTransactionEntriesOnNotColocatedBuckets() { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort); + createPartitionedRegion(REGION_NAME, null); + }); + + int dispatcherThreads = 2; + londonServer1VM.invoke("create London server " + londonServer1VM.getId(), () -> { + startServerWithSender(londonServer1VM.getId(), londonLocatorPort, newYorkId, newYorkName, + true, true, 10, dispatcherThreads); + createPartitionedRegion(REGION_NAME, newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + + // Adding events in transactions + // Transactions will contain objects assigned to different buckets but given that there is only + // one server, there will be no TransactionDataNotCollocatedException. + // With this and by using more than one dispatcher thread, we will provoke that + // it will be impossible for the batches to have complete transactions as some + // events for a transaction will be handled by one dispatcher thread and some other events by + // another thread. + final Map keyValue = new HashMap<>(); + int entries = 30; + for (int i = 0; i < entries; i++) { + keyValue.put(i, i); + } + + int entriesPerTransaction = 3; + londonServer1VM + .invoke(() -> doPutsInsideTransactions(REGION_NAME, keyValue, entriesPerTransaction)); + + londonServer1VM.invoke(() -> validateRegionSize(REGION_NAME, entries)); + + ArrayList senderStatsLondonServer1 = + (ArrayList) londonServer1VM.invoke(() -> getSenderStats(newYorkName, 0)); + + // The number of batches will be 4 because each + // dispatcher thread (there are 2) will send half the number of entries, + // each on 2 batches. + int batches = 4; + // queue size: + assertThat(senderStatsLondonServer1.get(0)).isEqualTo(0); + // eventsReceived: + assertThat(senderStatsLondonServer1.get(1)).isEqualTo(entries); + // events queued: + assertThat(senderStatsLondonServer1.get(2)).isEqualTo(entries); + // events distributed: + assertThat(senderStatsLondonServer1.get(3)).isEqualTo(entries); + // batches distributed: + assertThat(senderStatsLondonServer1.get(4)).isEqualTo(batches); + // batches redistributed: + assertThat(senderStatsLondonServer1.get(5)).isEqualTo(0); + // events not queued conflated: + assertThat(senderStatsLondonServer1.get(7)).isEqualTo(0); + // batches with incomplete transactions + assertThat(senderStatsLondonServer1.get(13)).isEqualTo(batches); + + newYorkServerVM.invoke(() -> checkGatewayReceiverStats(batches, entries, entries)); + } + + @Test + @Parameters({"true", "false"}) + public void testPRParallelPropagationWithVsWithoutGroupTransactionEventsWithBatchRedistribution( + boolean groupTransactionEvents) { + londonServer1VM.invoke("create London server " + londonServer1VM.getId(), () -> { + startServerWithSender(londonServer1VM.getId(), londonLocatorPort, newYorkId, newYorkName, + true, groupTransactionEvents, 10); + createCustomerOrderShipmentPartitionedRegion(newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + + newYorkServerVM.invoke("create New York server with receiver stopped", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort, false); + createCustomerOrderShipmentPartitionedRegion(null); + }); + + final Map custKeyValue = new HashMap<>(); + int intCustId = 1; + CustId custId = new CustId(intCustId); + custKeyValue.put(custId, new Customer()); + londonServer1VM.invoke(() -> putGivenKeyValues(customerRegionName, custKeyValue)); + + int transactions = 6; + final Map keyValues = new HashMap<>(); + for (int i = 0; i < transactions; i++) { + OrderId orderId = new OrderId(i, custId); + ShipmentId shipmentId1 = new ShipmentId(i, orderId); + ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); + ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); + keyValues.put(orderId, new Order()); + keyValues.put(shipmentId1, new Shipment()); + keyValues.put(shipmentId2, new Shipment()); + keyValues.put(shipmentId3, new Shipment()); + } + + // 6 transactions of 4 events each are sent with batch size = 10 + // - With group transaction events: + // The first batch would initially contain the first 2 transactions complete and the first + // 2 events of the next transaction (10 entries). + // As --group-transaction-events is configured in the senders, the remaining + // 2 events of the second transaction are added to the batch which makes + // the first batch to be sent with 12 events. The same happens with the + // second batch which will contain 12 events too. + // - Without group-transaction-events 3 batches will be sent. 2 + // with 10 events and one with 4. + int expectedBatchesSent; + if (groupTransactionEvents) { + expectedBatchesSent = 2; + } else { + expectedBatchesSent = 3; + } + int eventsPerTransaction = 4; + londonServer1VM.invoke(() -> doOrderAndShipmentPutsInsideTransactions(keyValues, + eventsPerTransaction)); + + int entries = (transactions * eventsPerTransaction) + 1; + + londonServer1VM.invoke(() -> validateRegionSize(customerRegionName, 1)); + londonServer1VM.invoke(() -> validateRegionSize(orderRegionName, transactions)); + londonServer1VM.invoke(() -> validateRegionSize(shipmentRegionName, transactions * 3)); + + // wait for batches to be redistributed and then start the receiver + londonServer1VM.invoke(() -> await() + .until(() -> getSenderStats(newYorkName, -1).get(5) > 0)); + + newYorkServerVM.invoke("Start New York receiver", this::startReceiver); + + ArrayList senderStatsLondonServer1 = + (ArrayList) londonServer1VM.invoke(() -> getSenderStats(newYorkName, 0)); + + // queue size: + assertThat(senderStatsLondonServer1.get(0)).isEqualTo(0); + // events received: + assertThat(senderStatsLondonServer1.get(1)).isEqualTo(entries); + // events queued: + assertThat(senderStatsLondonServer1.get(2)).isEqualTo(entries); + // events distributed: + assertThat(senderStatsLondonServer1.get(3)).isEqualTo(entries); + // batches distributed: + assertThat(senderStatsLondonServer1.get(4)).isEqualTo(expectedBatchesSent); + // batches redistributed: + assertThat(senderStatsLondonServer1.get(5)).isGreaterThan(0); + // events not queued conflated: + assertThat(senderStatsLondonServer1.get(7)).isEqualTo(0); + } + + @Test + public void testParallelPropagationHAWithGroupTransactionEvents() throws Exception { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort); + createPartitionedRegion(REGION_NAME, null); + }); + + int batchSize = 9; + int redundantCopies = 3; + for (VM server : londonServersVM) { + server.invoke("create London server " + server.getId(), () -> { + startServerWithSender(server.getId(), londonLocatorPort, newYorkId, newYorkName, true, true, + batchSize, redundantCopies); + createPartitionedRegion(REGION_NAME, newYorkName, redundantCopies); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + } + + int putsPerTransaction = 2; + int transactions = 1000; + AsyncInvocation asyncPutInvocation = + londonServer2VM.invokeAsync( + () -> doTxPutsWithRetryIfError(REGION_NAME, putsPerTransaction, transactions, 0)); + + newYorkServerVM.invoke(() -> await() + .untilAsserted(() -> assertThat(getRegionSize(REGION_NAME)).isGreaterThan(40))); + AsyncInvocation killServerInvocation = + londonServer1VM.invokeAsync(() -> cacheRule.getCache().close()); + asyncPutInvocation.await(); + killServerInvocation.await(); + + int entries = transactions * putsPerTransaction; + newYorkServerVM + .invoke(() -> validateRegionSize(REGION_NAME, transactions * putsPerTransaction)); + + List londonServerStats = + getSenderStats(newYorkName, 0, (VM[]) ArrayUtils.remove(londonServersVM, 0)); + + // queue size + assertThat(londonServerStats.get(0)).isEqualTo(0); + + // eventsReceived + // We may see two retried events (as transactions are made of 2 events) on all members due to + // the kill + assertThat(londonServerStats.get(1)).isLessThanOrEqualTo((entries + 2) * redundantCopies); + assertThat(londonServerStats.get(1)).isGreaterThanOrEqualTo(entries * redundantCopies); + + // queuedEvents + assertThat(londonServerStats.get(2)).isLessThanOrEqualTo((entries + 2) * redundantCopies); + assertThat(londonServerStats.get(2)).isGreaterThanOrEqualTo(entries * redundantCopies); + + // batches redistributed + assertThat(londonServerStats.get(5)).isEqualTo(0); + + // batchesReceived is equal to numberOfEntries/(batchSize+1) + // As transactions are 2 events long, for each batch it will always be necessary to + // add one more entry to the 9 events batch in order to have complete transactions in the batch. + int batchesReceived = (entries) / (batchSize + 1); + newYorkServerVM.invoke(() -> checkGatewayReceiverStatsHA(batchesReceived, entries, entries)); + } + + private void checkQueuesAreEmptyAndOnlyCompleteTransactionsAreReplicated(String senderId, + boolean isBatchesRedistributed) { + List senderStatsLondonServers = getSenderStats(senderId, 0, londonServersVM); + + // queue size: + assertThat(senderStatsLondonServers.get(0)).isEqualTo(0); + // batches redistributed: + int batchesRedistributed = senderStatsLondonServers.get(5); + if (isBatchesRedistributed) { + assertThat(batchesRedistributed).isGreaterThan(0); + } else { + assertThat(batchesRedistributed).isEqualTo(0); + } + // batches with incomplete transactions + assertThat(senderStatsLondonServers.get(13)).isEqualTo(0); + + for (VM londonServer : londonServersVM) { + londonServer.invoke(() -> validateGatewaySenderQueueAllBucketsDrained(senderId)); + } + } + + protected void validateGatewaySenderQueueAllBucketsDrained(final String senderId) { + IgnoredException exp = + IgnoredException.addIgnoredException(RegionDestroyedException.class.getName()); + IgnoredException exp1 = + IgnoredException.addIgnoredException(ForceReattemptException.class.getName()); + try { + GatewaySender sender = getGatewaySender(senderId); + final AbstractGatewaySender abstractSender = (AbstractGatewaySender) sender; + await().untilAsserted(() -> assertThat(abstractSender.getEventQueueSize()).isEqualTo(0)); + await().untilAsserted( + () -> assertThat(abstractSender.getSecondaryEventQueueSize()).isEqualTo(0)); + } finally { + exp.remove(); + exp1.remove(); + } + } + + public void createPartitionedRegion(String regionName, String senderId) { + createPartitionedRegion(regionName, senderId, 0); + } + + public void createPartitionedRegion(String regionName, String senderId, int redundantCopies) { + RegionFactory fact = + cacheRule.getCache().createRegionFactory(RegionShortcut.PARTITION); + if (senderId != null) { + fact.addGatewaySenderId(senderId); + } + PartitionAttributesFactory pfact = new PartitionAttributesFactory(); + pfact.setRedundantCopies(redundantCopies); + pfact.setTotalNumBuckets(10); + fact.setPartitionAttributes(pfact.create()); + fact.create(regionName); + } + + protected List getSenderStats(String senderId, int expectedQueueSize, + VM[] servers) { + List stats = null; + for (VM server : servers) { + List serverStats = + server.invoke(() -> getSenderStats(senderId, expectedQueueSize)); + if (stats == null) { + stats = serverStats; + } else { + for (int i = 0; i < stats.size(); i++) { + stats.set(i, stats.get(i) + serverStats.get(i)); + } + } + } + return stats; + } +} diff --git a/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/serial/TxGroupingSerialDUnitTest.java b/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/serial/TxGroupingSerialDUnitTest.java new file mode 100644 index 000000000000..190412d6ecb5 --- /dev/null +++ b/geode-wan-txgrouping/src/distributedTest/java/org/apache/geode/internal/cache/wan/txgrouping/serial/TxGroupingSerialDUnitTest.java @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.internal.cache.wan.txgrouping.serial; + +import static org.apache.geode.test.awaitility.GeodeAwaitility.await; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import junitparams.Parameters; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import org.apache.geode.cache.CacheClosedException; +import org.apache.geode.cache.RegionFactory; +import org.apache.geode.cache.RegionShortcut; +import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.internal.cache.ForceReattemptException; +import org.apache.geode.internal.cache.wan.AbstractGatewaySender; +import org.apache.geode.internal.cache.wan.GatewaySenderStats; +import org.apache.geode.internal.cache.wan.txgrouping.TxGroupingBaseDUnitTest; +import org.apache.geode.test.dunit.AsyncInvocation; +import org.apache.geode.test.dunit.IgnoredException; +import org.apache.geode.test.dunit.VM; +import org.apache.geode.test.junit.categories.WanTest; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@Category({WanTest.class}) +@RunWith(GeodeParamsRunner.class) +public class TxGroupingSerialDUnitTest extends TxGroupingBaseDUnitTest { + @Test + @Parameters({"true", "false"}) + public void testReplicatedSerialPropagationWithVsWithoutGroupTransactionEvents( + boolean groupTransactionEvents) { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort); + createReplicatedRegion(REGION_NAME, null); + }); + + for (VM server : londonServersVM) { + server.invoke("create London server " + server.getId(), () -> { + startServerWithSender(server.getId(), londonLocatorPort, newYorkId, newYorkName, false, + groupTransactionEvents, 10, 1); + createReplicatedRegion(REGION_NAME, newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + } + + final Map keyValues = new HashMap<>(); + int entries = 12; + for (int i = 0; i < entries; i++) { + keyValues.put(i, i + "_Value"); + } + + // 4 transactions of 3 events each are sent so that the first batch + // would initially contain the first 3 transactions complete and the first + // event of the next transaction (10 entries). + // If --group-transaction-events is configured in the senders, the remaining + // events of the third transaction are added to the batch which makes + // that the batch is sent with 12 events. + // If --group-transaction-events is not configured in the senders, the remaining + // events of the third transaction are added to the next batch which makes + // that the 2 batches are sent. One with 10 events and another one + // with 2 events. + int expectedBatchesSent = groupTransactionEvents ? 1 : 2; + int eventsPerTransaction = 3; + londonServer2VM.invoke(() -> doPutsInsideTransactions(REGION_NAME, keyValues, + eventsPerTransaction)); + + newYorkServerVM.invoke(() -> validateRegionSize(REGION_NAME, entries)); + + newYorkServerVM + .invoke(() -> checkGatewayReceiverStats(expectedBatchesSent, entries, entries, true)); + + londonServer1VM.invoke(() -> checkQueueStats(newYorkName, 0, entries, entries, entries)); + londonServer1VM.invoke(() -> checkBatchStats(newYorkName, expectedBatchesSent, false)); + londonServer1VM.invoke(() -> checkConflatedStats(newYorkName)); + + // wait until queue is empty + londonServer2VM.invoke(() -> await() + .until(() -> getSenderStats(newYorkName, -1).get(0) == 0)); + + londonServer2VM.invoke(() -> checkQueueStats(newYorkName, 0, entries, 0, 0)); + londonServer2VM.invoke(() -> checkBatchStats(newYorkName, 0, false)); + londonServer2VM.invoke(() -> checkConflatedStats(newYorkName)); + } + + @Test + @Parameters({"true", "false"}) + public void testReplicatedSerialPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions_SeveralClients( + boolean isBatchRedistributed) throws InterruptedException { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort, !isBatchRedistributed); + createReplicatedRegion(REGION_NAME, null); + }); + + int batchSize = 10; + for (VM server : londonServersVM) { + server.invoke("create London server " + server.getId(), () -> { + startServerWithSender(server.getId(), londonLocatorPort, newYorkId, newYorkName, false, + true, + batchSize, 1); + createReplicatedRegion(REGION_NAME, newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + } + + int clients = 2; + int eventsPerTransaction = batchSize + 1; + int entriesPerInvocation = eventsPerTransaction * 200; + + final List> data = new ArrayList<>(clients); + for (int clientId = 0; clientId < clients; clientId++) { + final Map keyValues = new HashMap<>(); + for (int i = entriesPerInvocation * clientId; i < entriesPerInvocation + * (clientId + 1); i++) { + keyValues.put(i, i + "_Value"); + } + data.add(keyValues); + } + + int entries = entriesPerInvocation * clients; + + List> putAsyncInvocations = new ArrayList<>(clients); + for (int i = 0; i < clients; i++) { + final int index = i; + AsyncInvocation asyncInvocation = + londonServer1VM.invokeAsync(() -> doPutsInsideTransactions(REGION_NAME, data.get(index), + eventsPerTransaction)); + putAsyncInvocations.add(asyncInvocation); + } + + for (AsyncInvocation invocation : putAsyncInvocations) { + invocation.await(); + } + + if (isBatchRedistributed) { + // wait for batches to be redistributed and then start the receiver + londonServer1VM.invoke(() -> await() + .until(() -> getSenderStats(newYorkName, -1).get(5) > 0)); + newYorkServerVM.invoke(this::startReceiver); + } + + newYorkServerVM.invoke(() -> validateRegionSize(REGION_NAME, entries)); + + checkQueuesAreEmptyAndOnlyCompleteTransactionsAreReplicated(isBatchRedistributed); + } + + @Test + @Parameters({"true", "false"}) + public void testReplicatedSerialPropagationWithVsWithoutGroupTransactionEventsWithBatchRedistribution( + boolean groupTransactionEvents) { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort, false); + createReplicatedRegion(REGION_NAME, null); + }); + + for (VM server : londonServersVM) { + server.invoke("create London server " + server.getId(), () -> { + startServerWithSender(server.getId(), londonLocatorPort, newYorkId, newYorkName, false, + groupTransactionEvents, 10, 1); + createReplicatedRegion(REGION_NAME, newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + } + + final Map keyValues = new HashMap<>(); + int entries = 24; + for (int i = 0; i < entries; i++) { + keyValues.put(i, i + "_Value"); + } + int eventsPerTransaction = 3; + londonServer2VM.invoke(() -> doPutsInsideTransactions(REGION_NAME, keyValues, + eventsPerTransaction)); + + // wait for batches to be redistributed and then start the receiver + londonServer1VM.invoke(() -> await() + .untilAsserted(() -> assertThat(getSenderStats(newYorkName, -1).get(5)).isGreaterThan(0))); + + newYorkServerVM.invoke(this::startReceiver); + + newYorkServerVM.invoke(() -> validateRegionSize(REGION_NAME, entries)); + + // 8 transactions of 3 events each are sent. + // - With group-transaction-events + // The first batch would initially contain the first 3 transactions complete + // and the first event of the next transaction (10 entries). + // As --group-transaction-events is configured in the senders, the remaining + // events of the third transaction are added to the batch which makes + // that the first batch is sent with 12 events. The same happens with the + // second batch which will contain 12 events too. + // - Without group-transaction-events 3 batches are sent, 2 with 10 events + // and one with 4. + int expectedBatchesSent = groupTransactionEvents ? 2 : 3; + + newYorkServerVM + .invoke(() -> checkGatewayReceiverStats(expectedBatchesSent, entries, entries, true)); + + londonServer1VM.invoke(() -> checkQueueStats(newYorkName, 0, entries, entries, entries)); + londonServer1VM.invoke(() -> checkBatchStats(newYorkName, expectedBatchesSent, true)); + + // wait until queue is empty + londonServer2VM.invoke(() -> getSenderStats(newYorkName, 0)); + + londonServer2VM.invoke(() -> checkQueueStats(newYorkName, 0, entries, 0, 0)); + londonServer2VM.invoke(() -> checkBatchStats(newYorkName, 0, false)); + } + + @Test + public void testReplicatedSerialPropagationHAWithGroupTransactionEvents() throws Exception { + newYorkServerVM.invoke("create New York server", () -> { + startServerWithReceiver(newYorkLocatorPort, newYorkReceiverPort); + createReplicatedRegion(REGION_NAME, null); + }); + + int batchSize = 9; + for (VM server : londonServersVM) { + server.invoke("create London server " + server.getId(), () -> { + startServerWithSender(server.getId(), londonLocatorPort, newYorkId, newYorkName, false, + true, batchSize, 1); + createReplicatedRegion(REGION_NAME, newYorkName); + GatewaySender sender = cacheRule.getCache().getGatewaySender(newYorkName); + await().untilAsserted(() -> assertThat(isRunning(sender)).isTrue()); + }); + } + + int putsPerTransaction = 2; + int transactions = 5000; + AsyncInvocation putsInvocation1 = + londonServer3VM.invokeAsync( + () -> doTxPutsWithRetryIfError(REGION_NAME, putsPerTransaction, transactions, 0)); + AsyncInvocation putsInvocation2 = + londonServer4VM.invokeAsync( + () -> doTxPutsWithRetryIfError(REGION_NAME, putsPerTransaction, transactions, 1)); + + newYorkServerVM.invoke(() -> await() + .untilAsserted(() -> assertThat(getRegionSize(REGION_NAME)).isGreaterThan(40))); + + AsyncInvocation killServerAsyncInvocation = + londonServer1VM.invokeAsync(() -> killPrimarySender(newYorkName)); + Boolean isKilled = killServerAsyncInvocation.get(); + if (!isKilled) { + AsyncInvocation killServerAsyncInvocation2 = + londonServer2VM.invokeAsync(() -> killPrimarySender(newYorkName)); + killServerAsyncInvocation2.await(); + } + putsInvocation1.await(); + putsInvocation2.await(); + killServerAsyncInvocation.await(); + + int entries = 2 * putsPerTransaction * transactions; + londonServer2VM.invoke(() -> validateRegionSize(REGION_NAME, entries)); + newYorkServerVM.invoke(() -> validateRegionSize(REGION_NAME, entries)); + + // batchesReceived is equal to numberOfEntries/(batchSize+1) + // As transactions are 2 events long, for each batch it will always be necessary to + // add one more entry to the 9 events batch in order to have complete transactions in the batch. + int batchesReceived = entries / (batchSize + 1); + newYorkServerVM.invoke(() -> checkGatewayReceiverStatsHA(batchesReceived, entries, entries)); + + londonServer2VM.invoke(() -> checkStats_Failover(newYorkName, entries)); + } + + private void checkQueuesAreEmptyAndOnlyCompleteTransactionsAreReplicated( + boolean isBatchesRedistributed) { + // Wait for sender queues to be empty + List> londonServersStats = new ArrayList(londonServersVM.length); + int i = 0; + for (VM londonServer : londonServersVM) { + londonServersStats.add(londonServer.invoke(() -> getSenderStats(newYorkName, 0))); + } + + int queueSize = londonServersStats.stream().map(x -> x.get(0)).reduce(0, (y, z) -> y + z); + assertThat(queueSize).isEqualTo(0); + + // batches redistributed: + int batchesRedistributed = + londonServersStats.stream().map(x -> x.get(5)).reduce(0, (y, z) -> y + z); + if (isBatchesRedistributed) { + assertThat(batchesRedistributed).isGreaterThan(0); + } else { + assertThat(batchesRedistributed).isEqualTo(0); + } + } + + private void createReplicatedRegion(String regionName, String senderId) { + RegionFactory fact = + cacheRule.getCache().createRegionFactory(RegionShortcut.REPLICATE); + if (senderId != null) { + fact.addGatewaySenderId(senderId); + } + fact.create(regionName); + } + + private void checkQueueStats(String senderId, final int queueSize, final int eventsReceived, + final int eventsQueued, final int eventsDistributed) { + GatewaySenderStats statistics = getGatewaySenderStats(senderId); + assertThat(statistics.getEventQueueSize()).isEqualTo(queueSize); + assertThat(statistics.getEventsReceived()).isEqualTo(eventsReceived); + assertThat(statistics.getEventsQueued()).isEqualTo(eventsQueued); + assertThat(statistics.getEventsDistributed()).isGreaterThanOrEqualTo(eventsDistributed); + } + + private void checkBatchStats(String senderId, final int batches, + final boolean batchesRedistributed) { + GatewaySenderStats statistics = getGatewaySenderStats(senderId); + assertThat(statistics.getBatchesDistributed()).isEqualTo(batches); + + if (batchesRedistributed) { + assertThat(statistics.getBatchesRedistributed()).isGreaterThan(0); + } else { + assertThat(statistics.getBatchesRedistributed()).isEqualTo(0); + } + } + + private void checkConflatedStats(String senderId) { + GatewaySenderStats statistics = getGatewaySenderStats(senderId); + assertThat(statistics.getEventsNotQueuedConflated()).isEqualTo(0); + } + + private void checkStats_Failover(String senderId, final int eventsReceived) { + GatewaySenderStats statistics = getGatewaySenderStats(senderId); + assertThat(statistics.getEventsReceived()).isEqualTo(eventsReceived); + assertThat((statistics.getEventsQueued() + statistics.getUnprocessedTokensAddedByPrimary() + + statistics.getUnprocessedEventsRemovedByPrimary())).isEqualTo(eventsReceived); + } + + private Boolean killPrimarySender(String senderId) { + IgnoredException ignoredException1 = IgnoredException.addIgnoredException("Could not connect"); + IgnoredException ignoredException2 = + IgnoredException.addIgnoredException(CacheClosedException.class.getName()); + IgnoredException ignoredException3 = + IgnoredException.addIgnoredException(ForceReattemptException.class.getName()); + try { + AbstractGatewaySender sender = (AbstractGatewaySender) getGatewaySender(senderId); + if (sender.isPrimary()) { + cacheRule.getCache().close(); + return Boolean.TRUE; + } + return Boolean.FALSE; + } finally { + ignoredException1.remove(); + ignoredException2.remove(); + ignoredException3.remove(); + } + } +} diff --git a/geode-wan-txgrouping/src/integrationTest/java/org/apache/geode/internal/cache/wan/txgrouping/WanTxGroupingConfigurationJUnitTest.java b/geode-wan-txgrouping/src/integrationTest/java/org/apache/geode/internal/cache/wan/txgrouping/WanTxGroupingConfigurationJUnitTest.java new file mode 100644 index 000000000000..fcf94ef3bd35 --- /dev/null +++ b/geode-wan-txgrouping/src/integrationTest/java/org/apache/geode/internal/cache/wan/txgrouping/WanTxGroupingConfigurationJUnitTest.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.internal.cache.wan.txgrouping; + +import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertEquals; + +import java.util.Set; + +import org.junit.After; +import org.junit.Test; + +import org.apache.geode.cache.Cache; +import org.apache.geode.cache.CacheFactory; +import org.apache.geode.cache.RegionFactory; +import org.apache.geode.cache.RegionShortcut; +import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.cache.wan.GatewaySenderFactory; +import org.apache.geode.internal.cache.wan.GatewaySenderException; + +public class WanTxGroupingConfigurationJUnitTest { + + private Cache cache; + + /** + * Test to validate that serial gateway sender group transaction events can + * be correctly set to true + */ + @Test + public void test_ValidateSerialGatewaySenderGroupTransactionEventsAttributeSetToTrue() { + cache = new CacheFactory().set(MCAST_PORT, "0").create(); + GatewaySenderFactory fact = cache.createGatewaySenderFactory(); + fact.setParallel(true); + fact.setBatchConflationEnabled(false); + boolean groupTransactionEvents = true; + fact.setManualStart(true); + fact.setGroupTransactionEvents(groupTransactionEvents); + GatewaySender sender1 = fact.create("TKSender", 2); + RegionFactory factory = cache.createRegionFactory(RegionShortcut.PARTITION); + factory.addGatewaySenderId(sender1.getId()); + Set senders = cache.getGatewaySenders(); + assertEquals(senders.size(), 1); + GatewaySender gatewaySender = senders.iterator().next(); + assertThat(sender1.mustGroupTransactionEvents()) + .isEqualTo(gatewaySender.mustGroupTransactionEvents()); + } + + @Test + public void test_create_SerialGatewaySender_ThrowsException_when_GroupTransactionEvents_isTrue_and_DispatcherThreads_is_greaterThanOne() { + cache = new CacheFactory().set(MCAST_PORT, "0").create(); + GatewaySenderFactory fact = cache.createGatewaySenderFactory(); + fact.setParallel(false); + fact.setDispatcherThreads(2); + fact.setGroupTransactionEvents(true); + assertThatThrownBy(() -> fact.create("NYSender", 2)) + .isInstanceOf(GatewaySenderException.class) + .hasMessageContaining( + "SerialGatewaySender NYSender cannot be created with group transaction events set to true when dispatcher threads is greater than 1"); + } + + @Test + public void test_create_GatewaySender_ThrowsException_when_GroupTransactionEvents_isTrue_and_BatchConflation_is_enabled() { + cache = new CacheFactory().set(MCAST_PORT, "0").create(); + GatewaySenderFactory fact = cache.createGatewaySenderFactory(); + fact.setBatchConflationEnabled(true); + fact.setGroupTransactionEvents(true); + assertThatThrownBy(() -> fact.create("NYSender", 2)) + .isInstanceOf(GatewaySenderException.class) + .hasMessageContaining( + "GatewaySender NYSender cannot be created with both group transaction events set to true and batch conflation enabled"); + } + + @After + public void tearDown() { + if (this.cache != null) { + this.cache.close(); + } + } +} diff --git a/geode-wan-txgrouping/src/main/java/org/apache/geode/cache/wan/internal/txgrouping/parallel/TxGroupingParallelGatewaySenderQueue.java b/geode-wan-txgrouping/src/main/java/org/apache/geode/cache/wan/internal/txgrouping/parallel/TxGroupingParallelGatewaySenderQueue.java index 76e1a315c977..19c6afe17757 100644 --- a/geode-wan-txgrouping/src/main/java/org/apache/geode/cache/wan/internal/txgrouping/parallel/TxGroupingParallelGatewaySenderQueue.java +++ b/geode-wan-txgrouping/src/main/java/org/apache/geode/cache/wan/internal/txgrouping/parallel/TxGroupingParallelGatewaySenderQueue.java @@ -17,20 +17,25 @@ import static org.apache.geode.cache.wan.GatewaySender.GET_TRANSACTION_EVENTS_FROM_QUEUE_WAIT_TIME_MS; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Predicate; import org.jetbrains.annotations.NotNull; +import org.apache.geode.cache.CacheException; import org.apache.geode.cache.Region; import org.apache.geode.cache.TransactionId; +import org.apache.geode.internal.cache.BucketRegionQueue; import org.apache.geode.internal.cache.PartitionedRegion; import org.apache.geode.internal.cache.wan.AbstractGatewaySender; import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl; +import org.apache.geode.internal.cache.wan.parallel.BucketRegionQueueUnavailableException; import org.apache.geode.internal.cache.wan.parallel.ParallelGatewaySenderQueue; public class TxGroupingParallelGatewaySenderQueue extends ParallelGatewaySenderQueue { @@ -96,6 +101,36 @@ protected void postProcessBatch(final @NotNull PartitionedRegion partitionedRegi } } + protected List peekEventsWithTransactionId(PartitionedRegion prQ, int bucketId, + TransactionId transactionId) throws CacheException { + List objects; + BucketRegionQueue brq = getBucketRegionQueueByBucketId(prQ, bucketId); + + try { + Predicate hasTransactionIdPredicate = + getHasTransactionIdPredicate(transactionId); + Predicate isLastEventInTransactionPredicate = + getIsLastEventInTransactionPredicate(); + objects = + brq.getElementsMatching(hasTransactionIdPredicate, isLastEventInTransactionPredicate); + } catch (BucketRegionQueueUnavailableException e) { + // BucketRegionQueue unavailable. Can be due to the BucketRegionQueue being destroyed. + return Collections.emptyList(); + } + + return objects; // OFFHEAP: ok since callers are careful to do destroys on region queue after + // finished with peeked objects. + } + + private static Predicate getIsLastEventInTransactionPredicate() { + return x -> x.isLastEventInTransaction(); + } + + private static Predicate getHasTransactionIdPredicate( + TransactionId transactionId) { + return x -> transactionId.equals(x.getTransactionId()); + } + private Map getIncompleteTransactionsInBatch( List batch) { Map incompleteTransactionsInBatch = new HashMap<>(); diff --git a/geode-wan-txgrouping/src/main/java/org/apache/geode/cache/wan/internal/txgrouping/serial/TxGroupingSerialGatewaySenderQueue.java b/geode-wan-txgrouping/src/main/java/org/apache/geode/cache/wan/internal/txgrouping/serial/TxGroupingSerialGatewaySenderQueue.java index 59781be0af24..2e411f9827ef 100644 --- a/geode-wan-txgrouping/src/main/java/org/apache/geode/cache/wan/internal/txgrouping/serial/TxGroupingSerialGatewaySenderQueue.java +++ b/geode-wan-txgrouping/src/main/java/org/apache/geode/cache/wan/internal/txgrouping/serial/TxGroupingSerialGatewaySenderQueue.java @@ -17,13 +17,19 @@ import static org.apache.geode.cache.wan.GatewaySender.GET_TRANSACTION_EVENTS_FROM_QUEUE_WAIT_TIME_MS; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.function.Predicate; +import org.apache.geode.annotations.VisibleForTesting; +import org.apache.geode.cache.CacheException; import org.apache.geode.cache.CacheListener; +import org.apache.geode.cache.EntryNotFoundException; import org.apache.geode.cache.TransactionId; import org.apache.geode.cache.asyncqueue.AsyncEvent; import org.apache.geode.internal.cache.wan.AbstractGatewaySender; @@ -32,6 +38,22 @@ public class TxGroupingSerialGatewaySenderQueue extends SerialGatewaySenderQueue { + /** + * Contains the set of peekedIds that were peeked to complete a transaction + * inside a batch when groupTransactionEvents is set. + */ + protected final Set extraPeekedIds = ConcurrentHashMap.newKeySet(); + + /** + * Contains the set of peekedIds that were peeked to complete a transaction + * inside a batch when groupTransactionEvents is set and whose event has been + * removed from the queue because an ack has been received from the receiver. + * Elements from this set are deleted when the event with the previous id + * is removed. + */ + private final Set extraPeekedIdsRemovedButPreviousIdNotRemoved = + ConcurrentHashMap.newKeySet(); + public TxGroupingSerialGatewaySenderQueue( final AbstractGatewaySender abstractSender, final String regionName, final CacheListener listener, final boolean cleanQueues) { @@ -126,4 +148,110 @@ private List peekEventsWithTransactionId(TransactionId transact 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. + */ + protected List getElementsMatching(Predicate condition, + Predicate stopCondition, + long lastKey) { + GatewaySenderEventImpl event; + List elementsMatching = new ArrayList<>(); + + long currentKey = lastKey; + + while ((currentKey = inc(currentKey)) != getTailKey()) { + if (extraPeekedIds.contains(currentKey)) { + continue; + } + event = (GatewaySenderEventImpl) optimalGet(currentKey); + if (event == null) { + continue; + } + + if (condition.test(event)) { + elementsMatching.add(new KeyAndEventPair(currentKey, event)); + + if (stopCondition.test(event)) { + break; + } + } + } + + return elementsMatching; + } + + @Override + public synchronized void remove() throws CacheException { + if (peekedIds.isEmpty()) { + return; + } + boolean wasEmpty = lastDispatchedKey == lastDestroyedKey; + Long key = peekedIds.remove(); + boolean isExtraPeekedId = extraPeekedIds.contains(key); + if (!isExtraPeekedId) { + updateHeadKey(key); + lastDispatchedKey = key; + } else { + extraPeekedIdsRemovedButPreviousIdNotRemoved.add(key); + } + removeIndex(key); + // Remove the entry at that key with a callback arg signifying it is + // a WAN queue so that AbstractRegionEntry.destroy can get the value + // even if it has been evicted to disk. In the normal case, the + // AbstractRegionEntry.destroy only gets the value in the VM. + try { + this.region.localDestroy(key, WAN_QUEUE_TOKEN); + this.stats.decQueueSize(); + } catch (EntryNotFoundException ok) { + // this is acceptable because the conflation can remove entries + // out from underneath us. + if (logger.isDebugEnabled()) { + logger.debug( + "{}: Did not destroy entry at {} it was not there. It should have been removed by conflation.", + this, key); + } + } + + // For those extraPeekedIds removed that are consecutive to lastDispatchedKey: + // - Update lastDispatchedKey with them so that they are removed + // by the batch removal thread. + // - Update the head key with them. + // - Remove them from extraPeekedIds. + long tmpKey = lastDispatchedKey; + while (extraPeekedIdsRemovedButPreviousIdNotRemoved.contains(tmpKey = inc(tmpKey))) { + extraPeekedIdsRemovedButPreviousIdNotRemoved.remove(tmpKey); + extraPeekedIds.remove(tmpKey); + updateHeadKey(tmpKey); + lastDispatchedKey = tmpKey; + } + + if (wasEmpty) { + synchronized (this) { + notifyAll(); + } + } + + if (logger.isDebugEnabled()) { + logger.debug( + "{}: Destroyed entry at key {} setting the lastDispatched Key to {}. The last destroyed entry was {}", + this, key, this.lastDispatchedKey, this.lastDestroyedKey); + } + } + + public void resetLastPeeked() { + super.resetLastPeeked(); + extraPeekedIds.clear(); + } + + @Override + protected boolean skipPeekedKey(Long key) { + return extraPeekedIds.contains(key); + } + + @VisibleForTesting + Set getExtraPeekedIds() { + return Collections.unmodifiableSet(extraPeekedIds); + } } diff --git a/geode-wan-txgrouping/src/test/java/org/apache/geode/cache/wan/internal/txgrouping/parallel/TxGroupingParallelGatewaySenderQueueJUnitTest.java b/geode-wan-txgrouping/src/test/java/org/apache/geode/cache/wan/internal/txgrouping/parallel/TxGroupingParallelGatewaySenderQueueJUnitTest.java new file mode 100644 index 000000000000..72821a7696e4 --- /dev/null +++ b/geode-wan-txgrouping/src/test/java/org/apache/geode/cache/wan/internal/txgrouping/parallel/TxGroupingParallelGatewaySenderQueueJUnitTest.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.cache.wan.internal.txgrouping.parallel; + +import static java.util.Collections.singletonList; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.geode.CancelCriterion; +import org.apache.geode.cache.DataPolicy; +import org.apache.geode.cache.PartitionAttributesFactory; +import org.apache.geode.cache.Region; +import org.apache.geode.internal.cache.BucketRegionQueue; +import org.apache.geode.internal.cache.GemFireCacheImpl; +import org.apache.geode.internal.cache.PartitionedRegion; +import org.apache.geode.internal.cache.TXId; +import org.apache.geode.internal.cache.wan.AbstractGatewaySender; +import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl; +import org.apache.geode.internal.cache.wan.GatewaySenderStats; + +public class TxGroupingParallelGatewaySenderQueueJUnitTest { + + private AbstractGatewaySender sender; + + @Before + public void createParallelGatewaySenderQueue() { + GemFireCacheImpl cache = mock(GemFireCacheImpl.class); + sender = mock(AbstractGatewaySender.class); + CancelCriterion cancelCriterion = mock(CancelCriterion.class); + when(sender.getCancelCriterion()).thenReturn(cancelCriterion); + when(sender.getCache()).thenReturn(cache); + when(sender.getMaximumQueueMemory()).thenReturn(100); + when(sender.getLifeCycleLock()).thenReturn(new ReentrantReadWriteLock()); + when(sender.getId()).thenReturn(""); + } + + private void mockGatewaySenderStats() { + GatewaySenderStats stats = mock(GatewaySenderStats.class); + when(sender.getStatistics()).thenReturn(stats); + } + + @Test + public void peekGetsExtraEventsWhenMustGroupTransactionEventsAndNotAllEventsForTransactionsInMaxSizeBatch() + throws Exception { + + GatewaySenderEventImpl event1 = createGatewaySenderEventImpl(1, false); + GatewaySenderEventImpl event2 = createGatewaySenderEventImpl(2, false); + GatewaySenderEventImpl event3 = createGatewaySenderEventImpl(1, true); + GatewaySenderEventImpl event4 = createGatewaySenderEventImpl(2, true); + GatewaySenderEventImpl event5 = createGatewaySenderEventImpl(3, false); + GatewaySenderEventImpl event6 = createGatewaySenderEventImpl(3, true); + + Queue backingList = new LinkedList<>(); + backingList.add(event1); + backingList.add(event2); + backingList.add(event3); + backingList.add(event4); + backingList.add(event5); + backingList.add(event6); + + BucketRegionQueue bucketRegionQueue = mockBucketRegionQueue(backingList); + + TestableParallelGatewaySenderQueue queue = new TestableParallelGatewaySenderQueue(sender, + Collections.emptySet(), 0, 1); + queue.setMockedAbstractBucketRegionQueue(bucketRegionQueue); + + List peeked = queue.peek(3, 100); + assertEquals(4, peeked.size()); + List peekedAfter = queue.peek(3, 100); + assertEquals(2, peekedAfter.size()); + } + + @Test + public void peekGetsExtraEventsWhenMustGroupTransactionEventsAndNotAllEventsForTransactionsInBatchByTime() + throws Exception { + + GatewaySenderEventImpl event1 = createGatewaySenderEventImpl(1, false); + GatewaySenderEventImpl event2 = createGatewaySenderEventImpl(2, false); + GatewaySenderEventImpl event3 = createGatewaySenderEventImpl(1, true); + GatewaySenderEventImpl event4 = createGatewaySenderEventImpl(2, true); + GatewaySenderEventImpl event5 = createGatewaySenderEventImpl(3, false); + GatewaySenderEventImpl event6 = createGatewaySenderEventImpl(3, true); + + Queue backingList = new LinkedList<>(); + backingList.add(event1); + backingList.add(event2); + backingList.add(event3); + backingList.add(null); + backingList.add(event4); + backingList.add(event5); + backingList.add(event6); + + BucketRegionQueue bucketRegionQueue = mockBucketRegionQueue(backingList); + + TestableParallelGatewaySenderQueue queue = new TestableParallelGatewaySenderQueue(sender, + Collections.emptySet(), 0, 1); + queue.setMockedAbstractBucketRegionQueue(bucketRegionQueue); + + List peeked = queue.peek(-1, 1); + assertEquals(4, peeked.size()); + List peekedAfter = queue.peek(-1, 100); + assertEquals(2, peekedAfter.size()); + } + + @Test + public void peekEventsFromIncompleteTransactionsDoesNotThrowConcurrentModificationExceptionWhenCompletingTwoTransactions() { + mockGatewaySenderStats(); + GatewaySenderEventImpl event1 = createGatewaySenderEventImpl(1, false); + GatewaySenderEventImpl event2 = createGatewaySenderEventImpl(2, false); + GatewaySenderEventImpl event3 = createGatewaySenderEventImpl(1, true); + GatewaySenderEventImpl event4 = createGatewaySenderEventImpl(2, true); + + Queue backingList = new LinkedList<>(); + backingList.add(event3); + backingList.add(event4); + BucketRegionQueue bucketRegionQueue = mockBucketRegionQueue(backingList); + + TestableParallelGatewaySenderQueue queue = new TestableParallelGatewaySenderQueue(sender, + Collections.emptySet(), 0, 1); + queue.setMockedAbstractBucketRegionQueue(bucketRegionQueue); + + List batch = new ArrayList<>(Arrays.asList(event1, event2)); + PartitionedRegion mockBucketRegion = mockPR("bucketRegion"); + queue.postProcessBatch(mockBucketRegion, batch); + } + + + private GatewaySenderEventImpl createGatewaySenderEventImpl(int transactionId, + boolean isLastEventInTransaction) { + GatewaySenderEventImpl event = mock(GatewaySenderEventImpl.class); + when(event.getTransactionId()).thenReturn(new TXId(null, transactionId)); + when(event.makeHeapCopyIfOffHeap()).thenReturn(event); + when(event.isLastEventInTransaction()).thenReturn(isLastEventInTransaction); + return event; + } + + private PartitionedRegion mockPR(String name) { + PartitionedRegion region = mock(PartitionedRegion.class); + when(region.getFullPath()).thenReturn(name); + when(region.getPartitionAttributes()).thenReturn(new PartitionAttributesFactory<>().create()); + when(region.getTotalNumberOfBuckets()).thenReturn(113); + when(region.getDataPolicy()).thenReturn(DataPolicy.PARTITION); + return region; + } + + private BucketRegionQueue mockBucketRegionQueue(final Queue backingList) { + PartitionedRegion mockBucketRegion = mockPR("bucketRegion"); + // These next mocked return calls are for when peek is called. It ends up checking these on the + // mocked pr region + when(mockBucketRegion.getLocalMaxMemory()).thenReturn(100); + when(mockBucketRegion.size()).thenReturn(backingList.size()); + BucketRegionQueue bucketRegionQueue = mock(BucketRegionQueue.class); + when(bucketRegionQueue.getPartitionedRegion()).thenReturn(mockBucketRegion); + when(bucketRegionQueue.peek()).thenAnswer(invocation -> pollAndWaitIfNull(backingList)); + when(bucketRegionQueue.getElementsMatching(any(), any())) + .thenAnswer(invocation -> singletonList(getFirstNotNull(backingList))); + return bucketRegionQueue; + } + + private Object pollAndWaitIfNull(Queue queue) { + Object object = queue.poll(); + if (object == null) { + try { + Thread.sleep(10); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + return object; + } + + private Object getFirstNotNull(Queue queue) { + Object object = queue.poll(); + while (object == null) { + object = queue.poll(); + } + return object; + } + + private static class TestableParallelGatewaySenderQueue + extends TxGroupingParallelGatewaySenderQueue { + + private BucketRegionQueue mockedAbstractBucketRegionQueue; + + public TestableParallelGatewaySenderQueue(final AbstractGatewaySender sender, + final Set> userRegions, final int idx, final int nDispatcher) { + super(sender, userRegions, idx, nDispatcher, false); + } + + public void setMockedAbstractBucketRegionQueue(BucketRegionQueue mocked) { + mockedAbstractBucketRegionQueue = mocked; + } + + @Override + public boolean areLocalBucketQueueRegionsPresent() { + return true; + } + + @Override + protected PartitionedRegion getRandomShadowPR() { + return mockedAbstractBucketRegionQueue.getPartitionedRegion(); + } + + @Override + protected int getRandomPrimaryBucket(PartitionedRegion pr) { + return 0; + } + + @Override + protected BucketRegionQueue getBucketRegionQueueByBucketId(PartitionedRegion prQ, + int bucketId) { + return mockedAbstractBucketRegionQueue; + } + } + +} diff --git a/geode-wan-txgrouping/src/test/java/org/apache/geode/cache/wan/internal/txgrouping/serial/TxGroupingSerialGatewaySenderQueueJUnitTest.java b/geode-wan-txgrouping/src/test/java/org/apache/geode/cache/wan/internal/txgrouping/serial/TxGroupingSerialGatewaySenderQueueJUnitTest.java new file mode 100644 index 000000000000..145359696489 --- /dev/null +++ b/geode-wan-txgrouping/src/test/java/org/apache/geode/cache/wan/internal/txgrouping/serial/TxGroupingSerialGatewaySenderQueueJUnitTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.cache.wan.internal.txgrouping.serial; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.geode.CancelCriterion; +import org.apache.geode.cache.Cache; +import org.apache.geode.cache.Region; +import org.apache.geode.cache.asyncqueue.AsyncEvent; +import org.apache.geode.distributed.internal.InternalDistributedSystem; +import org.apache.geode.internal.cache.GemFireCacheImpl; +import org.apache.geode.internal.cache.InternalRegionFactory; +import org.apache.geode.internal.cache.LocalRegion; +import org.apache.geode.internal.cache.TXId; +import org.apache.geode.internal.cache.wan.AbstractGatewaySender; +import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl; +import org.apache.geode.internal.cache.wan.GatewaySenderStats; +import org.apache.geode.internal.cache.wan.serial.SerialGatewaySenderQueue; +import org.apache.geode.internal.statistics.DummyStatisticsRegistry; +import org.apache.geode.metrics.internal.NoopMeterRegistry; + +public class TxGroupingSerialGatewaySenderQueueJUnitTest { + + private static final String QUEUE_REGION = "queueRegion"; + + private AbstractGatewaySender sender; + Region region; + InternalRegionFactory regionFactory; + + @Before + public void setup() { + InternalDistributedSystem mockInternalDistributedSystem = mock(InternalDistributedSystem.class); + when(mockInternalDistributedSystem.getStatisticsManager()) + .thenReturn(new DummyStatisticsRegistry("", 0)); + + GemFireCacheImpl cache = mock(GemFireCacheImpl.class); + when(cache.getInternalDistributedSystem()).thenReturn(mockInternalDistributedSystem); + when(cache.getMeterRegistry()).thenReturn(new NoopMeterRegistry()); + + region = createLocalRegionMock(); + + CancelCriterion cancelCriterion = mock(CancelCriterion.class); + when(cache.getCancelCriterion()).thenReturn(cancelCriterion); + + sender = mock(AbstractGatewaySender.class); + + when(sender.getCancelCriterion()).thenReturn(cancelCriterion); + when(sender.getCache()).thenReturn(cache); + when(cache.getRegion(any())).thenReturn(region); + when(sender.getMaximumQueueMemory()).thenReturn(100); + when(sender.getLifeCycleLock()).thenReturn(new ReentrantReadWriteLock()); + when(sender.getId()).thenReturn(""); + when(sender.getStatistics()).thenReturn(mock(GatewaySenderStats.class)); + } + + @Test + public void peekGetsExtraEventsWhenMustGroupTransactionEventsAndNotAllEventsForTransactionsInMaxSizeBatch() { + TestableTxGroupingSerialGatewaySenderQueue queue = + new TestableTxGroupingSerialGatewaySenderQueue(sender, + QUEUE_REGION); + + List> peeked = queue.peek(3, 100); + assertEquals(4, peeked.size()); + List> peekedAfter = queue.peek(3, 100); + assertEquals(3, peekedAfter.size()); + } + + @Test + public void peekGetsExtraEventsWhenMustGroupTransactionEventsAndNotAllEventsForTransactionsInBatchByTime() { + GatewaySenderEventImpl event1 = createMockGatewaySenderEventImpl(1, false, region); + GatewaySenderEventImpl event2 = createMockGatewaySenderEventImpl(2, false, region); + GatewaySenderEventImpl event3 = createMockGatewaySenderEventImpl(1, true, region); + GatewaySenderEventImpl event4 = createMockGatewaySenderEventImpl(2, true, region); + TxGroupingSerialGatewaySenderQueue.KeyAndEventPair eventPair1 = + new SerialGatewaySenderQueue.KeyAndEventPair(0L, event1); + SerialGatewaySenderQueue.KeyAndEventPair eventPair2 = + new SerialGatewaySenderQueue.KeyAndEventPair(1L, event2); + SerialGatewaySenderQueue.KeyAndEventPair eventPair3 = + new SerialGatewaySenderQueue.KeyAndEventPair(2L, event3); + + TestableTxGroupingSerialGatewaySenderQueue realQueue = + new TestableTxGroupingSerialGatewaySenderQueue(sender, + QUEUE_REGION); + + TestableTxGroupingSerialGatewaySenderQueue queue = spy(realQueue); + + doAnswer(invocation -> eventPair1) + .doAnswer(invocation -> eventPair2) + .doAnswer(invocation -> eventPair3) + .doAnswer(invocation -> null) + .when(queue).peekAhead(); + + doAnswer(invocation -> Collections + .singletonList(new TxGroupingSerialGatewaySenderQueue.KeyAndEventPair(1L, event4))) + .when(queue).getElementsMatching(any(), any(), anyLong()); + + List> peeked = queue.peek(-1, 1); + assertEquals(4, peeked.size()); + } + + @Test + public void peekEventsFromIncompleteTransactionsDoesNotThrowConcurrentModificationExceptionWhenCompletingTwoTransactions() { + GatewaySenderEventImpl event1 = createMockGatewaySenderEventImpl(1, false, region); + GatewaySenderEventImpl event2 = createMockGatewaySenderEventImpl(2, false, region); + + TestableTxGroupingSerialGatewaySenderQueue queue = + new TestableTxGroupingSerialGatewaySenderQueue(sender, + QUEUE_REGION); + + @SuppressWarnings("unchecked") + List> batch = new ArrayList(Arrays.asList(event1, event2)); + queue.postProcessBatch(batch, 0); + } + + @Test + public void removeExtraPeekedEventDoesNotRemoveFromExtraPeekedIdsUntilPreviousEventIsRemoved() { + TestableTxGroupingSerialGatewaySenderQueue queue = + new TestableTxGroupingSerialGatewaySenderQueue(sender, + QUEUE_REGION); + List> peeked = queue.peek(3, -1); + assertEquals(4, peeked.size()); + assertThat(queue.getLastPeekedId()).isEqualTo(2); + assertThat(queue.getExtraPeekedIds().contains(5L)).isTrue(); + + + for (Object ignored : peeked) { + queue.remove(); + } + assertThat(queue.getExtraPeekedIds().contains(5L)).isTrue(); + + peeked = queue.peek(3, -1); + assertEquals(3, peeked.size()); + assertThat(queue.getExtraPeekedIds().contains(5L)).isTrue(); + + for (Object ignored : peeked) { + queue.remove(); + } + assertThat(queue.getExtraPeekedIds().contains(5L)).isFalse(); + } + + private GatewaySenderEventImpl createMockGatewaySenderEventImpl(int transactionId, + boolean isLastEventInTransaction, Region region) { + GatewaySenderEventImpl event = mock(GatewaySenderEventImpl.class); + when(event.getTransactionId()).thenReturn(new TXId(null, transactionId)); + when(event.makeHeapCopyIfOffHeap()).thenReturn(event); + when(event.isLastEventInTransaction()).thenReturn(isLastEventInTransaction); + when(event.getRegion()).thenReturn(region); + return event; + } + + private LocalRegion createLocalRegionMock() { + GatewaySenderEventImpl event1 = createMockGatewaySenderEventImpl(1, false, region); + GatewaySenderEventImpl event2 = createMockGatewaySenderEventImpl(2, false, region); + GatewaySenderEventImpl event3 = createMockGatewaySenderEventImpl(1, true, region); + GatewaySenderEventImpl event4 = createMockGatewaySenderEventImpl(3, true, region); + GatewaySenderEventImpl event5 = createMockGatewaySenderEventImpl(4, true, region); + GatewaySenderEventImpl event6 = createMockGatewaySenderEventImpl(2, true, region); + GatewaySenderEventImpl event7 = createMockGatewaySenderEventImpl(5, false, region); + + LocalRegion region = mock(LocalRegion.class); + + when(region.getValueInVMOrDiskWithoutFaultIn(0L)).thenReturn(event1); + when(region.getValueInVMOrDiskWithoutFaultIn(1L)).thenReturn(event2); + when(region.getValueInVMOrDiskWithoutFaultIn(2L)).thenReturn(event3); + when(region.getValueInVMOrDiskWithoutFaultIn(3L)).thenReturn(event4); + when(region.getValueInVMOrDiskWithoutFaultIn(4L)).thenReturn(event5); + when(region.getValueInVMOrDiskWithoutFaultIn(5L)).thenReturn(event6); + when(region.getValueInVMOrDiskWithoutFaultIn(6L)).thenReturn(event7); + + Map> map = new HashMap<>(); + map.put(0L, event1); + map.put(1L, event2); + map.put(2L, event3); + map.put(3L, event4); + map.put(4L, event5); + map.put(5L, event6); + map.put(6L, event7); + + when(region.keySet()).thenReturn(map.keySet()); + return region; + } + + private static class TestableTxGroupingSerialGatewaySenderQueue + extends TxGroupingSerialGatewaySenderQueue { + public TestableTxGroupingSerialGatewaySenderQueue(final AbstractGatewaySender sender, + String regionName) { + super(sender, regionName, null, false); + } + + @Override + protected void addOverflowStatisticsToMBean(Cache cache, AbstractGatewaySender sender) {} + + } +} diff --git a/geode-wan-txgrouping/src/test/resources/expected-pom.xml b/geode-wan-txgrouping/src/test/resources/expected-pom.xml index 52e2802f9a6e..0f8d1279a2bc 100644 --- a/geode-wan-txgrouping/src/test/resources/expected-pom.xml +++ b/geode-wan-txgrouping/src/test/resources/expected-pom.xml @@ -48,12 +48,12 @@ org.apache.geode - geode-wan + geode-logging runtime org.apache.geode - geode-logging + geode-membership runtime @@ -61,10 +61,20 @@ geode-serialization runtime + + org.apache.geode + geode-tcp-server + runtime + org.apache.geode geode-core runtime + + org.apache.geode + geode-wan + runtime + diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderOperationsDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderOperationsDUnitTest.java index 9a75d6b9a1fd..96dd0b3da570 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderOperationsDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderOperationsDUnitTest.java @@ -1086,12 +1086,10 @@ public void testParallelGWSenderUpdateAttrWhilePaused() throws Exception { vm4.invoke(() -> doPuts(getUniqueName() + "_PR", 5000)); - updateGroupTransactionEvents(true); updateBatchTimeInterval(200); validateRegionSizes(getUniqueName() + "_PR", 0, vm2, vm3); - checkGroupTransactionEvents(true); checkBatchTimeInterval(200); vm4.invoke(() -> resumeSender("ln")); @@ -1424,58 +1422,6 @@ private void updateBatchTimeInterval(int batchTimeInterval) { }); } - private void updateGroupTransactionEvents(boolean groupTransactionEvents) { - vm4.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - boolean paused = false; - if (sender.isRunning() && !sender.isPaused()) { - sender.pause(); - paused = true; - } - sender.setGroupTransactionEvents(groupTransactionEvents); - if (paused) { - sender.resume(); - } - }); - vm5.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - boolean paused = false; - if (sender.isRunning() && !sender.isPaused()) { - sender.pause(); - paused = true; - } - sender.setGroupTransactionEvents(groupTransactionEvents); - if (paused) { - sender.resume(); - } - }); - vm6.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - boolean paused = false; - if (sender.isRunning() && !sender.isPaused()) { - sender.pause(); - paused = true; - } - sender.setGroupTransactionEvents(groupTransactionEvents); - if (paused) { - sender.resume(); - } - }); - vm7.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - boolean paused = false; - if (sender.isRunning() && !sender.isPaused()) { - sender.pause(); - paused = true; - } - sender.setGroupTransactionEvents(groupTransactionEvents); - if (paused) { - sender.resume(); - } - }); - - } - private void updateGatewayEventFilters(List filters) { vm4.invoke(() -> { AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); @@ -1565,25 +1511,6 @@ private void checkBatchTimeInterval(int batchTimeInterval) { }); } - private void checkGroupTransactionEvents(boolean groupTransactionEvents) { - vm4.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - assertThat(sender.mustGroupTransactionEvents()).isEqualTo(groupTransactionEvents); - }); - vm5.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - assertThat(sender.mustGroupTransactionEvents()).isEqualTo(groupTransactionEvents); - }); - vm6.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - assertThat(sender.mustGroupTransactionEvents()).isEqualTo(groupTransactionEvents); - }); - vm7.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - assertThat(sender.mustGroupTransactionEvents()).isEqualTo(groupTransactionEvents); - }); - } - private void stopSenders() { vm4.invoke(() -> stopSender("ln")); vm5.invoke(() -> stopSender("ln")); diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPropagationDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPropagationDUnitTest.java index fe3a0e6615e3..9d808eefb70c 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPropagationDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPropagationDUnitTest.java @@ -17,8 +17,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import java.util.HashMap; -import java.util.Map; import java.util.Set; import junitparams.Parameters; @@ -34,11 +32,6 @@ import org.apache.geode.cache.wan.GatewaySender; import org.apache.geode.internal.cache.GemFireCacheImpl; import org.apache.geode.internal.cache.RegionQueue; -import org.apache.geode.internal.cache.execute.data.CustId; -import org.apache.geode.internal.cache.execute.data.Order; -import org.apache.geode.internal.cache.execute.data.OrderId; -import org.apache.geode.internal.cache.execute.data.Shipment; -import org.apache.geode.internal.cache.execute.data.ShipmentId; import org.apache.geode.internal.cache.wan.AbstractGatewaySender; import org.apache.geode.internal.cache.wan.BatchException70; import org.apache.geode.internal.cache.wan.WANTestBase; @@ -1242,113 +1235,6 @@ public void testParallelPropagationTxNotificationsNotSentToAllRegionMembersWhenA vm5.invoke(() -> WANTestBase.validateEmptyBucketToTempQueueMap("ln")); } - @Test - public void testPartitionedParallelPropagationWithGroupTransactionEventsAndMixOfEventsInAndNotInTransactions() - throws Exception { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2, vm3); - createReceiverInVMs(vm2, vm3); - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - vm4.invoke(() -> setNumDispatcherThreadsForTheRun(2)); - vm5.invoke(() -> setNumDispatcherThreadsForTheRun(2)); - vm6.invoke(() -> setNumDispatcherThreadsForTheRun(2)); - vm7.invoke(() -> setNumDispatcherThreadsForTheRun(2)); - - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, true)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, true)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, true)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, true)); - - vm4.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 2, 10, - isOffHeap())); - vm5.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 2, 10, - isOffHeap())); - vm6.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 2, 10, - isOffHeap())); - vm7.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 2, 10, - isOffHeap())); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - vm2.invoke(() -> createCustomerOrderShipmentPartitionedRegion(null, 1, 8, isOffHeap())); - vm3.invoke(() -> createCustomerOrderShipmentPartitionedRegion(null, 1, 8, isOffHeap())); - - int customers = 4; - - int transactionsPerCustomer = 1000; - final Map keyValuesInTransactions = new HashMap<>(); - 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); - ShipmentId shipmentId1 = new ShipmentId(i, orderId); - ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); - ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); - keyValuesInTransactions.put(orderId, new Order()); - keyValuesInTransactions.put(shipmentId1, new Shipment()); - keyValuesInTransactions.put(shipmentId2, new Shipment()); - keyValuesInTransactions.put(shipmentId3, new Shipment()); - } - } - - int ordersPerCustomerNotInTransactions = 1000; - - final Map keyValuesNotInTransactions = new HashMap<>(); - for (int custId = 0; custId < customers; custId++) { - for (int i = 0; i < ordersPerCustomerNotInTransactions; i++) { - CustId custIdObject = new CustId(custId); - OrderId orderId = new OrderId(i + transactionsPerCustomer * customers, custIdObject); - keyValuesNotInTransactions.put(orderId, new Order()); - } - } - - // eventsPerTransaction is 1 (orders) + 3 (shipments) - int eventsPerTransaction = 4; - AsyncInvocation inv1 = - vm7.invokeAsync( - () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions, - eventsPerTransaction)); - - AsyncInvocation inv2 = - vm6.invokeAsync( - () -> WANTestBase.putGivenKeyValue(orderRegionName, keyValuesNotInTransactions)); - - inv1.await(); - inv2.await(); - - int entries = - ordersPerCustomerNotInTransactions * customers + transactionsPerCustomer * customers; - - vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - vm6.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - vm7.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - - vm2.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - vm3.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - - vm4.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - vm5.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - vm6.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - vm7.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - - vm7.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm6.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - } - private static RegionShortcut[] getRegionShortcuts() { return new RegionShortcut[] {RegionShortcut.PARTITION, RegionShortcut.PARTITION_PERSISTENT}; } diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java index 9b91b58353b9..3d1008cb7bda 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java @@ -17,7 +17,6 @@ import static org.apache.geode.internal.Assert.fail; import static org.apache.geode.test.awaitility.GeodeAwaitility.await; import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException; -import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; @@ -35,12 +34,6 @@ import org.apache.geode.cache.client.ClientCacheFactory; import org.apache.geode.cache.client.ClientRegionShortcut; import org.apache.geode.cache.client.internal.PoolImpl; -import org.apache.geode.internal.cache.execute.data.CustId; -import org.apache.geode.internal.cache.execute.data.Customer; -import org.apache.geode.internal.cache.execute.data.Order; -import org.apache.geode.internal.cache.execute.data.OrderId; -import org.apache.geode.internal.cache.execute.data.Shipment; -import org.apache.geode.internal.cache.execute.data.ShipmentId; import org.apache.geode.internal.cache.wan.WANTestBase; import org.apache.geode.test.dunit.AsyncInvocation; import org.apache.geode.test.dunit.VM; @@ -322,496 +315,6 @@ public void testPartitionedRegionParallelPropagation_AfterDispatch_NoRedundancy( vm2.invoke(() -> WANTestBase.checkGatewayReceiverStats(10, NUM_PUTS, NUM_PUTS)); } - @Test - public void testPRParallelPropagationWithoutGroupTransactionEventsSendsBatchesWithIncompleteTransactions() { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - createReceiverInVMs(vm2); - - createSenders(lnPort, false); - - createReceiverCustomerOrderShipmentPR(vm2); - - createSenderCustomerOrderShipmentPRs(vm4); - createSenderCustomerOrderShipmentPRs(vm5); - createSenderCustomerOrderShipmentPRs(vm6); - createSenderCustomerOrderShipmentPRs(vm7); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - final Map custKeyValue = new HashMap<>(); - int intCustId = 1; - CustId custId = new CustId(intCustId); - custKeyValue.put(custId, new Customer()); - vm4.invoke(() -> WANTestBase.putGivenKeyValue(customerRegionName, custKeyValue)); - - int transactions = 3; - final Map keyValues = new HashMap<>(); - for (int i = 0; i < transactions; i++) { - OrderId orderId = new OrderId(i, custId); - ShipmentId shipmentId1 = new ShipmentId(i, orderId); - ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); - ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); - keyValues.put(orderId, new Order()); - keyValues.put(shipmentId1, new Shipment()); - keyValues.put(shipmentId2, new Shipment()); - keyValues.put(shipmentId3, new Shipment()); - } - int eventsPerTransaction = 4; - vm4.invoke(() -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValues, - eventsPerTransaction)); - - int entries = (transactions * eventsPerTransaction) + 1; - - vm4.invoke(() -> WANTestBase.validateRegionSize(customerRegionName, 1)); - vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, transactions)); - vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, transactions * 3)); - - ArrayList v4List = - (ArrayList) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v5List = - (ArrayList) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v6List = - (ArrayList) vm6.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v7List = - (ArrayList) vm7.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - - // queue size: - assertEquals(0, v4List.get(0) + v5List.get(0) + v6List.get(0) + v7List.get(0)); - // eventsReceived: - assertEquals(entries, v4List.get(1) + v5List.get(1) + v6List.get(1) + v7List.get(1)); - // events queued: - assertEquals(entries, v4List.get(2) + v5List.get(2) + v6List.get(2) + v7List.get(2)); - // events distributed: - assertEquals(entries, v4List.get(3) + v5List.get(3) + v6List.get(3) + v7List.get(3)); - // batches distributed: - assertEquals(2, v4List.get(4) + v5List.get(4) + v6List.get(4) + v7List.get(4)); - // batches redistributed: - assertEquals(0, v4List.get(5) + v5List.get(5) + v6List.get(5) + v7List.get(5)); - } - - @Test - public void testPRParallelPropagationWithGroupTransactionEventsWithoutBatchRedistributionSendsBatchesWithCompleteTransactions_SeveralClients() { - testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions_SeveralClients( - false); - } - - @Test - public void testPRParallelPropagationWithGroupTransactionEventsWithBatchRedistributionSendsBatchesWithCompleteTransactions_SeveralClients() { - testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions_SeveralClients( - true); - } - - public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions_SeveralClients( - boolean isBatchesRedistributed) { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - - if (!isBatchesRedistributed) { - createReceiverInVMs(vm2); - } - - createSenders(lnPort, true); - - createReceiverCustomerOrderShipmentPR(vm2); - - createSenderCustomerOrderShipmentPRs(vm4); - createSenderCustomerOrderShipmentPRs(vm5); - createSenderCustomerOrderShipmentPRs(vm6); - createSenderCustomerOrderShipmentPRs(vm7); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - int clients = 4; - int transactions = 300; - // batchSize is 10. Each transaction will contain 1 order + 3 shipments = 4 events. - // As a result, all batches will contain extra events to complete the - // transactions it will deliver. - int shipmentsPerTransaction = 3; - - final List> customerData = new ArrayList<>(clients); - for (int intCustId = 0; intCustId < clients; intCustId++) { - final Map custKeyValue = new HashMap<>(); - CustId custId = new CustId(intCustId); - custKeyValue.put(custId, new Customer()); - customerData.add(new HashMap()); - vm4.invoke(() -> WANTestBase.putGivenKeyValue(customerRegionName, custKeyValue)); - - for (int i = 0; i < transactions; i++) { - OrderId orderId = new OrderId(i, custId); - customerData.get(intCustId).put(orderId, new Order()); - for (int j = 0; j < shipmentsPerTransaction; j++) { - customerData.get(intCustId).put(new ShipmentId(i + j, orderId), new Shipment()); - } - } - } - - List> asyncInvocations = new ArrayList<>(clients); - - int eventsPerTransaction = shipmentsPerTransaction + 1; - for (int i = 0; i < clients; i++) { - final int intCustId = i; - AsyncInvocation asyncInvocation = - vm4.invokeAsync(() -> WANTestBase.doOrderAndShipmentPutsInsideTransactions( - customerData.get(intCustId), - eventsPerTransaction)); - asyncInvocations.add(asyncInvocation); - } - - try { - for (AsyncInvocation asyncInvocation : asyncInvocations) { - asyncInvocation.await(); - } - } catch (InterruptedException e) { - fail("Interrupted"); - } - - vm4.invoke(() -> WANTestBase.validateRegionSize(customerRegionName, clients)); - vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, transactions * clients)); - vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, - transactions * shipmentsPerTransaction * clients)); - - if (isBatchesRedistributed) { - // wait for batches to be redistributed and then start the receiver - vm4.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0)); - createReceiverInVMs(vm2); - } - - // Check that all entries have been written in the receiver - vm2.invoke( - () -> WANTestBase.validateRegionSize(customerRegionName, clients)); - vm2.invoke( - () -> WANTestBase.validateRegionSize(orderRegionName, transactions * clients)); - vm2.invoke( - () -> WANTestBase.validateRegionSize(shipmentRegionName, - shipmentsPerTransaction * transactions * clients)); - - checkQueuesAreEmptyAndOnlyCompleteTransactionsAreReplicated(isBatchesRedistributed); - } - - private void checkQueuesAreEmptyAndOnlyCompleteTransactionsAreReplicated( - boolean isBatchesRedistributed) { - ArrayList v4List = - (ArrayList) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v5List = - (ArrayList) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v6List = - (ArrayList) vm6.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v7List = - (ArrayList) vm7.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - - // queue size: - assertEquals(0, v4List.get(0) + v5List.get(0) + v6List.get(0) + v7List.get(0)); - // batches redistributed: - int batchesRedistributed = v4List.get(5) + v5List.get(5) + v6List.get(5) + v7List.get(5); - if (isBatchesRedistributed) { - assertThat(batchesRedistributed).isGreaterThan(0); - } else { - assertThat(batchesRedistributed).isEqualTo(0); - } - // batches with incomplete transactions - assertThat(v4List.get(13) + v5List.get(13) + v6List.get(13) + v7List.get(7)).isEqualTo(0); - - vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm6.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm7.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - } - - @Test - public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions() { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - createReceiverInVMs(vm2); - - createSenders(lnPort, true); - - createReceiverCustomerOrderShipmentPR(vm2); - - createSenderCustomerOrderShipmentPRs(vm4); - createSenderCustomerOrderShipmentPRs(vm5); - createSenderCustomerOrderShipmentPRs(vm6); - createSenderCustomerOrderShipmentPRs(vm7); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - final Map custKeyValue = new HashMap<>(); - int intCustId = 1; - CustId custId = new CustId(intCustId); - custKeyValue.put(custId, new Customer()); - vm4.invoke(() -> WANTestBase.putGivenKeyValue(customerRegionName, custKeyValue)); - - int transactions = 3; - final Map keyValues = new HashMap<>(); - for (int i = 0; i < transactions; i++) { - OrderId orderId = new OrderId(i, custId); - ShipmentId shipmentId1 = new ShipmentId(i, orderId); - ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); - ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); - keyValues.put(orderId, new Order()); - keyValues.put(shipmentId1, new Shipment()); - keyValues.put(shipmentId2, new Shipment()); - keyValues.put(shipmentId3, new Shipment()); - } - - // 3 transactions of 4 events each are sent so that the batch would - // initially contain the first 2 transactions complete and the first - // 2 events of the last transaction (10 entries). - // As --group-transaction-events is configured in the senders, the remaining - // 2 events of the last transaction are added to the batch which makes - // that only one batch of 12 events is sent. - int eventsPerTransaction = 4; - vm4.invoke(() -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValues, - eventsPerTransaction)); - - int entries = (transactions * eventsPerTransaction) + 1; - - vm4.invoke(() -> WANTestBase.validateRegionSize(customerRegionName, 1)); - vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, transactions)); - vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, transactions * 3)); - - ArrayList v4List = - (ArrayList) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v5List = - (ArrayList) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v6List = - (ArrayList) vm6.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v7List = - (ArrayList) vm7.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - - // queue size: - assertEquals(0, v4List.get(0) + v5List.get(0) + v6List.get(0) + v7List.get(0)); - // eventsReceived: - assertEquals(entries, v4List.get(1) + v5List.get(1) + v6List.get(1) + v7List.get(1)); - // events queued: - assertEquals(entries, v4List.get(2) + v5List.get(2) + v6List.get(2) + v7List.get(2)); - // events distributed: - assertEquals(entries, v4List.get(3) + v5List.get(3) + v6List.get(3) + v7List.get(3)); - // batches distributed: - assertEquals(1, v4List.get(4) + v5List.get(4) + v6List.get(4) + v7List.get(4)); - // batches redistributed: - assertEquals(0, v4List.get(5) + v5List.get(5) + v6List.get(5) + v7List.get(5)); - // events not queued conflated: - assertEquals(0, v4List.get(7) + v5List.get(7) + v6List.get(7) + v7List.get(7)); - // batches with incomplete transactions - assertEquals(0, (int) v4List.get(13)); - - } - - @Test - public void testPRParallelPropagationWithGroupTransactionEventsWithIncompleteTransactions() { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - createReceiverInVMs(vm2); - - int dispThreads = 2; - createSenderInVm(lnPort, vm4, dispThreads); - - createReceiverPR(vm2, 0); - - createSenderPRInVM(0, vm4); - - startSenderInVMs("ln", vm4); - - // Adding events in transactions - // Transactions will contain objects assigned to different buckets but given that there is only - // one server, there will be no TransactionDataNotCollocatedException. - // With this and by using more than one dispatcher thread, we will provoke that - // it will be impossible for the batches to have complete transactions as some - // events for a transaction will be handled by one dispatcher thread and some other events by - // another thread. - final Map keyValue = new HashMap<>(); - int entries = 30; - for (int i = 0; i < entries; i++) { - keyValue.put(i, i); - } - - int entriesPerTransaction = 3; - vm4.invoke( - () -> WANTestBase.doPutsInsideTransactions(testName, keyValue, entriesPerTransaction)); - - vm4.invoke(() -> WANTestBase.validateRegionSize(testName, entries)); - - ArrayList v4List = - (ArrayList) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - - // The number of batches will be 4 because each - // dispatcher thread (there are 2) will send half the number of entries, - // each on 2 batches. - int batches = 4; - // queue size: - assertEquals(0, (int) v4List.get(0)); - // eventsReceived: - assertEquals(entries, (int) v4List.get(1)); - // events queued: - assertEquals(entries, (int) v4List.get(2)); - // events distributed: - assertEquals(entries, (int) v4List.get(3)); - // batches distributed: - assertEquals(batches, (int) v4List.get(4)); - // batches redistributed: - assertEquals(0, (int) v4List.get(5)); - // events not queued conflated: - assertEquals(0, (int) v4List.get(7)); - // batches with incomplete transactions - assertEquals(batches, (int) v4List.get(13)); - - vm2.invoke(() -> WANTestBase.checkGatewayReceiverStats(batches, entries, entries)); - } - - - @Test - public void testPRParallelPropagationWithBatchRedistWithoutGroupTransactionEventsSendsBatchesWithIncompleteTransactions() { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - - createSenders(lnPort, false); - - createSenderCustomerOrderShipmentPRs(vm4); - - startSenderInVMs("ln", vm4); - - final Map custKeyValue = new HashMap<>(); - int intCustId = 1; - CustId custId = new CustId(intCustId); - custKeyValue.put(custId, new Customer()); - vm4.invoke(() -> WANTestBase.putGivenKeyValue(customerRegionName, custKeyValue)); - - int transactions = 6; - final Map keyValues = new HashMap<>(); - for (int i = 0; i < transactions; i++) { - OrderId orderId = new OrderId(i, custId); - ShipmentId shipmentId1 = new ShipmentId(i, orderId); - ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); - ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); - keyValues.put(orderId, new Order()); - keyValues.put(shipmentId1, new Shipment()); - keyValues.put(shipmentId2, new Shipment()); - keyValues.put(shipmentId3, new Shipment()); - } - int eventsPerTransaction = 4; - vm4.invoke(() -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValues, - eventsPerTransaction)); - - int entries = (transactions * eventsPerTransaction) + 1; - - createReceiverCustomerOrderShipmentPR(vm2); - - vm4.invoke(() -> WANTestBase.validateRegionSize(customerRegionName, 1)); - vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, transactions)); - vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, transactions * 3)); - - // wait for batches to be redistributed and then start the receiver - vm4.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0)); - - createReceiverInVMs(vm2); - - ArrayList v4List = - (ArrayList) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - - // queue size: - assertEquals(0, (int) v4List.get(0)); - // events received: - assertEquals(entries, (int) v4List.get(1)); - // events queued: - assertEquals(entries, (int) v4List.get(2)); - // events distributed: - assertEquals(entries, (int) v4List.get(3)); - // batches distributed: - assertEquals(3, (int) v4List.get(4)); - // batches redistributed: - assertTrue("Batch was not redistributed", (v4List.get(5)) > 0); - } - - @Test - public void testPRParallelPropagationWithBatchRedistWithGroupTransactionEventsSendsBatchesWithCompleteTransactions() { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - - createSenders(lnPort, true); - - createReceiverCustomerOrderShipmentPR(vm2); - - createSenderCustomerOrderShipmentPRs(vm4); - - startSenderInVMs("ln", vm4); - - - final Map custKeyValue = new HashMap<>(); - int intCustId = 1; - CustId custId = new CustId(intCustId); - custKeyValue.put(custId, new Customer()); - vm4.invoke(() -> WANTestBase.putGivenKeyValue(customerRegionName, custKeyValue)); - - int transactions = 6; - final Map keyValues = new HashMap<>(); - for (int i = 0; i < transactions; i++) { - OrderId orderId = new OrderId(i, custId); - ShipmentId shipmentId1 = new ShipmentId(i, orderId); - ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); - ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); - keyValues.put(orderId, new Order()); - keyValues.put(shipmentId1, new Shipment()); - keyValues.put(shipmentId2, new Shipment()); - keyValues.put(shipmentId3, new Shipment()); - } - - // 6 transactions of 4 events each are sent so that the first batch - // would initially contain the first 2 transactions complete and the first - // 2 events of the next transaction (10 entries). - // As --group-transaction-events is configured in the senders, the remaining - // 2 events of the second transaction are added to the batch which makes - // that the first batch is sent with 12 events. The same happens with the - // second batch which will contain 12 events too. - int eventsPerTransaction = 4; - vm4.invoke(() -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValues, - eventsPerTransaction)); - - int entries = (transactions * eventsPerTransaction) + 1; - - vm4.invoke(() -> WANTestBase.validateRegionSize(customerRegionName, 1)); - vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, transactions)); - vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, transactions * 3)); - - // wait for batches to be redistributed and then start the receiver - vm4.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0)); - - createReceiverInVMs(vm2); - - ArrayList v4List = - (ArrayList) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - - // queue size: - assertEquals(0, (int) v4List.get(0)); - // events received: - assertEquals(entries, (int) v4List.get(1)); - // events queued: - assertEquals(entries, (int) v4List.get(2)); - // events distributed: - assertEquals(entries, (int) v4List.get(3)); - // batches distributed: - assertEquals(2, (int) v4List.get(4)); - // batches redistributed: - assertTrue("Batch was not redistributed", (v4List.get(5)) > 0); - // events not queued conflated: - assertEquals(0, (int) v4List.get(7)); - } - @Test public void testPartitionedRegionParallelPropagation_AfterDispatch_Redundancy_3() { Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); @@ -968,77 +471,6 @@ public void testParallelPropagationHA() throws Exception { vm2.invoke(() -> WANTestBase.checkGatewayReceiverStatsHA(NUM_PUTS, 1000, 1000)); } - @Category({WanTest.class}) - @Test - public void testParallelPropagationHAWithGroupTransactionEvents() throws Exception { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - - createReceiverPR(vm2, 0); - - createReceiverInVMs(vm2); - - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - createSenderPRs(3); - - int batchSize = 9; - boolean groupTransactionEvents = true; - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - AsyncInvocation inv1 = - vm5.invokeAsync(() -> WANTestBase.doTxPutsWithRetryIfError(testName, 2, 1000, 0)); - - vm2.invoke(() -> await() - .untilAsserted(() -> assertTrue("Waiting for some batches to be received", - getRegionSize(testName) > 40))); - AsyncInvocation inv3 = vm4.invokeAsync(() -> WANTestBase.killSender()); - inv1.await(); - inv3.await(); - - vm2.invoke(() -> WANTestBase.validateRegionSize(testName, 2000)); - - ArrayList v5List = - (ArrayList) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v6List = - (ArrayList) vm6.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - ArrayList v7List = - (ArrayList) vm7.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - - assertEquals(0, v5List.get(0) + v6List.get(0) + v7List.get(0)); // queue size - int receivedEvents = v5List.get(1) + v6List.get(1) + v7List.get(1); - // We may see two retried events (as transactions are made of 2 events) on all members due to - // the kill - assertTrue("Received " + receivedEvents, - 6000 <= receivedEvents && 6006 >= receivedEvents); // eventsReceived - int queuedEvents = v5List.get(2) + v6List.get(2) + v7List.get(2); - assertTrue("Queued " + queuedEvents, - 6000 <= queuedEvents && 6006 >= queuedEvents); // eventsQueued - assertEquals(0, v5List.get(5) + v6List.get(5) + v7List.get(5)); // batches redistributed - - // batchesReceived is equal to numberOfEntries/(batchSize+1) - // As transactions are 2 events long, for each batch it will always be necessary to - // add one more entry to the 9 events batch in order to have complete transactions in the batch. - int batchesReceived = (1000 + 1000) / (batchSize + 1); - vm2.invoke(() -> WANTestBase.checkGatewayReceiverStatsHA(batchesReceived, 2000, 2000)); - } - - /** * 1 region and sender configured on local site and 1 region and a receiver configured on remote * site. Puts to the local region are in progress. Remote region is destroyed in the middle. @@ -1366,18 +798,6 @@ protected void createReceiverPR(VM vm, int redundancy) { () -> WANTestBase.createPartitionedRegion(testName, null, redundancy, 10, isOffHeap())); } - protected void createReceiverCustomerOrderShipmentPR(VM vm) { - vm.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion(null, 0, 10, - isOffHeap())); - } - - protected void createSenderCustomerOrderShipmentPRs(VM vm) { - vm.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 0, 10, - isOffHeap())); - } - protected void createSenderPRs(int redundancy) { vm4.invoke( () -> WANTestBase.createPartitionedRegion(testName, "ln", redundancy, 10, isOffHeap())); @@ -1403,6 +823,15 @@ protected void startPausedSenders() { vm7.invoke(() -> pauseSender("ln")); } + protected void createSenders(Integer lnPort) { + createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); + + vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true)); + vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true)); + vm6.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true)); + vm7.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true)); + } + protected void createSendersWithConflation(Integer lnPort) { createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); @@ -1412,36 +841,11 @@ protected void createSendersWithConflation(Integer lnPort) { vm7.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, true, false, null, true)); } - protected void createSenderInVm(Integer lnPort, VM vm, - int dispatcherThreads) { - createCacheInVMs(lnPort, vm); - vm.invoke(() -> WANTestBase.setNumDispatcherThreadsForTheRun(dispatcherThreads)); - vm.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, - true)); - } - protected void createSenderInVm(Integer lnPort, VM vm) { createCacheInVMs(lnPort, vm); vm.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true)); } - protected void createSenders(Integer lnPort, boolean groupTransactionEvents) { - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm6.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm7.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, false, null, true, - groupTransactionEvents)); - } - - protected void createSenders(Integer lnPort) { - createSenders(lnPort, false); - } - private void verifyConflationIndexesSize(String senderId, int expectedSize, VM... vms) { int actualSize = 0; for (VM vm : vms) { diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderAlterOperationsDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderAlterOperationsDUnitTest.java index c734d0eb82f6..7ba96e582c68 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderAlterOperationsDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderAlterOperationsDUnitTest.java @@ -643,33 +643,6 @@ private void updateBatchTimeInterval(int batchTimeInterval) { }); } - private void updateGroupTransactionEvents(boolean groupTransactionEvents) { - vm4.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - boolean paused = false; - if (sender.isRunning() && !sender.isPaused()) { - sender.pause(); - paused = true; - } - sender.setGroupTransactionEvents(groupTransactionEvents); - if (paused) { - sender.resume(); - } - }); - vm5.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - boolean paused = false; - if (sender.isRunning() && !sender.isPaused()) { - sender.pause(); - paused = true; - } - sender.setGroupTransactionEvents(groupTransactionEvents); - if (paused) { - sender.resume(); - } - }); - } - private void updateGatewayEventFilters(List filters) { vm4.invoke(() -> { AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); @@ -719,19 +692,6 @@ private void checkBatchTimeInterval(int batchTimeInterval) { }); } - private void checkGroupTransactionEvents(boolean groupTransactionEvents) { - vm4.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - assertThat(sender.mustGroupTransactionEvents()).isEqualTo(groupTransactionEvents); - }); - vm5.invoke(() -> { - AbstractGatewaySender sender = (AbstractGatewaySender) cache.getGatewaySender("ln"); - assertThat(sender.mustGroupTransactionEvents()).isEqualTo(groupTransactionEvents); - }); - - } - - public static class MyGatewayEventFilter implements GatewayEventFilter, Serializable { String Id = "MyGatewayEventFilter"; diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANPropagation_PartitionedRegionDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANPropagation_PartitionedRegionDUnitTest.java index 199076e110e2..8415a79fc096 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANPropagation_PartitionedRegionDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANPropagation_PartitionedRegionDUnitTest.java @@ -14,8 +14,6 @@ */ package org.apache.geode.internal.cache.wan.serial; -import java.util.HashMap; -import java.util.Map; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -23,11 +21,6 @@ import org.apache.geode.CancelException; import org.apache.geode.cache.CacheClosedException; import org.apache.geode.internal.cache.ForceReattemptException; -import org.apache.geode.internal.cache.execute.data.CustId; -import org.apache.geode.internal.cache.execute.data.Order; -import org.apache.geode.internal.cache.execute.data.OrderId; -import org.apache.geode.internal.cache.execute.data.Shipment; -import org.apache.geode.internal.cache.execute.data.ShipmentId; import org.apache.geode.internal.cache.wan.WANTestBase; import org.apache.geode.test.dunit.AsyncInvocation; import org.apache.geode.test.dunit.IgnoredException; @@ -413,112 +406,4 @@ public void testPartitionedSerialPropagationWithParallelThreads() { vm2.invoke(() -> WANTestBase.validateRegionSize(getTestMethodName() + "_PR", 1000)); vm3.invoke(() -> WANTestBase.validateRegionSize(getTestMethodName() + "_PR", 1000)); } - - @Test - public void testPartitionedSerialPropagationWithGroupTransactionEventsAndMixOfEventsInAndNotInTransactions() - throws Exception { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2, vm3); - createReceiverInVMs(vm2, vm3); - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - vm4.invoke(() -> setNumDispatcherThreadsForTheRun(1)); - vm5.invoke(() -> setNumDispatcherThreadsForTheRun(1)); - vm6.invoke(() -> setNumDispatcherThreadsForTheRun(1)); - vm7.invoke(() -> setNumDispatcherThreadsForTheRun(1)); - - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, true)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, true)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, true)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, true)); - - - vm4.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 2, 10, - isOffHeap())); - vm5.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 2, 10, - isOffHeap())); - vm6.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 2, 10, - isOffHeap())); - vm7.invoke( - () -> WANTestBase.createCustomerOrderShipmentPartitionedRegion("ln", 2, 10, - isOffHeap())); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - vm2.invoke(() -> createCustomerOrderShipmentPartitionedRegion(null, 1, 8, isOffHeap())); - vm3.invoke(() -> createCustomerOrderShipmentPartitionedRegion(null, 1, 8, isOffHeap())); - - int customers = 4; - - int transactionsPerCustomer = 1000; - final Map keyValuesInTransactions = new HashMap<>(); - 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); - ShipmentId shipmentId1 = new ShipmentId(i, orderId); - ShipmentId shipmentId2 = new ShipmentId(i + 1, orderId); - ShipmentId shipmentId3 = new ShipmentId(i + 2, orderId); - keyValuesInTransactions.put(orderId, new Order()); - keyValuesInTransactions.put(shipmentId1, new Shipment()); - keyValuesInTransactions.put(shipmentId2, new Shipment()); - keyValuesInTransactions.put(shipmentId3, new Shipment()); - } - } - - int ordersPerCustomerNotInTransactions = 1000; - - final Map keyValuesNotInTransactions = new HashMap<>(); - for (int custId = 0; custId < customers; custId++) { - for (int i = 0; i < ordersPerCustomerNotInTransactions; i++) { - CustId custIdObject = new CustId(custId); - OrderId orderId = new OrderId(i + transactionsPerCustomer * customers, custIdObject); - keyValuesNotInTransactions.put(orderId, new Order()); - } - } - - // eventsPerTransaction is 1 (orders) + 3 (shipments) - int eventsPerTransaction = 4; - AsyncInvocation inv1 = - vm7.invokeAsync( - () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions, - eventsPerTransaction)); - - AsyncInvocation inv2 = - vm6.invokeAsync( - () -> WANTestBase.putGivenKeyValue(orderRegionName, keyValuesNotInTransactions)); - - inv1.await(); - inv2.await(); - - int entries = - ordersPerCustomerNotInTransactions * customers + transactionsPerCustomer * customers; - - vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - vm6.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - vm7.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - - vm2.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - vm3.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries)); - - vm4.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - vm5.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - vm6.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - vm7.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - - vm7.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm6.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln")); - } } diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java index 1a8cabe65404..74e5bc6dee44 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java @@ -17,13 +17,9 @@ import static org.apache.geode.test.awaitility.GeodeAwaitility.await; import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException; import static org.apache.geode.test.dunit.Wait.pause; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import org.junit.Test; @@ -90,361 +86,6 @@ public void testReplicatedSerialPropagation() { } - @Test - public void testReplicatedSerialPropagationWithoutGroupTransactionEventsSendsBatchesWithIncompleteTransactions() { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - vm2.invoke(WANTestBase::createReceiver); - - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - boolean groupTransactionEvents = false; - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - - vm2.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", null, isOffHeap())); - - startSenderInVMs("ln", vm4, vm5); - - vm4.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm5.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm6.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm7.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - - final Map keyValues = new HashMap(); - int entries = 12; - for (int i = 0; i < entries; i++) { - keyValues.put(i, i + "_Value"); - } - int eventsPerTransaction = 3; - vm5.invoke(() -> WANTestBase.doPutsInsideTransactions(testName + "_RR", keyValues, - eventsPerTransaction)); - - vm2.invoke(() -> WANTestBase.validateRegionSize(testName + "_RR", entries)); - - vm2.invoke(() -> WANTestBase.checkGatewayReceiverStats(2, entries, entries, true)); - - vm4.invoke(() -> WANTestBase.checkQueueStats("ln", 0, entries, entries, entries)); - vm4.invoke(() -> WANTestBase.checkBatchStats("ln", 2, true, false)); - - // wait until queue is empty - vm5.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(0) == 0)); - - vm5.invoke(() -> WANTestBase.checkQueueStats("ln", 0, entries, 0, 0)); - vm5.invoke(() -> WANTestBase.checkBatchStats("ln", 0)); - } - - @Test - public void testReplicatedSerialPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions() { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - vm2.invoke(WANTestBase::createReceiver); - - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - boolean groupTransactionEvents = true; - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - - vm2.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", null, isOffHeap())); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - vm4.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm5.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm6.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm7.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - - final Map keyValues = new HashMap(); - int entries = 12; - for (int i = 0; i < entries; i++) { - keyValues.put(i, i + "_Value"); - } - - // 4 transactions of 3 events each are sent so that the first batch - // would initially contain the first 3 transactions complete and the first - // event of the next transaction (10 entries). - // As --group-transaction-events is configured in the senders, the remaining - // events of the third transaction are added to the batch which makes - // that the batch is sent with 12 events. - int eventsPerTransaction = 3; - vm5.invoke(() -> WANTestBase.doPutsInsideTransactions(testName + "_RR", keyValues, - eventsPerTransaction)); - - vm2.invoke(() -> WANTestBase.validateRegionSize(testName + "_RR", entries)); - - vm2.invoke(() -> WANTestBase.checkGatewayReceiverStats(1, entries, entries, true)); - - vm4.invoke(() -> WANTestBase.checkQueueStats("ln", 0, entries, entries, entries)); - vm4.invoke(() -> WANTestBase.checkBatchStats("ln", 1, true)); - vm4.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - - // wait until queue is empty - vm5.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(0) == 0)); - - vm5.invoke(() -> WANTestBase.checkQueueStats("ln", 0, entries, 0, 0)); - vm5.invoke(() -> WANTestBase.checkBatchStats("ln", 0, true)); - vm5.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - } - - @Test - public void testReplicatedSerialPropagationWithGroupTransactionEventsWithoutBatchRedistributionSendsBatchesWithCompleteTransactions_SeveralClients() { - testReplicatedSerialPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions_SeveralClients( - false); - } - - @Test - public void testReplicatedSerialPropagationWithGroupTransactionEventsWithBatchRedistributionSendsBatchesWithCompleteTransactions_SeveralClients() { - testReplicatedSerialPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions_SeveralClients( - true); - } - - public void testReplicatedSerialPropagationWithGroupTransactionEventsSendsBatchesWithCompleteTransactions_SeveralClients( - boolean isBatchRedistribution) { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - if (!isBatchRedistribution) { - vm2.invoke(WANTestBase::createReceiver); - } - - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - final int batchSize = 10; - - boolean groupTransactionEvents = true; - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - - final String regionName = testName + "_RR"; - - vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap())); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap())); - vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap())); - vm6.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap())); - vm7.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap())); - - int clients = 2; - int eventsPerTransaction = batchSize + 1; - int entriesPerInvocation = eventsPerTransaction * 200; - - final List> data = new ArrayList<>(clients); - for (int clientId = 0; clientId < clients; clientId++) { - final Map keyValues = new HashMap<>(); - for (int i = entriesPerInvocation * clientId; i < entriesPerInvocation - * (clientId + 1); i++) { - keyValues.put(i, i + "_Value"); - } - data.add(keyValues); - } - - int entries = entriesPerInvocation * clients; - - List invocations = new ArrayList<>(clients); - for (int i = 0; i < clients; i++) { - final int index = i; - AsyncInvocation asyncInvocation = - vm4.invokeAsync(() -> WANTestBase.doPutsInsideTransactions(regionName, data.get(index), - eventsPerTransaction)); - invocations.add(asyncInvocation); - } - - try { - for (AsyncInvocation invocation : invocations) { - invocation.await(); - } - } catch (InterruptedException e) { - e.printStackTrace(); - } - - if (isBatchRedistribution) { - // wait for batches to be redistributed and then start the receiver - vm4.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0)); - vm2.invoke(WANTestBase::createReceiver); - } - - vm2.invoke(() -> WANTestBase.validateRegionSize(regionName, entries)); - - checkQueuesAreEmptyAndOnlyCompleteTransactionsAreReplicated(isBatchRedistribution); - } - - @Test - public void testReplicatedSerialPropagationWithBatchRedistWithoutGroupTransactionEventsSendsBatchesWithIncompleteTransactions() { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - boolean groupTransactionEvents = false; - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, 10, false, false, null, true, - groupTransactionEvents)); - - vm2.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", null, isOffHeap())); - - startSenderInVMs("ln", vm4, vm5); - - vm4.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm5.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm6.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm7.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - - final Map keyValues = new HashMap(); - int entries = 24; - for (int i = 0; i < entries; i++) { - keyValues.put(i, i + "_Value"); - } - int eventsPerTransaction = 3; - vm5.invoke(() -> WANTestBase.doPutsInsideTransactions(testName + "_RR", keyValues, - eventsPerTransaction)); - - // wait for batches to be redistributed and then start the receiver - vm4.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0)); - - vm2.invoke(WANTestBase::createReceiver); - - vm2.invoke(() -> WANTestBase.validateRegionSize(testName + "_RR", entries)); - - vm2.invoke(() -> WANTestBase.checkGatewayReceiverStats(3, entries, entries, true)); - - vm4.invoke(() -> WANTestBase.checkQueueStats("ln", 0, entries, entries, entries)); - vm4.invoke(() -> WANTestBase.checkBatchStats("ln", 3, true, true)); - - // wait until queue is empty - vm5.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(0) == 0)); - - vm5.invoke(() -> WANTestBase.checkQueueStats("ln", 0, entries, 0, 0)); - vm5.invoke(() -> WANTestBase.checkBatchStats("ln", 0)); - } - - @Test - public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEventsSendsBatchesWithCompleteTransactions() - throws Exception { - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - createCacheInVMs(nyPort, vm2); - - createCacheInVMs(lnPort, vm4, vm5, vm6, vm7); - - boolean groupTransactionEvents = true; - int batchSize = 10; - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - - vm2.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", null, isOffHeap())); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - vm4.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm5.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm6.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm7.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - - final Map keyValues = new HashMap(); - int entries = 24; - for (int i = 0; i < entries; i++) { - keyValues.put(i, i + "_Value"); - } - - // 8 transactions of 3 events each are sent so that the first batch - // events would initially contain the first 3 transactions complete and the first - // event of the next transaction (10 entries). - // As --group-transaction-events is configured in the senders, the remaining - // event of the third transaction is added to the batch which makes - // that the first batch is sent with 12 events. The same happens with the - // second batch which will contain 12 events too. - int eventsPerTransaction = 3; - vm5.invoke(() -> WANTestBase.doPutsInsideTransactions(testName + "_RR", keyValues, - eventsPerTransaction)); - - // wait for batches to be redistributed and then start the receiver - vm4.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0)); - - vm2.invoke(WANTestBase::createReceiver); - - vm2.invoke(() -> WANTestBase.validateRegionSize(testName + "_RR", entries)); - - vm2.invoke(() -> WANTestBase.checkGatewayReceiverStats(2, entries, entries, true)); - - vm4.invoke(() -> WANTestBase.checkQueueStats("ln", 0, entries, entries, entries)); - vm4.invoke(() -> WANTestBase.checkBatchStats("ln", 2, true, true)); - vm4.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - - // wait until queue is empty - vm5.invoke(() -> await() - .until(() -> WANTestBase.getSenderStats("ln", -1).get(0) == 0)); - - vm5.invoke(() -> WANTestBase.checkQueueStats("ln", 0, entries, 0, 0)); - vm5.invoke(() -> WANTestBase.checkBatchStats("ln", 0, true)); - vm5.invoke(() -> WANTestBase.checkConflatedStats("ln", 0)); - } - @Test public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception { Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); @@ -593,81 +234,6 @@ public void testReplicatedSerialPropagationHA() throws Exception { vm5.invoke(() -> WANTestBase.checkStats_Failover("ln", 10000)); } - @Test - public void testReplicatedSerialPropagationHAWithGroupTransactionEvents() throws Exception { - - Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); - Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort)); - - vm2.invoke(() -> WANTestBase.createCache(nyPort)); - vm2.invoke(WANTestBase::createReceiver); - - vm4.invoke(() -> WANTestBase.createCache(lnPort)); - vm5.invoke(() -> WANTestBase.createCache(lnPort)); - vm6.invoke(() -> WANTestBase.createCache(lnPort)); - vm7.invoke(() -> WANTestBase.createCache(lnPort)); - - int batchSize = 9; - boolean groupTransactionEvents = true; - vm4.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm5.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm6.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - vm7.invoke( - () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, false, null, true, - groupTransactionEvents)); - - vm2.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", null, isOffHeap())); - - startSenderInVMs("ln", vm4, vm5, vm6, vm7); - - vm4.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm5.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm6.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - vm7.invoke(() -> WANTestBase.createReplicatedRegion(testName + "_RR", "ln", isOffHeap())); - - AsyncInvocation inv1 = - vm6.invokeAsync(() -> WANTestBase.doTxPutsWithRetryIfError(testName + "_RR", 2, 5000, 0)); - AsyncInvocation inv2 = - vm7.invokeAsync(() -> WANTestBase.doTxPutsWithRetryIfError(testName + "_RR", 2, 5000, 1)); - - vm2.invoke(() -> await() - .untilAsserted(() -> assertEquals("Waiting for some batches to be received", true, - getRegionSize(testName + "_RR") > 40))); - - AsyncInvocation inv3 = vm4.invokeAsync(() -> WANTestBase.killSender("ln")); - Boolean isKilled = Boolean.FALSE; - try { - isKilled = (Boolean) inv3.getResult(); - } catch (Throwable e) { - fail("Unexpected exception while killing a sender"); - } - AsyncInvocation inv4; - if (!isKilled) { - inv4 = vm5.invokeAsync(() -> WANTestBase.killSender("ln")); - inv4.join(); - } - inv1.join(); - inv2.join(); - inv3.join(); - - vm5.invoke(() -> WANTestBase.validateRegionSize(testName + "_RR", 20000)); - vm2.invoke(() -> WANTestBase.validateRegionSize(testName + "_RR", 20000)); - - // batchesReceived is equal to numberOfEntries/(batchSize+1) - // As transactions are 2 events long, for each batch it will always be necessary to - // add one more entry to the 9 events batch in order to have complete transactions in the batch. - int batchesReceived = (10000 + 10000) / (batchSize + 1); - vm2.invoke(() -> WANTestBase.checkGatewayReceiverStatsHA(batchesReceived, 20000, 20000)); - - vm5.invoke(() -> WANTestBase.checkStats_Failover("ln", 20000)); - } - @Test public void testReplicatedSerialPropagationUnprocessedEvents() throws Exception { Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); @@ -921,31 +487,4 @@ public void testSerialPropagationConflation() throws Exception { vm4.invoke(() -> WANTestBase.checkQueueStats("ln", 0, 2000, 2000, 1500)); vm4.invoke(() -> WANTestBase.checkConflatedStats("ln", 500)); } - - private void checkQueuesAreEmptyAndOnlyCompleteTransactionsAreReplicated( - boolean isBatchesRedistributed) { - // Wait for sender queues to be empty - List v4List = - vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - List v5List = - vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - List v6List = - vm6.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - List v7List = - vm7.invoke(() -> WANTestBase.getSenderStats("ln", 0)); - - // queue size must be 0 - assertThat(v4List.get(0) + v5List.get(0) + v6List.get(0) + v7List.get(0)).isEqualTo(0); - - // batches redistributed: - int batchesRedistributed = v4List.get(5) + v5List.get(5) + v6List.get(5) + v7List.get(5); - if (isBatchesRedistributed) { - assertThat(batchesRedistributed).isGreaterThan(0); - } else { - assertThat(batchesRedistributed).isEqualTo(0); - } - - // batches with incomplete transactions must be 0 - assertThat(v4List.get(13) + v5List.get(13) + v6List.get(13) + v7List.get(13)).isEqualTo(0); - } } diff --git a/geode-wan/src/integrationTest/java/org/apache/geode/internal/cache/wan/misc/WANConfigurationJUnitTest.java b/geode-wan/src/integrationTest/java/org/apache/geode/internal/cache/wan/misc/WANConfigurationJUnitTest.java index 6dc0ea2dfdc8..cd67e846dfce 100644 --- a/geode-wan/src/integrationTest/java/org/apache/geode/internal/cache/wan/misc/WANConfigurationJUnitTest.java +++ b/geode-wan/src/integrationTest/java/org/apache/geode/internal/cache/wan/misc/WANConfigurationJUnitTest.java @@ -82,31 +82,6 @@ public void test_GatewaySender_without_Locator() { } } - @Test - public void test_create_SerialGatewaySender_ThrowsException_when_GroupTransactionEvents_isTrue_and_DispatcherThreads_is_greaterThanOne() { - cache = new CacheFactory().set(MCAST_PORT, "0").create(); - GatewaySenderFactory fact = cache.createGatewaySenderFactory(); - fact.setParallel(false); - fact.setDispatcherThreads(2); - fact.setGroupTransactionEvents(true); - assertThatThrownBy(() -> fact.create("NYSender", 2)) - .isInstanceOf(GatewaySenderException.class) - .hasMessageContaining( - "SerialGatewaySender NYSender cannot be created with group transaction events set to true when dispatcher threads is greater than 1"); - } - - @Test - public void test_create_GatewaySender_ThrowsException_when_GroupTransactionEvents_isTrue_and_BatchConflation_is_enabled() { - cache = new CacheFactory().set(MCAST_PORT, "0").create(); - GatewaySenderFactory fact = cache.createGatewaySenderFactory(); - fact.setBatchConflationEnabled(true); - fact.setGroupTransactionEvents(true); - assertThatThrownBy(() -> fact.create("NYSender", 2)) - .isInstanceOf(GatewaySenderException.class) - .hasMessageContaining( - "GatewaySender NYSender cannot be created with both group transaction events set to true and batch conflation enabled"); - } - /** * Test to validate that sender with same Id can not be added to cache. */