diff --git a/geode-assembly/src/integrationTest/resources/assembly_content.txt b/geode-assembly/src/integrationTest/resources/assembly_content.txt index 966298fe1abc..fbde538d9553 100644 --- a/geode-assembly/src/integrationTest/resources/assembly_content.txt +++ b/geode-assembly/src/integrationTest/resources/assembly_content.txt @@ -561,6 +561,7 @@ javadoc/org/apache/geode/cache/wan/EventSequenceID.html javadoc/org/apache/geode/cache/wan/GatewayEventFilter.html javadoc/org/apache/geode/cache/wan/GatewayEventSubstitutionFilter.html javadoc/org/apache/geode/cache/wan/GatewayQueueEvent.html +javadoc/org/apache/geode/cache/wan/GatewaySenderStartupAction.html javadoc/org/apache/geode/cache/wan/GatewayReceiver.html javadoc/org/apache/geode/cache/wan/GatewayReceiverFactory.html javadoc/org/apache/geode/cache/wan/GatewaySender.OrderPolicy.html diff --git a/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt b/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt index 835662d7a827..2a54afcbd856 100644 --- a/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt +++ b/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt @@ -106,3 +106,4 @@ org/apache/geode/internal/net/ByteBufferVendor$OpenAttemptTimedOut org/apache/geode/internal/cache/wan/GatewaySenderEventImpl$TransactionMetadataDisposition org/apache/geode/internal/cache/tier/InterestType org/apache/geode/internal/cache/tier/MessageType +org/apache/geode/cache/wan/GatewaySenderStartupAction diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java index 700cc4baeff9..43ecb5fb492e 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java +++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java @@ -178,7 +178,7 @@ public AsyncEventQueue create(String asyncQueueId, AsyncEventListener listener) asyncEventQueue = asyncEventQueueImpl; cache.addAsyncEventQueue(asyncEventQueueImpl); if (pauseEventsDispatching) { - sender.setStartEventProcessorInPausedState(); + sender.setStartEventProcessor(true); } if (!gatewaySenderAttributes.isManualStart()) { sender.start(); diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java index d5395df9e905..b60bc7ca31d7 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java +++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java @@ -56,6 +56,9 @@ public ParallelAsyncEventQueueImpl(InternalCache cache, StatisticsFactory statis isForInternalUse = true; } + @Override + public void recoverInStoppedState() {} + @Override public void start() { start(false); diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java index 06e6e594e2d5..8e4b33d9374f 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java +++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java @@ -59,6 +59,9 @@ public SerialAsyncEventQueueImpl(InternalCache cache, StatisticsFactory statisti } } + @Override + public void recoverInStoppedState() {} + @Override public void start() { start(false); diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java index 1176090e2be5..d28555cd6d75 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java +++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java @@ -2760,6 +2760,8 @@ public static class GatewaySender { protected String batchTimeInterval; @XmlAttribute(name = "enable-persistence") protected Boolean enablePersistence; + @XmlAttribute(name = "startup-action") + protected String startupAction; @XmlAttribute(name = "disk-store-name") protected String diskStoreName; @XmlAttribute(name = "disk-synchronous") @@ -2877,6 +2879,30 @@ public String getId() { return id; } + /** + * Gets the value of the startup-action property. + * + * possible object is + * {@link String } + * + * @return the gateway-sender startup action. + */ + public String getStartupAction() { + return startupAction; + } + + /** + * Sets the value of the startup-action property. + * + * allowed object is + * {@link String } + * + * @param value gateway-sender startup action + */ + public void setStartupAction(String value) { + this.startupAction = value; + } + /** * Sets the value of the id property. * @@ -2917,7 +2943,6 @@ public Boolean mustGroupTransactionEvents() { return groupTransactionEvents; } - public void setGroupTransactionEvents(Boolean value) { groupTransactionEvents = value; } diff --git a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySenderStartupAction.java b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySenderStartupAction.java new file mode 100644 index 000000000000..93cdfe6b66ab --- /dev/null +++ b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySenderStartupAction.java @@ -0,0 +1,63 @@ +/* + * 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; + +/* + * GatewaySenderStartupAction is persisted in cluster configuration when start, stop, pause or + * resume gateway-sender command is successfully executed. At startup member will read persisted + * startup-action parameter and act accordingly. + */ +public enum GatewaySenderStartupAction { + /* + * This action ("start") is persisted in cluster configuration after + * start or resume gateway-sender command is successfully executed. + * At startup member will start gateway sender. When set then + * this parameter has advantage over manual-start parameter. + */ + START("start"), + /* + * This action ("stop") is persisted in cluster configuration after + * stop gateway-sender command is successfully executed. At startup + * member will not start gateway-sender, but only recover + * gateway queues from persistent storage if needed. When set then + * this parameter has advantage over manual-start parameter. + */ + STOP("stop"), + /* + * This action ("pause") is persisted in cluster configuration after + * pause gateway-sender command is successfully executed. At startup + * member will start gateway-sender in paused state. When set then + * this parameter has advantage over manual-start parameter. + */ + PAUSE("pause"), + /* + * Used when startup-action parameter is not available in cluster configuration. + */ + NONE("none"); + + private String action; + + GatewaySenderStartupAction(String action) { + this.action = action; + } + + @Override + public String toString() { + return "GatewaySenderStartupAction {" + + "action='" + action + '\'' + + '}'; + } +} diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java index dc8351a5432b..7e8d4933b420 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java @@ -1569,7 +1569,6 @@ boolean recoverPersistentBuckets() { */ PartitionedRegion leaderRegion = ColocationHelper.getLeaderRegion(partitionedRegion); - // Check if the leader region or some child shadow PR region is persistent // and return the first persistent region found PartitionedRegion persistentLeader = getPersistentLeader(); diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java index e14a159fc26c..dcb654d67338 100755 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java @@ -137,6 +137,7 @@ import org.apache.geode.cache.query.internal.types.ObjectTypeImpl; import org.apache.geode.cache.query.types.ObjectType; import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.distributed.DistributedLockService; import org.apache.geode.distributed.DistributedMember; import org.apache.geode.distributed.DistributedSystemDisconnectedException; @@ -242,6 +243,7 @@ import org.apache.geode.internal.cache.wan.AsyncEventQueueConfigurationException; import org.apache.geode.internal.cache.wan.GatewaySenderConfigurationException; import org.apache.geode.internal.cache.wan.GatewaySenderException; +import org.apache.geode.internal.cache.wan.InternalGatewaySender; import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySenderQueue; import org.apache.geode.internal.cache.wan.parallel.ParallelGatewaySenderQueue; import org.apache.geode.internal.logging.log4j.LogMarker; @@ -1222,10 +1224,13 @@ public void postCreateRegion() { * get the ParallelGatewaySender to create the colocated partitioned region for this * region. */ + InternalGatewaySender senderImpl = (InternalGatewaySender) sender; if (sender.isRunning()) { - AbstractGatewaySender senderImpl = (AbstractGatewaySender) sender; ((ConcurrentParallelGatewaySenderQueue) senderImpl.getQueues() .toArray(new RegionQueue[1])[0]).addShadowPartitionedRegionForUserPR(this); + } else if (GatewaySenderStartupAction.STOP == senderImpl + .calculateStartupActionForGatewaySender()) { + senderImpl.recoverInStoppedState(); } } } 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 f96cdfe415ae..278a3de834a5 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 @@ -34,7 +34,6 @@ import org.apache.geode.CancelException; import org.apache.geode.InternalGemFireError; import org.apache.geode.annotations.Immutable; -import org.apache.geode.annotations.VisibleForTesting; import org.apache.geode.annotations.internal.MutableForTesting; import org.apache.geode.cache.CacheException; import org.apache.geode.cache.DataPolicy; @@ -49,6 +48,7 @@ import org.apache.geode.cache.wan.GatewayEventSubstitutionFilter; import org.apache.geode.cache.wan.GatewayQueueEvent; import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.cache.wan.GatewayTransportFilter; import org.apache.geode.distributed.GatewayCancelledException; import org.apache.geode.distributed.internal.DistributionAdvisee; @@ -154,6 +154,8 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di private int serialNumber; + protected GatewaySenderStartupAction startupAction; + protected GatewaySenderStats statistics; private Stopper stopper; @@ -186,6 +188,19 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di protected volatile ConcurrentLinkedQueue tmpDroppedEvents = new ConcurrentLinkedQueue<>(); + + /** + * Contains wan replication events that were dropped by parallel gateway senders. + * Activate this hook by setting system property ENABLE_TEST_HOOK_TEMP_DROPPED_EVENTS + */ + private volatile ConcurrentLinkedQueue testHookTempDroppedEvents; + + /** + * Only used for testing purpose. This property enables test hook which collects all + * wan replication events that are dropped by parallel gateway senders. + */ + private static final boolean ENABLE_TEST_HOOK_TEMP_DROPPED_EVENTS = + Boolean.getBoolean("enable-test-hook-temp-dropped-events"); /** * The number of seconds to wait before stopping the GatewaySender. Default is 0 seconds. */ @@ -270,7 +285,7 @@ public AbstractGatewaySender(InternalCache cache, StatisticsClock statisticsCloc transFilters = Collections.unmodifiableList(attrs.getGatewayTransportFilters()); listeners = attrs.getAsyncEventListeners(); substitutionFilter = attrs.getGatewayEventSubstitutionFilter(); - locatorDiscoveryCallback = attrs.getGatewayLocatoDiscoveryCallback(); + locatorDiscoveryCallback = attrs.getGatewayLocatorDiscoveryCallback(); isDiskSynchronous = attrs.isDiskSynchronous(); policy = attrs.getOrderPolicy(); dispatcherThreads = attrs.getDispatcherThreads(); @@ -293,8 +308,10 @@ public AbstractGatewaySender(InternalCache cache, StatisticsClock statisticsCloc } initializeEventIdIndex(); } + isBucketSorted = attrs.isBucketSorted(); forwardExpirationDestroy = attrs.isForwardExpirationDestroy(); + startupAction = attrs.getStartupAction(); } public GatewaySenderAdvisor getSenderAdvisor() { @@ -415,6 +432,31 @@ public int getSocketReadTimeout() { return socketReadTimeout; } + @Override + public GatewaySenderStartupAction getStartupAction() { + return startupAction; + } + + /** + * This method returns startup action of gateway-sender. The startup action is calculated + * based on the startup-action (please check {@link GatewaySenderStartupAction}) and + * manual-start parameters. If set, then startup-action parameter has advantage over + * the manual-start parameter. + * + * @see GatewaySenderStartupAction + */ + public GatewaySenderStartupAction calculateStartupActionForGatewaySender() { + // If startup-action parameter is not available, then use manual-start parameter + // to determine initial state of gateway-sender + if (this.getStartupAction() == GatewaySenderStartupAction.NONE) { + if (!this.isManualStart()) { + return GatewaySenderStartupAction.START; + } + return GatewaySenderStartupAction.STOP; + } + return this.getStartupAction(); + } + @Override public boolean isBatchConflationEnabled() { return isConflation; @@ -586,6 +628,9 @@ public boolean isForInternalUse() { @Override public abstract void start(); + @Override + public abstract void recoverInStoppedState(); + @Override public abstract void startWithCleanQueue(); @@ -922,8 +967,8 @@ public boolean isStartEventProcessorInPausedState() { } @Override - public void setStartEventProcessorInPausedState() { - startEventProcessorInPausedState = true; + public void setStartEventProcessor(boolean isPaused) { + startEventProcessorInPausedState = isPaused; } /** @@ -1059,7 +1104,7 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event, } // this filter is defined by Asif which exist in old wan too. new wan has - // other GatewaEventFilter. Do we need to get rid of this filter. Cheetah is + // other GatewayEventFilter. Do we need to get rid of this filter. Cheetah is // not considering this filter if (!filter.enqueueEvent(event)) { stats.incEventsFiltered(); @@ -1241,17 +1286,26 @@ private void recordDroppedEvent(EntryEventImpl event) { eventProcessor.registerEventDroppedInPrimaryQueue(event); } else { tmpDroppedEvents.add(event); + if (ENABLE_TEST_HOOK_TEMP_DROPPED_EVENTS) { + if (testHookTempDroppedEvents == null) { + testHookTempDroppedEvents = new ConcurrentLinkedQueue<>(); + } + testHookTempDroppedEvents.add(event); + } if (logger.isDebugEnabled()) { logger.debug("added to tmpDroppedEvents event: {}", event); } } } - @VisibleForTesting - int getTmpDroppedEventSize() { + protected int getTempDroppedEventSize() { return tmpDroppedEvents.size(); } + protected int getTempDroppedEventsHookSize() { + return testHookTempDroppedEvents.size(); + } + /** * During sender is getting started, if there are any cache operation on queue then that event * will be stored in temp queue. Once sender is started, these event from tmp queue will be added @@ -1267,10 +1321,7 @@ int getTmpDroppedEventSize() { public void enqueueTempEvents() { if (eventProcessor != null) {// Fix for defect #47308 // process tmpDroppedEvents - EntryEventImpl droppedEvent; - while ((droppedEvent = tmpDroppedEvents.poll()) != null) { - eventProcessor.registerEventDroppedInPrimaryQueue(droppedEvent); - } + processTempDroppedEvents(); TmpQueueEvent nextEvent = null; final GatewaySenderStats stats = getStatistics(); @@ -1304,6 +1355,22 @@ this, getId(), nextEvent.getOperation(), nextEvent), } } + /** + * During sender is recovered in stopped state, if there are any cache operations while + * queue and event processor is being created then these events should be stored in + * tmpDroppedEvents temporary queue. Once event processor is created then queue will be + * drained and ParallelQueueRemovalMessage will be sent. + */ + public void processTempDroppedEvents() { + if (this.eventProcessor != null) { + // process tmpDroppedEvents + EntryEventImpl droppedEvent; + while ((droppedEvent = tmpDroppedEvents.poll()) != null) { + this.eventProcessor.registerEventDroppedInPrimaryQueue(droppedEvent); + } + } + } + /** * Removes the EntryEventImpl, whose tailKey matches with the provided tailKey, from * tmpQueueEvents. diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderEventProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderEventProcessor.java index c7413aa91fca..88d239dc5638 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderEventProcessor.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderEventProcessor.java @@ -188,7 +188,7 @@ public int getTotalQueueSize() { return getQueue().size(); } - protected abstract void initializeMessageQueue(String id, boolean cleanQueues); + protected abstract void initializeMessageQueue(String id, boolean cleanQueues, boolean isStopped); public void enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue) throws IOException, CacheException { diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java index 74f9c42685ba..f941ef483fe1 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java @@ -23,6 +23,7 @@ import org.apache.geode.cache.wan.GatewayEventSubstitutionFilter; import org.apache.geode.cache.wan.GatewaySender; import org.apache.geode.cache.wan.GatewaySender.OrderPolicy; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.cache.wan.GatewayTransportFilter; public class GatewaySenderAttributes { @@ -41,6 +42,8 @@ public class GatewaySenderAttributes { private int batchTimeInterval = GatewaySender.DEFAULT_BATCH_TIME_INTERVAL; + private GatewaySenderStartupAction startupAction = GatewaySenderStartupAction.NONE; + private boolean isBatchConflationEnabled = GatewaySender.DEFAULT_BATCH_CONFLATION; private boolean isPersistenceEnabled = GatewaySender.DEFAULT_PERSISTENCE_ENABLED; @@ -132,8 +135,8 @@ public void setDiskStoreName(String diskstorename) { diskStoreName = diskstorename; } - public void setEventSubstitutionFilter(GatewayEventSubstitutionFilter eventsubstitutionfilter) { - eventSubstitutionFilter = eventsubstitutionfilter; + public void setEventSubstitutionFilter(GatewayEventSubstitutionFilter eventSubstitutionFilter) { + this.eventSubstitutionFilter = eventSubstitutionFilter; } public void setId(String idString) { @@ -248,7 +251,7 @@ public List getAsyncEventListeners() { return listeners; } - public LocatorDiscoveryCallback getGatewayLocatoDiscoveryCallback() { + public LocatorDiscoveryCallback getGatewayLocatorDiscoveryCallback() { return locatorDiscoveryCallback; } @@ -312,6 +315,14 @@ public GatewayEventSubstitutionFilter getGatewayEventSubstitutionFilter() { return eventSubstitutionFilter; } + public GatewaySenderStartupAction getStartupAction() { + return this.startupAction; + } + + public void setStartupAction(GatewaySenderStartupAction startupAction) { + this.startupAction = startupAction; + } + public boolean isMetaQueue() { return isMetaQueue; } diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java index c4f48b7dd6af..b3ba8a7acd6b 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java @@ -20,6 +20,7 @@ import java.util.concurrent.TimeUnit; import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.RegionQueue; import org.apache.geode.internal.statistics.StatisticsClock; @@ -46,7 +47,37 @@ public interface InternalGatewaySender extends GatewaySender { void destroy(boolean initiator); - void setStartEventProcessorInPausedState(); + void setStartEventProcessor(boolean isPaused); + + /** + * Recovers partition region used by parallel gateway-sender queue. Parallel gateway sender + * queue region is colocated with partition region on which is collecting events. It is necessary + * to recover colocated gateway sender queue region, so it doesn't block the colocated data + * region to reach the online status. + */ + void recoverInStoppedState(); + + /** + * Returns the startup-action of the GatewaySender. This action parameter is set + * after start, stop, pause and resume gateway-sender gfsh commands. + * + * @return startup action parameter value + * + * @see GatewaySenderStartupAction + */ + GatewaySenderStartupAction getStartupAction(); + + /** + * This method returns startup action of gateway-sender. The startup action is calculated + * based on the startup-action (please check {@link GatewaySenderStartupAction}) and + * manual-start parameters. If set, then startup-action parameter has advantage over + * the manual-start parameter. + * + * @return startup action + * + * @see GatewaySenderStartupAction + */ + GatewaySenderStartupAction calculateStartupActionForGatewaySender(); int getEventQueueSize(); diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySenderFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySenderFactory.java index 6b941c846015..8f01989c2a7b 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySenderFactory.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySenderFactory.java @@ -18,6 +18,7 @@ import org.apache.geode.cache.client.internal.LocatorDiscoveryCallback; import org.apache.geode.cache.wan.GatewaySender; import org.apache.geode.cache.wan.GatewaySenderFactory; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; public interface InternalGatewaySenderFactory extends GatewaySenderFactory { @@ -40,4 +41,15 @@ public interface InternalGatewaySenderFactory extends GatewaySenderFactory { * @param retries the maximum number of retries. */ GatewaySenderFactory setRetriesToGetTransactionEventsFromQueue(int retries); + + /** + * Sets startup-action of the GatewaySender. This action parameter is set after + * start, stop, pause and resume gateway-sender gfsh commands. + * + * @param gatewaySenderStartupAction Gateway-sender gatewaySenderStartupAction + * + * @see GatewaySenderStartupAction + */ + GatewaySenderFactory setStartupAction( + GatewaySenderStartupAction gatewaySenderStartupAction); } diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ConcurrentParallelGatewaySenderEventProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ConcurrentParallelGatewaySenderEventProcessor.java index 18f035fa17bb..e2ae07ba6b7b 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ConcurrentParallelGatewaySenderEventProcessor.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ConcurrentParallelGatewaySenderEventProcessor.java @@ -71,6 +71,11 @@ public class ConcurrentParallelGatewaySenderEventProcessor public ConcurrentParallelGatewaySenderEventProcessor(AbstractGatewaySender sender, ThreadsMonitoring tMonitoring, boolean cleanQueues) { + this(sender, tMonitoring, cleanQueues, false); + } + + public ConcurrentParallelGatewaySenderEventProcessor(AbstractGatewaySender sender, + ThreadsMonitoring tMonitoring, boolean cleanQueues, boolean recoverQueuesOnly) { super("Event Processor for GatewaySender_" + sender.getId(), sender, tMonitoring); logger.info("ConcurrentParallelGatewaySenderEventProcessor: dispatcher threads {}", sender.getDispatcherThreads()); @@ -99,26 +104,26 @@ public ConcurrentParallelGatewaySenderEventProcessor(AbstractGatewaySender sende logger.debug("The target PRs are {} Dispatchers: {}", targetRs, nDispatcher); } - createProcessors(sender.getDispatcherThreads(), targetRs, cleanQueues); + createProcessors(sender.getDispatcherThreads(), targetRs, cleanQueues, recoverQueuesOnly); - // this.queue = parallelQueue; queue = new ConcurrentParallelGatewaySenderQueue(sender, processors); } protected void createProcessors(int dispatcherThreads, Set> targetRs, - boolean cleanQueues) { + boolean cleanQueues, boolean recoverQueuesOnly) { processors = new ParallelGatewaySenderEventProcessor[sender.getDispatcherThreads()]; if (logger.isDebugEnabled()) { logger.debug("Creating AsyncEventProcessor"); } for (int i = 0; i < sender.getDispatcherThreads(); i++) { processors[i] = new ParallelGatewaySenderEventProcessor(sender, i, - sender.getDispatcherThreads(), getThreadMonitorObj(), cleanQueues); + sender.getDispatcherThreads(), getThreadMonitorObj(), cleanQueues, + recoverQueuesOnly); } } @Override - protected void initializeMessageQueue(String id, boolean cleanQueues) { + protected void initializeMessageQueue(String id, boolean cleanQueues, boolean isStopped) { // nothing } diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderEventProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderEventProcessor.java index f01e227bced9..5c1100c286c7 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderEventProcessor.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderEventProcessor.java @@ -51,7 +51,7 @@ protected ParallelGatewaySenderEventProcessor(AbstractGatewaySender sender, super("Event Processor for GatewaySender_" + sender.getId(), sender, tMonitoring); index = 0; nDispatcher = 1; - initializeMessageQueue(sender.getId(), cleanQueues); + initializeMessageQueue(sender.getId(), cleanQueues, false); } /** @@ -62,11 +62,21 @@ protected ParallelGatewaySenderEventProcessor(AbstractGatewaySender sender, int super("Event Processor for GatewaySender_" + sender.getId() + "_" + index, sender, tMonitoring); this.index = index; this.nDispatcher = nDispatcher; - initializeMessageQueue(sender.getId(), cleanQueues); + initializeMessageQueue(sender.getId(), cleanQueues, false); + } + + protected ParallelGatewaySenderEventProcessor(AbstractGatewaySender sender, + int id, int nDispatcher, ThreadsMonitoring tMonitoring, + boolean cleanQueues, boolean recoverQueuesOnly) { + super("Event Processor for GatewaySender_" + sender.getId() + "_" + id, sender, tMonitoring); + this.index = id; + this.nDispatcher = nDispatcher; + initializeMessageQueue(sender.getId(), cleanQueues, recoverQueuesOnly); } @Override - protected void initializeMessageQueue(String id, boolean cleanQueues) { + protected void initializeMessageQueue(String id, boolean cleanQueues, + boolean recoverQueuesOnly) { Set> targetRs = new HashSet<>(); for (InternalRegion region : sender.getCache().getApplicationRegions()) { if (region.getAllGatewaySenderIds().contains(id)) { @@ -78,12 +88,15 @@ protected void initializeMessageQueue(String id, boolean cleanQueues) { } ParallelGatewaySenderQueue queue = - new ParallelGatewaySenderQueue(sender, targetRs, index, nDispatcher, cleanQueues); + new ParallelGatewaySenderQueue(sender, targetRs, index, nDispatcher, cleanQueues, + recoverQueuesOnly); - queue.start(); + if (!recoverQueuesOnly) { + queue.start(); + } this.queue = queue; - if (queue.localSize() > 0) { + if (!recoverQueuesOnly && queue.localSize() > 0) { queue.notifyEventProcessorIfRequired(); } } 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 7dc5f18e63fb..ec355469121a 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 @@ -246,12 +246,14 @@ private Object deserialize(Object serializedBytes) { public ParallelGatewaySenderQueue(AbstractGatewaySender sender, Set> userRegions, int idx, - int nDispatcher, boolean cleanQueues) { - this(sender, userRegions, idx, nDispatcher, new MetaRegionFactory(), cleanQueues); + int nDispatcher, boolean cleanQueues, boolean recoverQueuesOnly) { + this(sender, userRegions, idx, nDispatcher, new MetaRegionFactory(), cleanQueues, + recoverQueuesOnly); } ParallelGatewaySenderQueue(AbstractGatewaySender sender, Set> userRegions, int idx, - int nDispatcher, MetaRegionFactory metaRegionFactory, boolean cleanQueues) { + int nDispatcher, MetaRegionFactory metaRegionFactory, boolean cleanQueues, + boolean recoverQueuesOnly) { this.metaRegionFactory = metaRegionFactory; @@ -295,8 +297,9 @@ public ParallelGatewaySenderQueue(AbstractGatewaySender sender, Set queueEmptyLock = new StoppableReentrantLock(sender.getCancelCriterion()); queueEmptyCondition = queueEmptyLock.newCondition(); - // initialize the conflation thread pool if conflation is enabled - if (sender.isBatchConflationEnabled()) { + removalThread = new BatchRemovalThread(this.sender.getCache(), this); + // initialize the conflation thread pool if conflation is enabled and queues are started + if (sender.isBatchConflationEnabled() && !recoverQueuesOnly) { initializeConflationThreadPool(); } } @@ -306,10 +309,7 @@ public void start() { // at present, this won't be accessed by multiple threads, // still, it is safer approach to synchronize it synchronized (ParallelGatewaySenderQueue.class) { - if (removalThread == null) { - removalThread = new BatchRemovalThread(sender.getCache(), this); - removalThread.start(); - } + removalThread.start(); } } diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/ConcurrentSerialGatewaySenderEventProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/ConcurrentSerialGatewaySenderEventProcessor.java index 10cd2dcc1a04..dbef0b7e03c8 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/ConcurrentSerialGatewaySenderEventProcessor.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/ConcurrentSerialGatewaySenderEventProcessor.java @@ -69,7 +69,7 @@ public ConcurrentSerialGatewaySenderEventProcessor(AbstractGatewaySender sender, super("Event Processor for GatewaySender_" + sender.getId(), sender, tMonitoring); this.sender = sender; - initializeMessageQueue(sender.getId(), cleanQueues); + initializeMessageQueue(sender.getId(), cleanQueues, false); queues = new HashSet<>(); for (SerialGatewaySenderEventProcessor processor : processors) { queues.add(processor.getQueue()); @@ -86,7 +86,7 @@ public int getTotalQueueSize() { } @Override - protected void initializeMessageQueue(String id, boolean cleanQueues) { + protected void initializeMessageQueue(String id, boolean cleanQueues, boolean isStopped) { for (int i = 0; i < sender.getDispatcherThreads(); i++) { processors.add( new SerialGatewaySenderEventProcessor(sender, id + "." + i, getThreadMonitorObj(), diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderEventProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderEventProcessor.java index 2caad0cc49a0..19d506e633e6 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderEventProcessor.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderEventProcessor.java @@ -112,14 +112,14 @@ public SerialGatewaySenderEventProcessor(AbstractGatewaySender sender, String id ThreadsMonitoring tMonitoring, boolean cleanQueues) { super("Event Processor for GatewaySender_" + id, sender, tMonitoring); synchronized (unprocessedEventsLock) { - initializeMessageQueue(id, cleanQueues); + initializeMessageQueue(id, cleanQueues, false); unprocessedEvents = new LinkedHashMap<>(); unprocessedTokens = new LinkedHashMap<>(); } } @Override - protected void initializeMessageQueue(String id, boolean cleanQueues) { + protected void initializeMessageQueue(String id, boolean cleanQueues, boolean isStopped) { // Create the region name String regionName = id + "_SERIAL_GATEWAY_SENDER_QUEUE"; diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXml.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXml.java index 3e70904a162c..5ddd68b6708e 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXml.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXml.java @@ -712,6 +712,8 @@ public abstract class CacheXml implements EntityResolver2, ErrorHandler { protected static final String ENABLE_BATCH_CONFLATION = "enable-batch-conflation"; /** The name of the enable-conflation attribute */ protected static final String ENABLE_PERSISTENCE = "enable-persistence"; + /** The name of the state */ + protected static final String STARTUP_ACTION = "startup-action"; protected static final String DISPATCHER_THREADS = "dispatcher-threads"; diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlGenerator.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlGenerator.java index 68928d2329ce..d65825dfd8bf 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlGenerator.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlGenerator.java @@ -109,6 +109,7 @@ import org.apache.geode.cache.wan.GatewayEventSubstitutionFilter; import org.apache.geode.cache.wan.GatewayReceiver; import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.cache.wan.GatewayTransportFilter; import org.apache.geode.distributed.Role; import org.apache.geode.internal.Assert; @@ -126,6 +127,7 @@ import org.apache.geode.internal.cache.extension.Extensible; import org.apache.geode.internal.cache.extension.Extension; import org.apache.geode.internal.cache.persistence.DefaultDiskDirs; +import org.apache.geode.internal.cache.wan.InternalGatewaySender; import org.apache.geode.internal.size.SizeClassOnceObjectSizer; import org.apache.geode.management.internal.configuration.utils.XmlConstants; import org.apache.geode.pdx.ReflectionBasedAutoSerializer; @@ -413,7 +415,6 @@ private void generate(PrintWriter pw) { @Override public void parse(InputSource input) throws SAXException { Assert.assertTrue(handler != null); - boolean isClientCache = creation instanceof ClientCacheCreation; handler.startDocument(); @@ -1435,6 +1436,13 @@ private void generateGatewaySender(GatewaySender sender) throws SAXException { atts.addAttribute("", "", ALERT_THRESHOLD, "", String.valueOf(sender.getAlertThreshold())); } + // gateway-sender startup action + InternalGatewaySender internalGatewaySender = (InternalGatewaySender) sender; + if (internalGatewaySender.getStartupAction() != GatewaySenderStartupAction.NONE) { + atts.addAttribute("", "", STARTUP_ACTION, "", + internalGatewaySender.getStartupAction().name()); + } + // dispatcher-threads if (generateDefaults() || sender.getDispatcherThreads() != GatewaySender.DEFAULT_DISPATCHER_THREADS) { diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParser.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParser.java index cb8b31f38bd4..13984ffd2b23 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParser.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParser.java @@ -26,6 +26,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.ServiceLoader; import java.util.Set; @@ -50,6 +51,7 @@ import org.apache.geode.DataSerializable; import org.apache.geode.DataSerializer; import org.apache.geode.InternalGemFireException; +import org.apache.geode.annotations.VisibleForTesting; import org.apache.geode.cache.Cache; import org.apache.geode.cache.CacheException; import org.apache.geode.cache.CacheListener; @@ -103,6 +105,7 @@ 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.cache.wan.GatewaySenderStartupAction; import org.apache.geode.cache.wan.GatewayTransportFilter; import org.apache.geode.compression.Compressor; import org.apache.geode.internal.Assert; @@ -114,6 +117,7 @@ import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.PartitionAttributesImpl; import org.apache.geode.internal.cache.PartitionedRegionHelper; +import org.apache.geode.internal.cache.wan.InternalGatewaySenderFactory; import org.apache.geode.internal.classloader.ClassPathLoader; import org.apache.geode.internal.datasource.ConfigProperty; import org.apache.geode.internal.datasource.DataSourceCreateException; @@ -173,6 +177,21 @@ public class CacheXmlParser extends CacheXml implements ContentHandler { */ protected Locator documentLocator; + /** + * No arg constructor for {@link CacheXmlParser}. + */ + public CacheXmlParser() {} + + /** + * Constructor for {@link CacheXmlParser} which is used in test. + * + * @param cache Cache + */ + @VisibleForTesting + CacheXmlParser(CacheCreation cache) { + this.cache = cache; + } + ////////////////////// Static Methods ////////////////////// /** * Parses XML data and from it creates an instance of CacheXmlParser that can be used @@ -642,6 +661,25 @@ private void startGatewaySender(Attributes atts) { gatewaySenderFactory.setPersistenceEnabled(Boolean.parseBoolean(enablePersistence)); } + String id = atts.getValue(ID); + // Gateway-sender startup action + InternalGatewaySenderFactory internalGatewaySenderFactory = + (InternalGatewaySenderFactory) gatewaySenderFactory; + String startupAction = atts.getValue(STARTUP_ACTION); + if (startupAction == null) { + internalGatewaySenderFactory.setStartupAction(GatewaySenderStartupAction.NONE); + } else if (Objects.equals(startupAction, GatewaySenderStartupAction.START.name().toLowerCase()) + || + Objects.equals(startupAction, GatewaySenderStartupAction.STOP.name().toLowerCase()) || + Objects.equals(startupAction, GatewaySenderStartupAction.PAUSE.name().toLowerCase())) { + internalGatewaySenderFactory + .setStartupAction(GatewaySenderStartupAction.valueOf(startupAction.toUpperCase())); + } else { + throw new InternalGemFireException( + String.format("An invalid startup-action value (%s) was configured for gateway sender %s", + startupAction, id)); + } + String diskStoreName = atts.getValue(DISK_STORE_NAME); gatewaySenderFactory.setDiskStoreName(diskStoreName); @@ -675,7 +713,6 @@ private void startGatewaySender(Attributes atts) { gatewaySenderFactory.setDispatcherThreads(Integer.parseInt(dispatcherThreads)); } - String id = atts.getValue(ID); String orderPolicy = atts.getValue(ORDER_POLICY); if (orderPolicy != null) { try { diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelAsyncEventQueueCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelAsyncEventQueueCreation.java index b9433dcbc65a..b2972b2e8f0a 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelAsyncEventQueueCreation.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelAsyncEventQueueCreation.java @@ -45,6 +45,9 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event, @Override public void start() {} + @Override + public void recoverInStoppedState() {} + @Override public void startWithCleanQueue() {} diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelGatewaySenderCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelGatewaySenderCreation.java index 561f0b46349c..64971f68273b 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelGatewaySenderCreation.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelGatewaySenderCreation.java @@ -45,6 +45,9 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event, @Override public void start() {} + @Override + public void recoverInStoppedState() {} + @Override public void startWithCleanQueue() {} diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialAsyncEventQueueCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialAsyncEventQueueCreation.java index 834f6ad474a2..fc5647345dbf 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialAsyncEventQueueCreation.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialAsyncEventQueueCreation.java @@ -44,6 +44,9 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event, @Override public void start() {} + @Override + public void recoverInStoppedState() {} + @Override public void startWithCleanQueue() {} diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialGatewaySenderCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialGatewaySenderCreation.java index a239fd997259..f45905ca2a8b 100644 --- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialGatewaySenderCreation.java +++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialGatewaySenderCreation.java @@ -45,6 +45,9 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event, @Override public void start() {} + @Override + public void recoverInStoppedState() {} + @Override public void startWithCleanQueue() {} diff --git a/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java b/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java index 6831c2ce505d..43ef0af3da0d 100644 --- a/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java +++ b/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java @@ -350,6 +350,4 @@ public interface GatewaySenderMXBean { */ int getEventsExceedingAlertThreshold(); - - } diff --git a/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd b/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd index db6841a60287..b708dbe8ce07 100755 --- a/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd +++ b/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd @@ -204,6 +204,7 @@ declarative caching XML file elements unless indicated otherwise. + diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderTest.java index 193a120458a2..87dd813e7e9f 100644 --- a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderTest.java +++ b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderTest.java @@ -132,6 +132,9 @@ public void fillInProfile(DistributionAdvisor.Profile profile) {} @Override public void start() {} + @Override + public void recoverInStoppedState() {} + @Override public boolean isPrimary() { return true; @@ -193,7 +196,9 @@ public int getTimesEnqueueEventCalled() { } @Override - protected void initializeMessageQueue(String id, boolean cleanQueues) {} + protected void initializeMessageQueue(String id, boolean cleanQueues, boolean isStopped) { + + } @Override protected void rebalance() {} @@ -221,6 +226,11 @@ public void fillInProfile(DistributionAdvisor.Profile profile) {} @Override public void start() {} + @Override + public void recoverInStoppedState() { + + } + @Override public void startWithCleanQueue() {} diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java index 7e8000f0d1af..ab15aa7f476d 100644 --- a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java +++ b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java @@ -87,7 +87,7 @@ public void createParallelGatewaySenderQueue() { when(sender.getId()).thenReturn(""); metaRegionFactory = mock(MetaRegionFactory.class); queue = new ParallelGatewaySenderQueue(sender, Collections.emptySet(), 0, 1, metaRegionFactory, - false); + false, false); } @Test @@ -568,7 +568,7 @@ public void testCleanQueueExecuteShadowPRWaitForBucketRecovery() { ParallelGatewaySenderQueue queue1 = new ParallelGatewaySenderQueue(sender, targetRs, 0, 1, metaRegionFactory, - true); + true, false); verify(shadowRegion, times(1)).shadowPRWaitForBucketRecovery(); @@ -639,7 +639,7 @@ public void testCleanQueueExecuteShadowPRWaitForRegionDestroyingToFinish() { ParallelGatewaySenderQueue queue1 = new ParallelGatewaySenderQueue(sender, targetRs, 0, 1, metaRegionFactory, - true); + true, false); verify(cache, times(3)).getRegion("_PARALLEL_GATEWAY_SENDER_QUEUE", true); } @@ -704,7 +704,7 @@ private static class TestableParallelGatewaySenderQueue extends ParallelGatewayS public TestableParallelGatewaySenderQueue(final AbstractGatewaySender sender, final Set> userRegions, final int idx, final int nDispatcher, final MetaRegionFactory metaRegionFactory) { - super(sender, userRegions, idx, nDispatcher, metaRegionFactory, false); + super(sender, userRegions, idx, nDispatcher, metaRegionFactory, false, false); } diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/serial/TestSerialGatewaySenderEventProcessor.java b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/serial/TestSerialGatewaySenderEventProcessor.java index a17b9a893b02..41ae28cfb1fd 100644 --- a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/serial/TestSerialGatewaySenderEventProcessor.java +++ b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/serial/TestSerialGatewaySenderEventProcessor.java @@ -25,7 +25,7 @@ public TestSerialGatewaySenderEventProcessor(AbstractGatewaySender sender, Strin } @Override - protected void initializeMessageQueue(String id, boolean cleanQueues) { + protected void initializeMessageQueue(String id, boolean cleanQueues, boolean isStopped) { // Overridden to not create the RegionQueue in the constructor. } diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParserTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParserTest.java new file mode 100644 index 000000000000..71f6be8d1d3c --- /dev/null +++ b/geode-core/src/test/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParserTest.java @@ -0,0 +1,196 @@ +/* + * 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.xmlcache; + +import static org.apache.geode.internal.cache.GemFireCacheImpl.DEFAULT_LOCK_LEASE; +import static org.apache.geode.internal.cache.GemFireCacheImpl.DEFAULT_LOCK_TIMEOUT; +import static org.apache.geode.internal.cache.GemFireCacheImpl.DEFAULT_SEARCH_TIMEOUT; +import static org.apache.geode.internal.cache.xmlcache.CacheXml.COPY_ON_READ; +import static org.apache.geode.internal.cache.xmlcache.CacheXml.IS_SERVER; +import static org.apache.geode.internal.cache.xmlcache.CacheXml.LOCK_LEASE; +import static org.apache.geode.internal.cache.xmlcache.CacheXml.LOCK_TIMEOUT; +import static org.apache.geode.internal.cache.xmlcache.CacheXml.MESSAGE_SYNC_INTERVAL; +import static org.apache.geode.internal.cache.xmlcache.CacheXml.REMOTE_DISTRIBUTED_SYSTEM_ID; +import static org.apache.geode.internal.cache.xmlcache.CacheXml.SEARCH_TIMEOUT; +import static org.apache.geode.internal.cache.xmlcache.CacheXml.STARTUP_ACTION; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.catchThrowable; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.xml.sax.SAXException; +import org.xml.sax.helpers.AttributesImpl; + +import org.apache.geode.InternalGemFireException; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; +import org.apache.geode.internal.cache.ha.HARegionQueue; +import org.apache.geode.internal.cache.wan.InternalGatewaySenderFactory; + + +public class CacheXmlParserTest { + + @Mock + private CacheCreation cacheCreation; + + @Before + public void setUp() { + cacheCreation = mock(CacheCreation.class); + } + + @Test + public void testStartCacheParametersSet() throws SAXException { + AttributesImpl attributes = new AttributesImpl(); + XmlGeneratorUtils.addAttribute(attributes, LOCK_LEASE, "10"); + XmlGeneratorUtils.addAttribute(attributes, LOCK_TIMEOUT, "15"); + XmlGeneratorUtils.addAttribute(attributes, SEARCH_TIMEOUT, "20"); + XmlGeneratorUtils.addAttribute(attributes, MESSAGE_SYNC_INTERVAL, "20"); + XmlGeneratorUtils.addAttribute(attributes, IS_SERVER, "true"); + XmlGeneratorUtils.addAttribute(attributes, COPY_ON_READ, "true"); + + CacheXmlParser parser = new CacheXmlParser(); + parser.startElement("http://geode.apache.org/schema/cache", "cache", "cache", attributes); + + // Check that parameters are set + CacheCreation cache = parser.getCacheCreation(); + assertThat(cache.getLockLease()).isEqualTo(10); + assertThat(cache.getLockTimeout()).isEqualTo(15); + assertThat(cache.getSearchTimeout()).isEqualTo(20); + assertThat(cache.getMessageSyncInterval()).isEqualTo(20); + assertThat(cache.isServer()).isTrue(); + assertThat(cache.getCopyOnRead()).isTrue(); + // Reset MessageSyncInterval to default value, because it is static variable + HARegionQueue.setMessageSyncInterval(HARegionQueue.DEFAULT_MESSAGE_SYNC_INTERVAL); + } + + @Test + public void testStartCacheParametersDefaultValues() throws SAXException { + AttributesImpl attributes = new AttributesImpl(); + + CacheXmlParser parser = new CacheXmlParser(); + parser.startElement("http://geode.apache.org/schema/cache", "cache", "cache", attributes); + + // Check that parameters are set to default values + CacheCreation cache = parser.getCacheCreation(); + assertThat(cache.getLockLease()).isEqualTo(DEFAULT_LOCK_LEASE); + assertThat(cache.getLockTimeout()).isEqualTo(DEFAULT_LOCK_TIMEOUT); + assertThat(cache.getSearchTimeout()).isEqualTo(DEFAULT_SEARCH_TIMEOUT); + assertThat(cache.getMessageSyncInterval()) + .isEqualTo(HARegionQueue.DEFAULT_MESSAGE_SYNC_INTERVAL); + assertThat(cache.isServer()).isFalse(); + assertThat(cache.getCopyOnRead()).isFalse(); + } + + @Test + public void testStartAndEndGatewaySender() throws SAXException { + AttributesImpl attributes = new AttributesImpl(); + + XmlGeneratorUtils.addAttribute(attributes, REMOTE_DISTRIBUTED_SYSTEM_ID, "1"); + XmlGeneratorUtils.addAttribute(attributes, CacheXml.ID, "gateway-sender"); + + InternalGatewaySenderFactory gatewaySenderFactory = mock(InternalGatewaySenderFactory.class); + when(cacheCreation.createGatewaySenderFactory()).thenReturn(gatewaySenderFactory); + + CacheXmlParser parser = new CacheXmlParser(cacheCreation); + parser.startElement("http://geode.apache.org/schema/cache", "gateway-sender", "gateway-sender", + attributes); + + parser.endElement("http://geode.apache.org/schema/cache", "gateway-sender", "gateway-sender"); + verify(gatewaySenderFactory).create("gateway-sender", 1); + } + + @Test + public void testStartGatewaySenderStartupActionParameterStart() throws SAXException { + AttributesImpl attributes = new AttributesImpl(); + XmlGeneratorUtils.addAttribute(attributes, STARTUP_ACTION, "start"); + + InternalGatewaySenderFactory gatewaySenderFactory = mock(InternalGatewaySenderFactory.class); + when(cacheCreation.createGatewaySenderFactory()).thenReturn(gatewaySenderFactory); + + CacheXmlParser parser = new CacheXmlParser(cacheCreation); + parser.startElement("http://geode.apache.org/schema/cache", "gateway-sender", "gateway-sender", + attributes); + + verify(gatewaySenderFactory).setStartupAction(GatewaySenderStartupAction.START); + } + + @Test + public void testStartGatewaySenderStartupActionParameterStop() throws SAXException { + AttributesImpl attributes = new AttributesImpl(); + XmlGeneratorUtils.addAttribute(attributes, STARTUP_ACTION, "stop"); + + InternalGatewaySenderFactory gatewaySenderFactory = mock(InternalGatewaySenderFactory.class); + when(cacheCreation.createGatewaySenderFactory()).thenReturn(gatewaySenderFactory); + + CacheXmlParser parser = new CacheXmlParser(cacheCreation); + parser.startElement("http://geode.apache.org/schema/cache", "gateway-sender", "gateway-sender", + attributes); + + verify(gatewaySenderFactory).setStartupAction(GatewaySenderStartupAction.STOP); + } + + @Test + public void testStartGatewaySenderStartupActionParameterPause() throws SAXException { + AttributesImpl attributes = new AttributesImpl(); + XmlGeneratorUtils.addAttribute(attributes, STARTUP_ACTION, "pause"); + + InternalGatewaySenderFactory gatewaySenderFactory = mock(InternalGatewaySenderFactory.class); + when(cacheCreation.createGatewaySenderFactory()).thenReturn(gatewaySenderFactory); + + CacheXmlParser parser = new CacheXmlParser(cacheCreation); + parser.startElement("http://geode.apache.org/schema/cache", "gateway-sender", "gateway-sender", + attributes); + + verify(gatewaySenderFactory).setStartupAction(GatewaySenderStartupAction.PAUSE); + } + + @Test + public void testStartGatewaySenderStartupActionParameterNull() throws SAXException { + AttributesImpl attributes = new AttributesImpl(); + + InternalGatewaySenderFactory gatewaySenderFactory = mock(InternalGatewaySenderFactory.class); + when(cacheCreation.createGatewaySenderFactory()).thenReturn(gatewaySenderFactory); + + CacheXmlParser parser = new CacheXmlParser(cacheCreation); + parser.startElement("http://geode.apache.org/schema/cache", "gateway-sender", "gateway-sender", + attributes); + + verify(gatewaySenderFactory).setStartupAction(GatewaySenderStartupAction.NONE); + } + + @Test + public void testGatewaySenderStartupActionParameterInvalidValue() { + AttributesImpl attributes = new AttributesImpl(); + XmlGeneratorUtils.addAttribute(attributes, CacheXml.ID, "sender1"); + XmlGeneratorUtils.addAttribute(attributes, STARTUP_ACTION, "pausede"); + + InternalGatewaySenderFactory gatewaySenderFactory = mock(InternalGatewaySenderFactory.class); + when(cacheCreation.createGatewaySenderFactory()).thenReturn(gatewaySenderFactory); + + CacheXmlParser parser = new CacheXmlParser(cacheCreation); + Throwable thrown = + catchThrowable(() -> parser.startElement("http://geode.apache.org/schema/cache", + "gateway-sender", "gateway-sender", attributes)); + + assertThat(thrown) + .isInstanceOf(InternalGemFireException.class) + .hasMessage( + "An invalid startup-action value (pausede) was configured for gateway sender sender1"); + } + +} diff --git a/geode-docs/configuring/cluster_config/gfsh_persist.html.md.erb b/geode-docs/configuring/cluster_config/gfsh_persist.html.md.erb index e7a4bb9473a2..d534d35f4756 100644 --- a/geode-docs/configuring/cluster_config/gfsh_persist.html.md.erb +++ b/geode-docs/configuring/cluster_config/gfsh_persist.html.md.erb @@ -85,6 +85,10 @@ The following `gfsh` commands cause the configuration to be written to all locat - `create gateway-receiver` - `destroy gateway-receiver` - `alter query-service` +- `start gateway-sender` +- `stop gateway-sender` +- `pause gateway-sender` +- `resume gateway-sender` **\*** Note that the `configure pdx` command must be executed *before* starting your data members. This command does not affect any currently running members in the system. Data members (with cluster configuration enabled) that are started after running this command will pick up the new PDX configuration. diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/CliUtils.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/CliUtils.java index ec464d56f2f9..6c6052fd7702 100755 --- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/CliUtils.java +++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/CliUtils.java @@ -33,6 +33,7 @@ import java.util.zip.Deflater; import java.util.zip.Inflater; +import org.apache.geode.cache.configuration.CacheConfig; import org.apache.geode.distributed.DistributedMember; import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.classloader.ClassPathLoader; @@ -87,6 +88,24 @@ public static String getMemberNameOrId(DistributedMember distributedMember) { return nameOrId; } + public static boolean updateGatewaySenderStartupAction(CacheConfig config, Object configObject) { + boolean gatewaySenderConfigUpdated = false; + List gatewaySenders = config.getGatewaySenders(); + if (gatewaySenders.isEmpty() || configObject == null) { + return false; + } + + CacheConfig.GatewaySender gatewaySenderConfig = ((CacheConfig.GatewaySender) configObject); + String gatewaySenderId = gatewaySenderConfig.getId(); + for (CacheConfig.GatewaySender gatewaySender : gatewaySenders) { + if (gatewaySender.getId().equals(gatewaySenderId)) { + gatewaySender.setStartupAction(gatewaySenderConfig.getStartupAction()); + gatewaySenderConfigUpdated = true; + } + } + return gatewaySenderConfigUpdated; + } + public static Set getMembersWithAsyncEventQueue(InternalCache cache, String queueId) { Set members = ManagementUtils.findMembers(null, null, cache); diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/PauseGatewaySenderCommand.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/PauseGatewaySenderCommand.java index 52f89106f364..c0d8b1c0166c 100644 --- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/PauseGatewaySenderCommand.java +++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/PauseGatewaySenderCommand.java @@ -15,6 +15,7 @@ package org.apache.geode.management.internal.cli.commands; + import java.util.Set; import javax.management.ObjectName; @@ -23,19 +24,24 @@ import org.springframework.shell.core.annotation.CliOption; import org.apache.geode.cache.Cache; +import org.apache.geode.cache.configuration.CacheConfig; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.distributed.DistributedMember; import org.apache.geode.management.GatewaySenderMXBean; import org.apache.geode.management.cli.CliMetaData; import org.apache.geode.management.cli.ConverterHint; -import org.apache.geode.management.cli.GfshCommand; +import org.apache.geode.management.cli.SingleGfshCommand; +import org.apache.geode.management.cli.UpdateAllConfigurationGroupsMarker; import org.apache.geode.management.internal.SystemManagementService; +import org.apache.geode.management.internal.cli.CliUtils; import org.apache.geode.management.internal.cli.result.model.ResultModel; import org.apache.geode.management.internal.cli.result.model.TabularResultModel; import org.apache.geode.management.internal.i18n.CliStrings; import org.apache.geode.management.internal.security.ResourceOperation; import org.apache.geode.security.ResourcePermission; -public class PauseGatewaySenderCommand extends GfshCommand { +public class PauseGatewaySenderCommand extends SingleGfshCommand implements + UpdateAllConfigurationGroupsMarker { @CliCommand(value = CliStrings.PAUSE_GATEWAYSENDER, help = CliStrings.PAUSE_GATEWAYSENDER__HELP) @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN) @@ -67,6 +73,7 @@ public ResultModel pauseGatewaySender(@CliOption(key = CliStrings.PAUSE_GATEWAYS } ResultModel resultModel = new ResultModel(); + boolean isGatewaySenderPaused = false; TabularResultModel resultData = resultModel.addTable(CliStrings.PAUSE_GATEWAYSENDER); for (DistributedMember member : dsMembers) { if (cache.getDistributedSystem().getDistributedMember().getId().equals(member.getId())) { @@ -87,6 +94,7 @@ public ResultModel pauseGatewaySender(@CliOption(key = CliStrings.PAUSE_GATEWAYS resultData.addMemberStatusResultRow(member.getId(), CliStrings.GATEWAY_OK, CliStrings.format( CliStrings.GATEWAY_SENDER_0_IS_PAUSED_ON_MEMBER_1, senderId, member.getId())); + isGatewaySenderPaused = true; } } else { resultData.addMemberStatusResultRow(member.getId(), @@ -102,6 +110,18 @@ public ResultModel pauseGatewaySender(@CliOption(key = CliStrings.PAUSE_GATEWAYS } } + // Persist new action to Cluster Configuration + if (isGatewaySenderPaused && onMember == null) { + CacheConfig.GatewaySender gatewaySenderConfig = new CacheConfig.GatewaySender(); + gatewaySenderConfig.setStartupAction(GatewaySenderStartupAction.PAUSE.name().toLowerCase()); + gatewaySenderConfig.setId(senderId); + resultModel.setConfigObject(gatewaySenderConfig); + } return resultModel; } + + @Override + public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) { + return CliUtils.updateGatewaySenderStartupAction(config, configObject); + } } diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ResumeGatewaySenderCommand.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ResumeGatewaySenderCommand.java index 8d40a4423df5..6eadfd2639fc 100644 --- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ResumeGatewaySenderCommand.java +++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ResumeGatewaySenderCommand.java @@ -23,19 +23,24 @@ import org.springframework.shell.core.annotation.CliOption; import org.apache.geode.cache.Cache; +import org.apache.geode.cache.configuration.CacheConfig; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.distributed.DistributedMember; import org.apache.geode.management.GatewaySenderMXBean; import org.apache.geode.management.cli.CliMetaData; import org.apache.geode.management.cli.ConverterHint; -import org.apache.geode.management.cli.GfshCommand; +import org.apache.geode.management.cli.SingleGfshCommand; +import org.apache.geode.management.cli.UpdateAllConfigurationGroupsMarker; import org.apache.geode.management.internal.SystemManagementService; +import org.apache.geode.management.internal.cli.CliUtils; import org.apache.geode.management.internal.cli.result.model.ResultModel; import org.apache.geode.management.internal.cli.result.model.TabularResultModel; import org.apache.geode.management.internal.i18n.CliStrings; import org.apache.geode.management.internal.security.ResourceOperation; import org.apache.geode.security.ResourcePermission; -public class ResumeGatewaySenderCommand extends GfshCommand { +public class ResumeGatewaySenderCommand extends SingleGfshCommand implements + UpdateAllConfigurationGroupsMarker { @CliCommand(value = CliStrings.RESUME_GATEWAYSENDER, help = CliStrings.RESUME_GATEWAYSENDER__HELP) @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN) @@ -66,6 +71,7 @@ public ResultModel resumeGatewaySender(@CliOption(key = CliStrings.RESUME_GATEWA GatewaySenderMXBean bean; ResultModel resultModel = new ResultModel(); + boolean isGatewaySenderResumed = false; TabularResultModel resultData = resultModel.addTable(CliStrings.RESUME_GATEWAYSENDER); for (DistributedMember member : dsMembers) { if (cache.getDistributedSystem().getDistributedMember().getId().equals(member.getId())) { @@ -81,6 +87,7 @@ public ResultModel resumeGatewaySender(@CliOption(key = CliStrings.RESUME_GATEWA resultData.addMemberStatusResultRow(member.getId(), CliStrings.GATEWAY_OK, CliStrings.format( CliStrings.GATEWAY_SENDER_0_IS_RESUMED_ON_MEMBER_1, senderId, member.getId())); + isGatewaySenderResumed = true; } else { resultData.addMemberStatusResultRow(member.getId(), CliStrings.GATEWAY_ERROR, @@ -101,6 +108,18 @@ public ResultModel resumeGatewaySender(@CliOption(key = CliStrings.RESUME_GATEWA } } + // Persist new action to Cluster Configuration + if (isGatewaySenderResumed && onMember == null) { + CacheConfig.GatewaySender gatewaySenderConfig = new CacheConfig.GatewaySender(); + gatewaySenderConfig.setStartupAction(GatewaySenderStartupAction.START.name().toLowerCase()); + gatewaySenderConfig.setId(senderId); + resultModel.setConfigObject(gatewaySenderConfig); + } return resultModel; } + + @Override + public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) { + return CliUtils.updateGatewaySenderStartupAction(config, configObject); + } } diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StartGatewaySenderCommand.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StartGatewaySenderCommand.java index d02277d4ac28..2bcb1d539123 100644 --- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StartGatewaySenderCommand.java +++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StartGatewaySenderCommand.java @@ -15,6 +15,7 @@ package org.apache.geode.management.internal.cli.commands; + import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -30,20 +31,25 @@ import org.springframework.shell.core.annotation.CliOption; import org.apache.geode.cache.Cache; +import org.apache.geode.cache.configuration.CacheConfig; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.distributed.DistributedMember; import org.apache.geode.logging.internal.executors.LoggingExecutors; import org.apache.geode.management.GatewaySenderMXBean; import org.apache.geode.management.cli.CliMetaData; import org.apache.geode.management.cli.ConverterHint; -import org.apache.geode.management.cli.GfshCommand; +import org.apache.geode.management.cli.SingleGfshCommand; +import org.apache.geode.management.cli.UpdateAllConfigurationGroupsMarker; import org.apache.geode.management.internal.SystemManagementService; +import org.apache.geode.management.internal.cli.CliUtils; import org.apache.geode.management.internal.cli.result.model.ResultModel; import org.apache.geode.management.internal.cli.result.model.TabularResultModel; import org.apache.geode.management.internal.i18n.CliStrings; import org.apache.geode.management.internal.security.ResourceOperation; import org.apache.geode.security.ResourcePermission; -public class StartGatewaySenderCommand extends GfshCommand { +public class StartGatewaySenderCommand extends SingleGfshCommand implements + UpdateAllConfigurationGroupsMarker { @CliCommand(value = CliStrings.START_GATEWAYSENDER, help = CliStrings.START_GATEWAYSENDER__HELP) @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN) @@ -173,6 +179,7 @@ public ResultModel startGatewaySender(@CliOption(key = CliStrings.START_GATEWAYS execService.shutdown(); } + boolean isGatewaySenderStarted = false; ResultModel resultModel = new ResultModel(); TabularResultModel resultData = resultModel.addTable(CliStrings.START_GATEWAYSENDER); for (Future> future : futures) { @@ -182,6 +189,9 @@ public ResultModel startGatewaySender(@CliOption(key = CliStrings.START_GATEWAYS memberStatus = future.get(); resultData.addMemberStatusResultRow(memberStatus.get(0), memberStatus.get(1), memberStatus.get(2)); + if (memberStatus.get(1).equals(CliStrings.GATEWAY_OK)) { + isGatewaySenderStarted = true; + } } catch (InterruptedException | ExecutionException ite) { resultData.addMemberStatusResultRow(member.getId(), CliStrings.GATEWAY_ERROR, @@ -191,6 +201,19 @@ public ResultModel startGatewaySender(@CliOption(key = CliStrings.START_GATEWAYS } execService.shutdown(); + // Persist new action to Cluster Configuration + if (onMember == null && isGatewaySenderStarted) { + CacheConfig.GatewaySender gatewaySenderConfig = new CacheConfig.GatewaySender(); + gatewaySenderConfig.setStartupAction(GatewaySenderStartupAction.START.name().toLowerCase()); + gatewaySenderConfig.setId(senderId); + resultModel.setConfigObject(gatewaySenderConfig); + } + return resultModel; } + + @Override + public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) { + return CliUtils.updateGatewaySenderStartupAction(config, configObject); + } } diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StopGatewaySenderCommand.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StopGatewaySenderCommand.java index 7603bd213631..d28a8e8f9fd8 100644 --- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StopGatewaySenderCommand.java +++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StopGatewaySenderCommand.java @@ -31,18 +31,23 @@ import org.apache.geode.annotations.VisibleForTesting; import org.apache.geode.cache.Cache; +import org.apache.geode.cache.configuration.CacheConfig; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.distributed.DistributedMember; import org.apache.geode.management.cli.CliMetaData; import org.apache.geode.management.cli.ConverterHint; -import org.apache.geode.management.cli.GfshCommand; +import org.apache.geode.management.cli.SingleGfshCommand; +import org.apache.geode.management.cli.UpdateAllConfigurationGroupsMarker; import org.apache.geode.management.internal.SystemManagementService; +import org.apache.geode.management.internal.cli.CliUtils; import org.apache.geode.management.internal.cli.result.model.ResultModel; import org.apache.geode.management.internal.cli.result.model.TabularResultModel; import org.apache.geode.management.internal.i18n.CliStrings; import org.apache.geode.management.internal.security.ResourceOperation; import org.apache.geode.security.ResourcePermission; -public class StopGatewaySenderCommand extends GfshCommand { +public class StopGatewaySenderCommand extends SingleGfshCommand implements + UpdateAllConfigurationGroupsMarker { private final ExecutorService executorService; private final StopGatewaySenderOnMember stopperOnMember; @@ -83,11 +88,11 @@ public ResultModel stopGatewaySender(@CliOption(key = CliStrings.STOP_GATEWAYSEN return ResultModel.createError(CliStrings.NO_MEMBERS_FOUND_MESSAGE); } - return executeStopGatewaySender(senderId.trim(), getCache(), dsMembers); + return executeStopGatewaySender(senderId.trim(), getCache(), dsMembers, onMember); } public ResultModel executeStopGatewaySender(String id, Cache cache, - Set dsMembers) { + Set dsMembers, String[] onMember) { List dsMembersList = new ArrayList<>(dsMembers); List>> callables = new ArrayList<>(); @@ -107,13 +112,14 @@ public ResultModel executeStopGatewaySender(String id, Cache cache, ite.getMessage())); } - return buildResultModelFromMembersResponses(id, dsMembersList, futures); + return buildResultModelFromMembersResponses(id, dsMembersList, futures, onMember); } private ResultModel buildResultModelFromMembersResponses(String id, - List dsMembers, List>> futures) { + List dsMembers, List>> futures, String[] onMember) { ResultModel resultModel = new ResultModel(); TabularResultModel resultData = resultModel.addTable(CliStrings.STOP_GATEWAYSENDER); + boolean isGatewaySenderStopped = false; Iterator memberIterator = dsMembers.iterator(); for (Future> future : futures) { DistributedMember member = memberIterator.next(); @@ -122,6 +128,9 @@ private ResultModel buildResultModelFromMembersResponses(String id, memberStatus = future.get(); resultData.addMemberStatusResultRow(memberStatus.get(0), memberStatus.get(1), memberStatus.get(2)); + if (memberStatus.get(1).equals(CliStrings.GATEWAY_OK)) { + isGatewaySenderStopped = true; + } } catch (InterruptedException | ExecutionException ite) { resultData.addMemberStatusResultRow(member.getId(), CliStrings.GATEWAY_ERROR, @@ -129,6 +138,14 @@ private ResultModel buildResultModelFromMembersResponses(String id, id, ite.getMessage())); } } + + // Persist new action to Cluster Configuration + if (isGatewaySenderStopped && onMember == null) { + CacheConfig.GatewaySender gatewaySenderConfig = new CacheConfig.GatewaySender(); + gatewaySenderConfig.setStartupAction(GatewaySenderStartupAction.STOP.name().toLowerCase()); + gatewaySenderConfig.setId(id); + resultModel.setConfigObject(gatewaySenderConfig); + } return resultModel; } @@ -137,4 +154,9 @@ interface StopGatewaySenderOnMember { List executeStopGatewaySenderOnMember(String id, Cache cache, SystemManagementService managementService, DistributedMember member); } + + @Override + public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) { + return CliUtils.updateGatewaySenderStartupAction(config, configObject); + } } diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderCreateFunction.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderCreateFunction.java index 9d26114ddff0..55ec483508e5 100644 --- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderCreateFunction.java +++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderCreateFunction.java @@ -26,8 +26,10 @@ import org.apache.geode.cache.wan.GatewaySender; import org.apache.geode.cache.wan.GatewaySender.OrderPolicy; import org.apache.geode.cache.wan.GatewaySenderFactory; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.cache.wan.GatewayTransportFilter; import org.apache.geode.internal.cache.execute.InternalFunction; +import org.apache.geode.internal.cache.wan.InternalGatewaySenderFactory; import org.apache.geode.logging.internal.log4j.api.LogService; import org.apache.geode.management.internal.cli.CliUtils; import org.apache.geode.management.internal.functions.CliFunctionResult; @@ -97,6 +99,12 @@ private GatewaySender createGatewaySender(Cache cache, gateway.setManualStart(manualStart); } + String startupAction = gatewaySenderCreateArgs.getStartupAction(); + if (startupAction != null) { + ((InternalGatewaySenderFactory) gateway) + .setStartupAction(GatewaySenderStartupAction.valueOf(startupAction.toUpperCase())); + } + Integer maxQueueMemory = gatewaySenderCreateArgs.getMaxQueueMemory(); if (maxQueueMemory != null) { gateway.setMaximumQueueMemory(maxQueueMemory); diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderFunctionArgs.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderFunctionArgs.java index 640e69bd2c6e..2087442cf1a2 100644 --- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderFunctionArgs.java +++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderFunctionArgs.java @@ -48,6 +48,7 @@ public class GatewaySenderFunctionArgs implements Serializable { private final List gatewayEventFilters; private final List gatewayTransportFilters; private final Boolean enforceThreadsConnectSameReceiver; + private final String startupAction; public GatewaySenderFunctionArgs(CacheConfig.GatewaySender sender) { id = sender.getId(); @@ -85,12 +86,17 @@ public GatewaySenderFunctionArgs(CacheConfig.GatewaySender sender) { .collect(Collectors.toList())) .orElse(null); enforceThreadsConnectSameReceiver = sender.getEnforceThreadsConnectSameReceiver(); + startupAction = sender.getStartupAction(); } private Integer string2int(String x) { return Optional.ofNullable(x).map(Integer::valueOf).orElse(null); } + public String getStartupAction() { + return this.startupAction; + } + public String getId() { return id; } diff --git a/geode-gfsh/src/main/resources/org/apache/geode/gfsh/internal/management/sanctioned-geode-gfsh-serializables.txt b/geode-gfsh/src/main/resources/org/apache/geode/gfsh/internal/management/sanctioned-geode-gfsh-serializables.txt index e9a15586e71a..fc3d6b9a163d 100644 --- a/geode-gfsh/src/main/resources/org/apache/geode/gfsh/internal/management/sanctioned-geode-gfsh-serializables.txt +++ b/geode-gfsh/src/main/resources/org/apache/geode/gfsh/internal/management/sanctioned-geode-gfsh-serializables.txt @@ -67,7 +67,7 @@ org/apache/geode/management/internal/cli/functions/GatewayReceiverCreateFunction org/apache/geode/management/internal/cli/functions/GatewaySenderCreateFunction,true,8746830191680509335 org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction,true,1 org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunctionArgs,true,3848480256348119530,id:java/lang/String,ifExists:boolean -org/apache/geode/management/internal/cli/functions/GatewaySenderFunctionArgs,true,4636678328980816780,alertThreshold:java/lang/Integer,batchSize:java/lang/Integer,batchTimeInterval:java/lang/Integer,diskStoreName:java/lang/String,diskSynchronous:java/lang/Boolean,dispatcherThreads:java/lang/Integer,enableBatchConflation:java/lang/Boolean,enablePersistence:java/lang/Boolean,enforceThreadsConnectSameReceiver:java/lang/Boolean,gatewayEventFilters:java/util/List,gatewayTransportFilters:java/util/List,groupTransactionEvents:java/lang/Boolean,id:java/lang/String,manualStart:java/lang/Boolean,maxQueueMemory:java/lang/Integer,orderPolicy:java/lang/String,parallel:java/lang/Boolean,remoteDSId:java/lang/Integer,socketBufferSize:java/lang/Integer,socketReadTimeout:java/lang/Integer +org/apache/geode/management/internal/cli/functions/GatewaySenderFunctionArgs,true,4636678328980816780,alertThreshold:java/lang/Integer,batchSize:java/lang/Integer,batchTimeInterval:java/lang/Integer,diskStoreName:java/lang/String,diskSynchronous:java/lang/Boolean,dispatcherThreads:java/lang/Integer,enableBatchConflation:java/lang/Boolean,enablePersistence:java/lang/Boolean,enforceThreadsConnectSameReceiver:java/lang/Boolean,gatewayEventFilters:java/util/List,gatewayTransportFilters:java/util/List,groupTransactionEvents:java/lang/Boolean,id:java/lang/String,manualStart:java/lang/Boolean,maxQueueMemory:java/lang/Integer,orderPolicy:java/lang/String,parallel:java/lang/Boolean,remoteDSId:java/lang/Integer,socketBufferSize:java/lang/Integer,socketReadTimeout:java/lang/Integer,startupAction:java/lang/String org/apache/geode/management/internal/cli/functions/GetMemberConfigInformationFunction,true,1 org/apache/geode/management/internal/cli/functions/GetRegionDescriptionFunction,true,1 org/apache/geode/management/internal/cli/functions/GetRegionsFunction,true,1 diff --git a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/PauseGatewaySenderCommandTest.java b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/PauseGatewaySenderCommandTest.java new file mode 100644 index 000000000000..227c8e504232 --- /dev/null +++ b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/PauseGatewaySenderCommandTest.java @@ -0,0 +1,287 @@ +/* + * 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.management.internal.cli.commands; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.util.HashSet; +import java.util.Set; + +import javax.management.ObjectName; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; + +import org.apache.geode.cache.Cache; +import org.apache.geode.cache.Region; +import org.apache.geode.cache.configuration.CacheConfig; +import org.apache.geode.distributed.DistributedMember; +import org.apache.geode.distributed.DistributedSystem; +import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService; +import org.apache.geode.internal.cache.AbstractRegion; +import org.apache.geode.internal.config.JAXBService; +import org.apache.geode.management.GatewaySenderMXBean; +import org.apache.geode.management.internal.SystemManagementService; +import org.apache.geode.test.junit.rules.GfshParserRule; + +public class PauseGatewaySenderCommandTest { + @ClassRule + public static GfshParserRule gfsh = new GfshParserRule(); + + private PauseGatewaySenderCommand command; + private InternalConfigurationPersistenceService service; + private final Set groupSet = new HashSet<>(); + private Set memberSet; + private DistributedMember distributedMember1; + private DistributedMember distributedMember2; + private Cache cache; + private DistributedSystem system; + private DistributedSystem locatorSystem; + private DistributedMember locatorDistributedMember; + private SystemManagementService serviceMgm; + private GatewaySenderMXBean bean1; + private GatewaySenderMXBean bean2; + private CacheConfig config; + private CacheConfig.GatewaySender gs1Config; + private ObjectName objectName1; + private ObjectName objectName2; + private static final String GATEWAY_SENDER_1 = "gatewaySender1"; + private static final String MEMBER_1 = "member1"; + private static final String MEMBER_2 = "member2"; + + @Before + public void setUp() { + command = spy(PauseGatewaySenderCommand.class); + service = + spy(new InternalConfigurationPersistenceService(JAXBService.create(CacheConfig.class))); + Region configRegion = mock(AbstractRegion.class); + cache = mock(Cache.class); + system = mock(DistributedSystem.class); + locatorSystem = mock(DistributedSystem.class); + locatorDistributedMember = mock(DistributedMember.class); + distributedMember1 = mock(DistributedMember.class); + distributedMember2 = mock(DistributedMember.class); + serviceMgm = mock(SystemManagementService.class); + bean1 = mock(GatewaySenderMXBean.class); + bean2 = mock(GatewaySenderMXBean.class); + objectName1 = mock(ObjectName.class); + objectName2 = mock(ObjectName.class); + + doReturn(service).when(command).getConfigurationPersistenceService(); + doReturn(cache).when(command).getCache(); + doReturn(serviceMgm).when(command).getManagementService(); + + doReturn(true).when(service).lockSharedConfiguration(); + doNothing().when(service).unlockSharedConfiguration(); + doReturn(null).when(service).getConfiguration(any()); + doReturn(configRegion).when(service).getConfigurationRegion(); + doCallRealMethod().when(service).updateCacheConfig(any(), any()); + + config = new CacheConfig(); + gs1Config = new CacheConfig.GatewaySender(); + gs1Config.setId(GATEWAY_SENDER_1); + + memberSet = new HashSet<>(); + } + + @Test + public void mandatoryOption() { + gfsh.executeAndAssertThat(command, "pause gateway-sender").statusIsError() + .containsOutput("Invalid command"); + } + + @Test + public void pauseGatewaySenderSuccessfully() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + doReturn(false).when(bean1).isPaused(); + + gfsh.executeAndAssertThat(command, "pause gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is paused on member " + MEMBER_1) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK"); + } + + @Test + public void pauseGatewaySenderOnTwoMembersSuccessfully() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(objectName1).when(serviceMgm).getGatewaySenderMBeanName(distributedMember1, + GATEWAY_SENDER_1); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + // member 2 + when(system.getDistributedMember()).thenReturn(distributedMember2); + memberSet.add(distributedMember2); + doReturn(MEMBER_2).when(distributedMember2).getId(); + doReturn(objectName2).when(serviceMgm).getGatewaySenderMBeanName(distributedMember2, + GATEWAY_SENDER_1); + doReturn(bean2).when(serviceMgm).getMBeanProxy(any(), any()); + + // setup for both members + doReturn(memberSet).when(command).findMembers(null, null); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + doReturn(true).when(bean2).isRunning(); + doReturn(false).when(bean1).isPaused(); + doReturn(false).when(bean2).isPaused(); + + + gfsh.executeAndAssertThat(command, "pause gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is paused on member " + MEMBER_1) + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is paused on member " + MEMBER_2) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK"); + } + + @Test + public void pauseGatewaySenderOnOneMembersSuccessfullyAndOneFail() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(objectName1).when(serviceMgm).getGatewaySenderMBeanName(distributedMember1, + GATEWAY_SENDER_1); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + // member 2 + when(system.getDistributedMember()).thenReturn(distributedMember2); + memberSet.add(distributedMember2); + doReturn(MEMBER_2).when(distributedMember2).getId(); + doReturn(objectName2).when(serviceMgm).getGatewaySenderMBeanName(distributedMember2, + GATEWAY_SENDER_1); + doReturn(null).when(serviceMgm).getMBeanProxy(objectName2, GatewaySenderMXBean.class); + + // setup for both members + doReturn(memberSet).when(command).findMembers(null, null); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + doReturn(false).when(bean1).isPaused(); + + + gfsh.executeAndAssertThat(command, "pause gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .containsOutput( + "GatewaySender " + GATEWAY_SENDER_1 + " is not available on member " + MEMBER_2) + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is paused on member " + MEMBER_1) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error", "OK"); + } + + @Test + public void memberAlreadyPaused() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + doReturn(true).when(bean1).isPaused(); + + gfsh.executeAndAssertThat(command, "pause gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .containsOutput( + "GatewaySender " + GATEWAY_SENDER_1 + " is already paused on member " + MEMBER_1) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error"); + } + + @Test + public void memberIsNotRunning() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(false).when(bean1).isRunning(); + + gfsh.executeAndAssertThat(command, "pause gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput( + "GatewaySender " + GATEWAY_SENDER_1 + " is not running on member " + MEMBER_1) + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error"); + } +} diff --git a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/ResumeGatewaySenderCommandTest.java b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/ResumeGatewaySenderCommandTest.java new file mode 100644 index 000000000000..7f7a0a226671 --- /dev/null +++ b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/ResumeGatewaySenderCommandTest.java @@ -0,0 +1,291 @@ +/* + * 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.management.internal.cli.commands; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.util.HashSet; +import java.util.Set; + +import javax.management.ObjectName; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; + +import org.apache.geode.cache.Cache; +import org.apache.geode.cache.Region; +import org.apache.geode.cache.configuration.CacheConfig; +import org.apache.geode.distributed.DistributedMember; +import org.apache.geode.distributed.DistributedSystem; +import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService; +import org.apache.geode.internal.cache.AbstractRegion; +import org.apache.geode.internal.config.JAXBService; +import org.apache.geode.management.GatewaySenderMXBean; +import org.apache.geode.management.internal.SystemManagementService; +import org.apache.geode.test.junit.rules.GfshParserRule; + +public class ResumeGatewaySenderCommandTest { + @ClassRule + public static GfshParserRule gfsh = new GfshParserRule(); + + private ResumeGatewaySenderCommand command; + private InternalConfigurationPersistenceService service; + private final Set groupSet = new HashSet<>(); + private Set memberSet; + private DistributedMember distributedMember1; + private DistributedMember distributedMember2; + private Cache cache; + private DistributedSystem system; + private DistributedSystem locatorSystem; + private DistributedMember locatorDistributedMember; + private SystemManagementService serviceMgm; + private GatewaySenderMXBean bean1; + private GatewaySenderMXBean bean2; + private CacheConfig config; + private CacheConfig.GatewaySender gs1Config; + private ObjectName objectName1; + private ObjectName objectName2; + private static final String GATEWAY_SENDER_1 = "gatewaySender1"; + private static final String MEMBER_1 = "member1"; + private static final String MEMBER_2 = "member2"; + + @Before + public void setUp() { + command = spy(ResumeGatewaySenderCommand.class); + service = + spy(new InternalConfigurationPersistenceService(JAXBService.create(CacheConfig.class))); + Region configRegion = mock(AbstractRegion.class); + cache = mock(Cache.class); + system = mock(DistributedSystem.class); + locatorSystem = mock(DistributedSystem.class); + locatorDistributedMember = mock(DistributedMember.class); + distributedMember1 = mock(DistributedMember.class); + distributedMember2 = mock(DistributedMember.class); + serviceMgm = mock(SystemManagementService.class); + bean1 = mock(GatewaySenderMXBean.class); + bean2 = mock(GatewaySenderMXBean.class); + objectName1 = mock(ObjectName.class); + objectName2 = mock(ObjectName.class); + + doReturn(service).when(command).getConfigurationPersistenceService(); + doReturn(cache).when(command).getCache(); + doReturn(serviceMgm).when(command).getManagementService(); + + doReturn(true).when(service).lockSharedConfiguration(); + doNothing().when(service).unlockSharedConfiguration(); + doReturn(null).when(service).getConfiguration(any()); + doReturn(configRegion).when(service).getConfigurationRegion(); + doCallRealMethod().when(service).updateCacheConfig(any(), any()); + + config = new CacheConfig(); + gs1Config = new CacheConfig.GatewaySender(); + gs1Config.setId(GATEWAY_SENDER_1); + + memberSet = new HashSet<>(); + } + + @Test + public void mandatoryOption() { + gfsh.executeAndAssertThat(command, "resume gateway-sender").statusIsError() + .containsOutput("Invalid command"); + } + + @Test + public void resumeGatewaySenderSuccessfully() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + doReturn(true).when(bean1).isPaused(); + + gfsh.executeAndAssertThat(command, "resume gateway-sender --id=gatewaySender1") + .statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is resumed on member " + MEMBER_1) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK"); + } + + @Test + public void resumeGatewaySenderOnTwoMembersSuccessfully() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(objectName1).when(serviceMgm).getGatewaySenderMBeanName(distributedMember1, + GATEWAY_SENDER_1); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + // member 2 + when(system.getDistributedMember()).thenReturn(distributedMember2); + memberSet.add(distributedMember2); + doReturn(MEMBER_2).when(distributedMember2).getId(); + doReturn(objectName2).when(serviceMgm).getGatewaySenderMBeanName(distributedMember2, + GATEWAY_SENDER_1); + doReturn(bean2).when(serviceMgm).getMBeanProxy(any(), any()); + + // setup for both members + doReturn(memberSet).when(command).findMembers(null, null); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + doReturn(true).when(bean2).isRunning(); + doReturn(true).when(bean1).isPaused(); + doReturn(true).when(bean2).isPaused(); + + gfsh.executeAndAssertThat(command, "resume gateway-sender --id=gatewaySender1") + .statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is resumed on member " + MEMBER_1) + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is resumed on member " + MEMBER_2) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK"); + } + + @Test + public void resumeGatewaySenderOnOneMembersSuccessfullyAndOneFail() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(objectName1).when(serviceMgm).getGatewaySenderMBeanName(distributedMember1, + GATEWAY_SENDER_1); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + // member 2 + when(system.getDistributedMember()).thenReturn(distributedMember2); + memberSet.add(distributedMember2); + doReturn(MEMBER_2).when(distributedMember2).getId(); + doReturn(objectName2).when(serviceMgm).getGatewaySenderMBeanName(distributedMember2, + GATEWAY_SENDER_1); + doReturn(null).when(serviceMgm).getMBeanProxy(objectName2, GatewaySenderMXBean.class); + + // setup for both members + doReturn(memberSet).when(command).findMembers(null, null); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + doReturn(true).when(bean1).isPaused(); + + + gfsh.executeAndAssertThat(command, "resume gateway-sender --id=gatewaySender1") + .statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .containsOutput( + "GatewaySender " + GATEWAY_SENDER_1 + " is not available on member " + MEMBER_2) + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is resumed on member " + MEMBER_1) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error", "OK"); + } + + @Test + public void memberAlreadyStarted() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + doReturn(false).when(bean1).isPaused(); + + gfsh.executeAndAssertThat(command, "resume gateway-sender --id=gatewaySender1") + .statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .containsOutput( + "GatewaySender " + GATEWAY_SENDER_1 + " is not paused on member " + MEMBER_1) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error"); + } + + @Test + public void memberIsNotRunning() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(false).when(bean1).isRunning(); + + gfsh.executeAndAssertThat(command, "resume gateway-sender --id=gatewaySender1") + .statusIsSuccess() + .containsOutput( + "GatewaySender " + GATEWAY_SENDER_1 + " is not running on member " + MEMBER_1) + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error"); + } +} diff --git a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/StartGatewaySenderCommandTest.java b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/StartGatewaySenderCommandTest.java new file mode 100644 index 000000000000..85edb8d5f038 --- /dev/null +++ b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/StartGatewaySenderCommandTest.java @@ -0,0 +1,318 @@ +/* + * 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.management.internal.cli.commands; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.util.HashSet; +import java.util.Set; + +import javax.management.ObjectName; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; + +import org.apache.geode.cache.Cache; +import org.apache.geode.cache.Region; +import org.apache.geode.cache.configuration.CacheConfig; +import org.apache.geode.distributed.DistributedMember; +import org.apache.geode.distributed.DistributedSystem; +import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService; +import org.apache.geode.internal.cache.AbstractRegion; +import org.apache.geode.internal.config.JAXBService; +import org.apache.geode.management.GatewaySenderMXBean; +import org.apache.geode.management.internal.SystemManagementService; +import org.apache.geode.test.junit.rules.GfshParserRule; + +public class StartGatewaySenderCommandTest { + @ClassRule + public static GfshParserRule gfsh = new GfshParserRule(); + + private StartGatewaySenderCommand command; + private InternalConfigurationPersistenceService service; + private final Set groupSet = new HashSet<>(); + private Set memberSet; + private DistributedMember distributedMember1; + private DistributedMember distributedMember2; + private Cache cache; + private DistributedSystem system; + private DistributedSystem locatorSystem; + private DistributedMember locatorDistributedMember; + private SystemManagementService serviceMgm; + private GatewaySenderMXBean bean1; + private GatewaySenderMXBean bean2; + private CacheConfig config; + private CacheConfig.GatewaySender gs1Config; + private ObjectName objectName1; + private ObjectName objectName2; + private static final String GATEWAY_SENDER_1 = "gatewaySender1"; + private static final String MEMBER_1 = "member1"; + private static final String MEMBER_2 = "member2"; + + @Before + public void setUp() { + command = spy(StartGatewaySenderCommand.class); + service = + spy(new InternalConfigurationPersistenceService(JAXBService.create(CacheConfig.class))); + Region configRegion = mock(AbstractRegion.class); + cache = mock(Cache.class); + system = mock(DistributedSystem.class); + locatorSystem = mock(DistributedSystem.class); + locatorDistributedMember = mock(DistributedMember.class); + distributedMember1 = mock(DistributedMember.class); + distributedMember2 = mock(DistributedMember.class); + serviceMgm = mock(SystemManagementService.class); + bean1 = mock(GatewaySenderMXBean.class); + bean2 = mock(GatewaySenderMXBean.class); + objectName1 = mock(ObjectName.class); + objectName2 = mock(ObjectName.class); + + doReturn(service).when(command).getConfigurationPersistenceService(); + doReturn(cache).when(command).getCache(); + doReturn(serviceMgm).when(command).getManagementService(); + + doReturn(true).when(service).lockSharedConfiguration(); + doNothing().when(service).unlockSharedConfiguration(); + doReturn(null).when(service).getConfiguration(any()); + doReturn(configRegion).when(service).getConfigurationRegion(); + doCallRealMethod().when(service).updateCacheConfig(any(), any()); + + config = new CacheConfig(); + gs1Config = new CacheConfig.GatewaySender(); + gs1Config.setId(GATEWAY_SENDER_1); + + memberSet = new HashSet<>(); + } + + @Test + public void mandatoryOption() { + gfsh.executeAndAssertThat(command, "start gateway-sender").statusIsError() + .containsOutput("Invalid command"); + } + + @Test + public void startGatewaySenderSuccessfully() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(false).when(bean1).isRunning(); + + gfsh.executeAndAssertThat(command, "start gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is started on member " + MEMBER_1) + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK"); + } + + @Test + public void startGatewaySenderOnTwoMembersSuccessfully() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(objectName1).when(serviceMgm).getGatewaySenderMBeanName(distributedMember1, + GATEWAY_SENDER_1); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + // member 2 + when(system.getDistributedMember()).thenReturn(distributedMember2); + memberSet.add(distributedMember2); + doReturn(MEMBER_2).when(distributedMember2).getId(); + doReturn(objectName2).when(serviceMgm).getGatewaySenderMBeanName(distributedMember2, + GATEWAY_SENDER_1); + doReturn(bean2).when(serviceMgm).getMBeanProxy(any(), any()); + + // setup for both members + doReturn(memberSet).when(command).findMembers(null, null); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(false).when(bean1).isRunning(); + doReturn(false).when(bean2).isRunning(); + + gfsh.executeAndAssertThat(command, "start gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is started on member " + MEMBER_1) + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is started on member " + MEMBER_2) + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK"); + } + + @Test + public void startGatewaySenderOnOneMembersSuccessfullyAndOneFail() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(objectName1).when(serviceMgm).getGatewaySenderMBeanName(distributedMember1, + GATEWAY_SENDER_1); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + // member 2 + when(system.getDistributedMember()).thenReturn(distributedMember2); + memberSet.add(distributedMember2); + doReturn(MEMBER_2).when(distributedMember2).getId(); + doReturn(objectName2).when(serviceMgm).getGatewaySenderMBeanName(distributedMember2, + GATEWAY_SENDER_1); + doReturn(null).when(serviceMgm).getMBeanProxy(objectName2, GatewaySenderMXBean.class); + + // setup for both members + doReturn(memberSet).when(command).findMembers(null, null); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(false).when(bean1).isRunning(); + + gfsh.executeAndAssertThat(command, "start gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .containsOutput( + "GatewaySender " + GATEWAY_SENDER_1 + " is not available on member " + MEMBER_2) + .containsOutput("GatewaySender " + GATEWAY_SENDER_1 + " is started on member " + MEMBER_1) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error", "OK"); + } + + @Test + public void startGatewaySenderTwoMemberCleanQueues() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(objectName1).when(serviceMgm).getGatewaySenderMBeanName(distributedMember1, + GATEWAY_SENDER_1); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + // member 2 + when(system.getDistributedMember()).thenReturn(distributedMember2); + memberSet.add(distributedMember2); + doReturn(MEMBER_2).when(distributedMember2).getId(); + doReturn(objectName2).when(serviceMgm).getGatewaySenderMBeanName(distributedMember2, + GATEWAY_SENDER_1); + doReturn(bean2).when(serviceMgm).getMBeanProxy(any(), any()); + + // setup for both members + doReturn(memberSet).when(command).findMembers(null, null); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(false).when(bean1).isRunning(); + doReturn(false).when(bean2).isRunning(); + + gfsh.executeAndAssertThat(command, + "start gateway-sender --id=gatewaySender1 --clean-queues=true").statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK"); + } + + @Test + public void memberAlreadyStarted() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + + gfsh.executeAndAssertThat(command, "start gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .containsOutput( + "GatewaySender " + GATEWAY_SENDER_1 + " is already started on member " + MEMBER_1) + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error"); + } + + @Test + public void memberIsNotAvailable() { + doReturn(locatorSystem).when(cache).getDistributedSystem(); + doReturn(locatorDistributedMember).when(locatorSystem).getDistributedMember(); + doReturn("locator").when(locatorDistributedMember).getId(); + + // member 1 + when(system.getDistributedMember()).thenReturn(distributedMember1); + memberSet.add(distributedMember1); + doReturn(memberSet).when(command).findMembers(null, null); + doReturn(MEMBER_1).when(distributedMember1).getId(); + doReturn(bean1).when(serviceMgm).getMBeanProxy(any(), any()); + config.getGatewaySenders().add(gs1Config); + + groupSet.add("cluster"); + doReturn(groupSet).when(service).getGroups(); + doReturn(config).when(service).getCacheConfig("cluster"); + doReturn(config).when(service).getCacheConfig("cluster", true); + + doReturn(true).when(bean1).isRunning(); + + gfsh.executeAndAssertThat(command, "start gateway-sender --id=gatewaySender1").statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error"); + } +} diff --git a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/StopGatewaySenderCommandTest.java b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/StopGatewaySenderCommandTest.java index 05c3b07cd140..8b65cc0f22cb 100644 --- a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/StopGatewaySenderCommandTest.java +++ b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/StopGatewaySenderCommandTest.java @@ -42,6 +42,8 @@ import org.mockito.ArgumentCaptor; import org.apache.geode.cache.Cache; +import org.apache.geode.cache.configuration.CacheConfig; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.distributed.DistributedMember; import org.apache.geode.management.internal.SystemManagementService; import org.apache.geode.management.internal.cli.result.model.ResultModel; @@ -113,7 +115,7 @@ public void stopGatewaySenderStartsOneThreadPerMemberAndBuildsOutputAccordingToF doReturn(managementService).when(command).getManagementService(); // act - ResultModel result = command.executeStopGatewaySender(senderId, cache, members); + ResultModel result = command.executeStopGatewaySender(senderId, cache, members, null); // assert assertThat(result.isSuccessful()).isTrue(); @@ -127,6 +129,11 @@ public void stopGatewaySenderStartsOneThreadPerMemberAndBuildsOutputAccordingToF assertThat(message).containsExactlyInAnyOrder(gatewaySenderIsStoppedMsg, gatewaySenderIsStoppedMsg, gatewaySenderIsStoppedMsg); + // check that cluster configuration is updated + CacheConfig.GatewaySender config = (CacheConfig.GatewaySender) result.getConfigObject(); + assertThat(config.getStartupAction()) + .isEqualTo(GatewaySenderStartupAction.STOP.name().toLowerCase()); + ArgumentCaptor callablesCaptor = ArgumentCaptor.forClass(Collection.class); verify(executorService, times(1)).invokeAll((callablesCaptor.capture())); @@ -144,12 +151,11 @@ public void stopGatewaySenderInterruptedReturnsError() throws InterruptedExcepti doReturn(managementService).when(command).getManagementService(); // act - ResultModel result = command.executeStopGatewaySender(senderId, cache, members); + ResultModel result = command.executeStopGatewaySender(senderId, cache, members, null); // assert assertThat(result.isSuccessful()).isFalse(); assertThat(result.getInfoSection("info").getContent().get(0)).isEqualTo( "Could not invoke stop gateway sender sender1 operation on members due to interruption2"); } - } diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/cache/wan/internal/cli/commands/WanCopyRegionCommandDistributedTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/cache/wan/internal/cli/commands/WanCopyRegionCommandDistributedTest.java index 759a919d40c4..da55cedc42f5 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/cache/wan/internal/cli/commands/WanCopyRegionCommandDistributedTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/cache/wan/internal/cli/commands/WanCopyRegionCommandDistributedTest.java @@ -34,11 +34,13 @@ 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.assertj.core.api.Assertions.fail; import java.text.NumberFormat; import java.text.ParseException; import java.util.Arrays; import java.util.Collections; +import java.util.LinkedList; import java.util.List; import java.util.Locale; import java.util.Properties; @@ -73,6 +75,7 @@ import org.apache.geode.management.internal.cli.util.CommandStringBuilder; import org.apache.geode.management.internal.i18n.CliStrings; 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.dunit.rules.DistributedErrorCollector; import org.apache.geode.test.dunit.rules.DistributedExecutorServiceRule; @@ -1166,12 +1169,28 @@ private int create2WanSitesAndClient(VM locatorInA, List serversInA, String private void createSender(boolean isParallelGatewaySender, List servers, String senderId, int remoteDsId) { - for (VM server : servers) { - server.invoke(() -> createSender(senderId, remoteDsId, isParallelGatewaySender, - 100, 10, false, - false, null, true)); + List> tasks = new LinkedList<>(); + for (VM vm : servers) { + tasks.add(vm.invokeAsync( + () -> vm.invoke(() -> { + IgnoredException ignoreInterruptedException = + addIgnoredException(InterruptedException.class.getName()); + try { + createSender(senderId, remoteDsId, isParallelGatewaySender, + 100, 10, false, + false, null, false); + } finally { + ignoreInterruptedException.remove(); + } + }))); + } + for (AsyncInvocation invocation : tasks) { + try { + invocation.await(); + } catch (InterruptedException e) { + fail("Creating and starting senders was interrupted"); + } } - startSenderInVMsAsync(senderId, servers.toArray(new VM[0])); } private void createReceivers(List serversInB, VM serverInC) { diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/ClusterConfigStartStopPauseAndResumeGatewaySenderOperationDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/ClusterConfigStartStopPauseAndResumeGatewaySenderOperationDUnitTest.java new file mode 100644 index 000000000000..077dd0e08057 --- /dev/null +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/ClusterConfigStartStopPauseAndResumeGatewaySenderOperationDUnitTest.java @@ -0,0 +1,750 @@ +/* + * 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; + +import static org.apache.geode.distributed.ConfigurationProperties.DISTRIBUTED_SYSTEM_ID; +import static org.apache.geode.distributed.ConfigurationProperties.REMOTE_LOCATORS; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.getMember; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.validateGatewayReceiverMXBeanProxy; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.validateGatewaySenderMXBeanProxy; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.verifyReceiverState; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.verifySenderState; +import static org.apache.geode.test.awaitility.GeodeAwaitility.await; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import org.apache.geode.cache.Region; +import org.apache.geode.cache.wan.GatewaySender; +import org.apache.geode.internal.cache.InternalCache; +import org.apache.geode.internal.cache.RegionQueue; +import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySenderEventProcessor; +import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySenderQueue; +import org.apache.geode.internal.cache.wan.parallel.ParallelGatewaySenderEventProcessor; +import org.apache.geode.internal.cache.wan.serial.ConcurrentSerialGatewaySenderEventProcessor; +import org.apache.geode.internal.cache.wan.serial.SerialGatewaySenderEventProcessor; +import org.apache.geode.internal.membership.utils.AvailablePort; +import org.apache.geode.management.internal.cli.util.CommandStringBuilder; +import org.apache.geode.management.internal.i18n.CliStrings; +import org.apache.geode.test.dunit.rules.ClientVM; +import org.apache.geode.test.dunit.rules.ClusterStartupRule; +import org.apache.geode.test.dunit.rules.MemberVM; +import org.apache.geode.test.junit.categories.WanTest; +import org.apache.geode.test.junit.rules.GfshCommandRule; + +@Category({WanTest.class}) +@RunWith(JUnitParamsRunner.class) +public class ClusterConfigStartStopPauseAndResumeGatewaySenderOperationDUnitTest + implements Serializable { + + @Rule + public ClusterStartupRule clusterStartupRule = new ClusterStartupRule(9); + + @Rule + public transient GfshCommandRule gfsh = new GfshCommandRule(); + + private MemberVM locatorSite1; + private MemberVM locatorSite2; + private MemberVM server1Site1; + private MemberVM server2Site1; + private MemberVM server1Site2; + private MemberVM server2Site2; + + private ClientVM clientSite1; + private ClientVM clientSite2; + + /** + * Verify that gateway-sender startup action is persisted after pause and resume gateway-sender + * commands are executed, and that gateway-sender works as expected after member restart: + * + * - Region type: PARTITION and non-redundant + * - Gateway sender configured without queue persistence + * + * 1. Pause gateway-sender + * 2. Run some traffic and verify that data is enqueued in gateway-sender queues + * 3. Restart all servers that host gateway-sender + * 4. Run some traffic and verify that data is enqueued in gateway-sender queues, old + * data should be lost from queue after servers are restarted + * 5. Resume gateway-sender + * 6. Verify that latest traffic is sent over the gateway-sender to remote site + */ + @Test + @Parameters({"true", "false"}) + public void testThatPauseStateRemainAfterTheRestartOfMembers(String isParallel) throws Exception { + configureSites(false, "PARTITION", "0", isParallel); + + executeGfshCommand(CliStrings.PAUSE_GATEWAYSENDER); + verifyGatewaySenderState(true, true); + server2Site2.invoke(() -> waitAllDispatcherThreadsToPause("ln")); + server1Site2.invoke(() -> waitAllDispatcherThreadsToPause("ln")); + + // Do some puts and check that data has been enqueued + Set keysQueue = clientSite2.invoke(() -> doPutsInRange(0, 15)); + server1Site2.invoke(() -> checkQueueSize("ln", keysQueue.size())); + + // stop servers on site #2 + server1Site2.stop(false); + server2Site2.stop(false); + + // start again servers in Site #2 + server1Site2 = clusterStartupRule.startServerVM(5, locatorSite2.getPort()); + server2Site2 = clusterStartupRule.startServerVM(6, locatorSite2.getPort()); + + verifyGatewaySenderState(true, true); + + // Do some puts and check that data has been enqueued, previous queue should be lost + Set keysQueue1 = clientSite2.invoke(() -> doPutsInRange(20, 35)); + server1Site2.invoke(() -> checkQueueSize("ln", keysQueue1.size())); + + executeGfshCommand(CliStrings.RESUME_GATEWAYSENDER); + verifyGatewaySenderState(true, false); + + // check that queue is empty + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + // check that data replicated to remote site + clientSite1.invoke(() -> checkDataAvailable(keysQueue1)); + } + + /** + * Verify that gateway-sender queue is persisted while in paused state and it is recovered after + * the restart of servers. + * + * - Region type: PARTITION_PERSISTENT and non-redundant + * - Gateway sender configured with queue persistence + * + * 1. Pause gateway-sender + * 2. Restart all servers that host gateway-sender + * 3. Run some traffic and verify that data is enqueued in gateway-sender queues + * 4. Restart all servers that host gateway-sender + * 5. Run some traffic and verify that data is enqueued in gateway-sender queues, old + * data should be recovered after servers restarted + * 6. Resume gateway-sender + * 5. Verify that complete traffic is sent over the gateway-sender to the remote site + */ + @Test + @Parameters({"true", "false"}) + public void testThatPauseStateRemainAfterRestartAllServersPersistent(String isParallel) + throws Exception { + configureSites(true, "PARTITION_PERSISTENT", "0", isParallel); + + executeGfshCommand(CliStrings.PAUSE_GATEWAYSENDER); + verifyGatewaySenderState(true, true); + server2Site2.invoke(() -> waitAllDispatcherThreadsToPause("ln")); + server1Site2.invoke(() -> waitAllDispatcherThreadsToPause("ln")); + + // Do some puts and check that data has been enqueued + Set keysQueue = clientSite2.invoke(() -> doPutsInRange(0, 15)); + server1Site2.invoke(() -> checkQueueSize("ln", keysQueue.size())); + + // stop servers on site #2 + server1Site2.stop(false); + server2Site2.stop(false); + + Thread thread = new Thread( + () -> server1Site2 = clusterStartupRule.startServerVM(5, locatorSite2.getPort())); + Thread thread1 = new Thread( + () -> server2Site2 = clusterStartupRule.startServerVM(6, locatorSite2.getPort())); + // start threads + thread.start(); + thread1.start(); + thread.join(); + thread.join(); + + verifyGatewaySenderState(true, true); + + // Do some puts and check that data has been enqueued, previous queue data should not be lost + Set keysQueue1 = clientSite2.invoke(() -> doPutsInRange(20, 35)); + server1Site2.invoke(() -> checkQueueSize("ln", keysQueue1.size() + keysQueue.size())); + + executeGfshCommand(CliStrings.RESUME_GATEWAYSENDER); + verifyGatewaySenderState(true, false); + + // check that queue is empty + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + // check that data replicated to remote site + clientSite1.invoke(() -> checkDataAvailable(keysQueue1)); + clientSite1.invoke(() -> checkDataAvailable(keysQueue)); + } + + /** + * Verify that gateway-sender is recovered from redundant server after the + * restart of member. + * + * - Region type: PARTITION and redundant + * - Gateway sender configured without queue persistence + * + * 1. Pause gateway-sender + * 2. Run some traffic and verify that data is enqueued in gateway-sender queues + * 3. Restart one server that host gateway-sender + * 4. Run some traffic and verify that data is enqueued in gateway-sender queues, old + * data should not be lost from queue after servers are restarted because of redundancy + * 5. Resume gateway-sender + * 6. Verify that queued traffic is sent over the gateway-sender to remote site + */ + @Test + @Parameters({"true", "false"}) + public void testThatPauseStateRemainAfterRestartOneServerRedundant(String isParallel) + throws Exception { + configureSites(false, "PARTITION", "1", isParallel); + + executeGfshCommand(CliStrings.PAUSE_GATEWAYSENDER); + verifyGatewaySenderState(true, true); + server2Site2.invoke(() -> waitAllDispatcherThreadsToPause("ln")); + server1Site2.invoke(() -> waitAllDispatcherThreadsToPause("ln")); + + // Do some puts and check that data has been enqueued + Set keys1 = clientSite2.invoke(() -> doPutsInRange(0, 15)); + server1Site2.invoke(() -> checkQueueSize("ln", keys1.size())); + + // stop server on site #2 + server1Site2.stop(false); + + // start again server in Site #2 + server1Site2 = clusterStartupRule.startServerVM(5, locatorSite2.getPort()); + + verifyGatewaySenderState(true, true); + + // Do some puts and check that data has been enqueued, previous queue should not be lost + // due to configured redundancy + Set keys = clientSite2.invoke(() -> doPutsInRange(20, 35)); + server1Site2.invoke(() -> checkQueueSize("ln", keys.size() + keys1.size())); + + executeGfshCommand(CliStrings.RESUME_GATEWAYSENDER); + verifyGatewaySenderState(true, false); + + // check that queue is empty + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + // check that data replicated to other site + clientSite1.invoke(() -> checkDataAvailable(keys)); + clientSite1.invoke(() -> checkDataAvailable(keys1)); + } + + /** + * Verify that gateway-sender startup action is persisted after stop and start gateway-sender + * commands are executed, and that gateway-sender works as expected after member restart: + * + * - Region type: PARTITION and non-redundant + * - Gateway sender configured without queue persistence + * + * 1. Stop gateway-sender + * 2. Run some traffic and verify that data is stored in partition region, and not replicated to + * the other site + * 3. Restart servers that host gateway-sender + * 4. Run some traffic and verify that partition region is recovered and that data is not + * replicated to the other site + * 5. Start gateway-sender + * 6. Run some traffic and verify that traffic is sent over the gateway-sender to remote site + */ + @Test + @Parameters({"true", "false"}) + public void testThatStopStateRemainAfterTheRestartOfMembers(String isParallel) throws Exception { + configureSites(false, "PARTITION", "0", isParallel); + + executeGfshCommand(CliStrings.STOP_GATEWAYSENDER); + verifyGatewaySenderState(false, false); + + Set keys1 = clientSite2.invoke(() -> doPutsInRange(0, 15)); + clientSite2.invoke(() -> checkDataAvailable(keys1)); + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + + // stop servers on site #2 + server1Site2.stop(false); + server2Site2.stop(false); + + // start again servers in Site #2 + server1Site2 = clusterStartupRule.startServerVM(5, locatorSite2.getPort()); + server2Site2 = clusterStartupRule.startServerVM(6, locatorSite2.getPort()); + + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + verifyGatewaySenderState(false, false); + + Set keys = clientSite2.invoke(() -> doPutsInRange(20, 35)); + clientSite2.invoke(() -> checkDataAvailable(keys)); + + executeGfshCommand(CliStrings.START_GATEWAYSENDER); + verifyGatewaySenderState(true, false); + + Set keys3 = clientSite2.invoke(() -> doPutsInRange(40, 55)); + clientSite2.invoke(() -> checkDataAvailable(keys3)); + clientSite1.invoke(() -> checkDataAvailable(keys3)); + } + + /** + * Verify that colocated partition regions (gws queue and region) are created + * after servers are restarted and gateway-sender is created in stopped state. + * + * - Region type: PARTITION_PERSISTENT and non-redundant + * - Gateway sender configured with queue persistence + * + * 1. Stop gateway-sender + * 2. Run some traffic and verify that data is stored in partition region, and not replicated to + * the remote site + * 3. Restart servers that host gateway-sender + * 4. Run some traffic and verify that partition region is recovered and that data is not + * replicated to the other site + * 5. Start gateway-sender + * 6. Run some traffic and verify that onl latest traffic is sent over the gateway-sender + * to remote site + */ + @Test + @Parameters({"true", "false"}) + public void testThatStopStateRemainAfterTheRestartOfMembersAndAllRegionsRecover(String isParallel) + throws Exception { + configureSites(true, "PARTITION_PERSISTENT", "0", isParallel); + + executeGfshCommand(CliStrings.STOP_GATEWAYSENDER); + verifyGatewaySenderState(false, false); + + Set keys1 = clientSite2.invoke(() -> doPutsInRange(0, 15)); + clientSite2.invoke(() -> checkDataAvailable(keys1)); + + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + + // stop servers on site #2 + server1Site2.stop(false); + server2Site2.stop(false); + + Thread thread = new Thread( + () -> server1Site2 = clusterStartupRule.startServerVM(5, locatorSite2.getPort())); + Thread thread1 = new Thread( + () -> server2Site2 = clusterStartupRule.startServerVM(6, locatorSite2.getPort())); + // start threads + thread.start(); + thread1.start(); + thread.join(); + thread1.join(); + + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + verifyGatewaySenderState(false, false); + + // check that partition region is created and that accepts new traffic + Set keys = clientSite2.invoke(() -> doPutsInRange(20, 35)); + clientSite2.invoke(() -> checkDataAvailable(keys)); + + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + + executeGfshCommand(CliStrings.START_GATEWAYSENDER); + verifyGatewaySenderState(true, false); + + Set keys3 = clientSite2.invoke(() -> doPutsInRange(40, 55)); + clientSite2.invoke(() -> checkDataAvailable(keys3)); + clientSite1.invoke(() -> checkDataAvailable(keys3)); + } + + /** + * Verify that parallel gateway-sender queue recovers only data enqueued prior to stop command + * from disk-store. + * + * - Region type: PARTITION_PERSISTENT and non-redundant + * - Gateway sender configured with queue persistence + * + * 1. Pause gateway-sender + * 2. Run some traffic and verify that data is enqueued in gateway-sender queues + * 3. Stop gateway-sender + * 4. Run some traffic and verify that data is stored in region, and not enqueued + * 6. Restart all servers + * 7. Check that data that is enqueued prior to stop command is recovered from persistent storage, + * and that gateway-sender remained in stopped state + * 8. Start gateway-senders + * 9. Check that data is not replicated to remote site + */ + @Test + @Parameters({"false", "true"}) + public void testThatStopStateRemainAfterTheRestartAndQueueDataIsRecovered(String isParallel) + throws Exception { + configureSites(true, "PARTITION_PERSISTENT", "0", isParallel); + + executeGfshCommand(CliStrings.PAUSE_GATEWAYSENDER); + verifyGatewaySenderState(true, true); + server2Site2.invoke(() -> waitAllDispatcherThreadsToPause("ln")); + server1Site2.invoke(() -> waitAllDispatcherThreadsToPause("ln")); + + Set keysQueued = clientSite2.invoke(() -> doPutsInRange(70, 85)); + clientSite2.invoke(() -> checkDataAvailable(keysQueued)); + server1Site2.invoke(() -> checkQueueSize("ln", keysQueued.size())); + + executeGfshCommand(CliStrings.STOP_GATEWAYSENDER); + + verifyGatewaySenderState(false, isParallel.equals("true")); + + Set keysNotQueued = clientSite2.invoke(() -> doPutsInRange(100, 105)); + clientSite2.invoke(() -> checkDataAvailable(keysNotQueued)); + + // stop servers on site #2 + server1Site2.stop(false); + server2Site2.stop(false); + + // start again servers in Site #2 + Thread thread = new Thread( + () -> server1Site2 = clusterStartupRule.startServerVM(5, locatorSite2.getPort())); + Thread thread1 = new Thread( + () -> server2Site2 = clusterStartupRule.startServerVM(6, locatorSite2.getPort())); + // start threads + thread.start(); + thread1.start(); + thread.join(); + thread1.join(); + + if (isParallel.equals("true")) { + server1Site2.invoke(() -> checkQueueSize("ln", keysQueued.size())); + } + verifyGatewaySenderState(false, false); + executeGfshCommand(CliStrings.START_GATEWAYSENDER); + verifyGatewaySenderState(true, false); + + // Check that data is sent over the gateway-sender + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + clientSite1.invoke(() -> checkDataAvailable(keysQueued)); + clientSite1.invoke(() -> checkDataNotAvailable(keysNotQueued)); + } + + /** + * This test case verifies that the server during recovery of gateway-sender in stopped state + * stores events in tmpDroppedEvents queue, and handles them after sender has been recovered. + */ + @Test + public void testEventsAreQueuedInTmpDroppedEventQueueDuringRecoveryOfSenderInStoppedState() + throws Exception { + String property = "enable-test-hook-temp-dropped-events"; + try { + setSystemPropertyOnServersSite2(property, "true"); + configureSites(true, "PARTITION_REDUNDANT_PERSISTENT", "1", "true"); + + List allMembers = new ArrayList<>(); + allMembers.add(server1Site2); + allMembers.add(server2Site2); + + executeGfshCommand(CliStrings.STOP_GATEWAYSENDER); + verifyGatewaySenderState(false, false); + + Set keys1 = clientSite2.invoke(() -> doPutsInRange(0, 500)); + clientSite2.invoke(() -> checkDataAvailable(keys1)); + server1Site2.invoke(() -> checkQueueSize("ln", 0)); + + MemberVM startServer = clusterStartupRule.startServerVM(9, locatorSite2.getPort()); + + // perform rebalance operation to redistribute primaries on running servers + String command = new CommandStringBuilder(CliStrings.REBALANCE) + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + startServer.stop(false); + + Thread thread = new Thread( + () -> clientSite2.invoke(() -> doPutsInRange(0, 15000))); + thread.start(); + + startServer = clusterStartupRule.startServerVM(9, locatorSite2.getPort()); + + allMembers.add(startServer); + + for (MemberVM member : allMembers) { + member.invoke(() -> { + // test that non of the events are stored in primary and secondary buckets + testLocalQueueIsEmpty("ln"); + // check that tmpDroppedEvent queue has been drained after it recovered in stopped state + verifyTmpDroppedEventSize("ln", 0); + }); + } + + // check that events are stored in tmp dropped queue during the startup of server + startServer.invoke(() -> verifyThatTempDroppedEventsHookIsTriggered("ln")); + } finally { + setSystemPropertyOnServersSite2(property, null); + } + } + + public static void verifyTmpDroppedEventSize(String senderId, int size) { + GatewaySender sender = getGatewaySender(senderId); + + AbstractGatewaySender ags = (AbstractGatewaySender) sender; + await().untilAsserted(() -> assertEquals("Expected tmpDroppedEvents size: " + size + + " but actual size: " + ags.getTempDroppedEventSize(), size, + ags.getTempDroppedEventSize())); + } + + public static void verifyThatTempDroppedEventsHookIsTriggered(String senderId) { + GatewaySender sender = getGatewaySender(senderId); + + AbstractGatewaySender ags = (AbstractGatewaySender) sender; + await().untilAsserted( + () -> assertNotEquals("Expected tmpDroppedEvents to be queued in tmpDroppedEvents queue", 0, + ags.getTempDroppedEventsHookSize())); + } + + public static void testLocalQueueIsEmpty(String senderId) { + await() + .untilAsserted(() -> assertThatLocalQueueSizeIsZero(senderId)); + } + + public static void assertThatLocalQueueSizeIsZero(String senderId) { + GatewaySender sender = getGatewaySender(senderId); + int totalSize = 0; + Set queues = ((AbstractGatewaySender) sender).getQueues(); + if (queues != null) { + for (RegionQueue q : queues) { + ConcurrentParallelGatewaySenderQueue prQ = (ConcurrentParallelGatewaySenderQueue) q; + totalSize += prQ.localSize(true); + } + } + assertEquals(0, totalSize); + } + + private void setSystemPropertyOnServersSite2(String key, String value) { + if (value != null) { + clusterStartupRule.getVM(5).invoke(() -> System.setProperty(key, value)); + clusterStartupRule.getVM(6).invoke(() -> System.setProperty(key, value)); + clusterStartupRule.getVM(9).invoke(() -> System.setProperty(key, value)); + } else { + clusterStartupRule.getVM(5).invoke(() -> System.clearProperty(key)); + clusterStartupRule.getVM(6).invoke(() -> System.clearProperty(key)); + clusterStartupRule.getVM(9).invoke(() -> System.clearProperty(key)); + } + } + + void configureSites(boolean enableGWSPersistence, String regionShortcut, String redundancy, + String isParallel) + throws Exception { + String enablePersistenceParameter = "false"; + if (enableGWSPersistence) { + enablePersistenceParameter = "true"; + } + + // Start locators for site #1 + Properties props = new Properties(); + props.setProperty(DISTRIBUTED_SYSTEM_ID, "" + 1); + locatorSite1 = clusterStartupRule.startLocatorVM(1, props); + + // Start locators for site #2 + props.setProperty(DISTRIBUTED_SYSTEM_ID, "" + 2); + props.setProperty(REMOTE_LOCATORS, "localhost[" + locatorSite1.getPort() + "]"); + locatorSite2 = clusterStartupRule.startLocatorVM(2, props); + + // Start servers and gateway-receiver for site #1 + Properties properties = new Properties(); + server1Site1 = clusterStartupRule.startServerVM(3, properties, locatorSite1.getPort()); + server2Site1 = clusterStartupRule.startServerVM(4, properties, locatorSite1.getPort()); + + connectGfshToSite(locatorSite1); + CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_GATEWAYRECEIVER); + csb.addOptionWithValueCheck(CliStrings.CREATE_GATEWAYRECEIVER__MANUALSTART, "false"); + csb.addOption(CliStrings.CREATE_GATEWAYRECEIVER__STARTPORT, + "" + AvailablePort.AVAILABLE_PORTS_LOWER_BOUND); + csb.addOption(CliStrings.CREATE_GATEWAYRECEIVER__ENDPORT, + "" + AvailablePort.AVAILABLE_PORTS_UPPER_BOUND); + gfsh.executeAndAssertThat(csb.getCommandString()).statusIsSuccess(); + + // verify that gateway-receiver has been successfully created on site #1 + server1Site1.invoke(() -> verifyReceiverState(true)); + server2Site1.invoke(() -> verifyReceiverState(true)); + locatorSite1 + .invoke(() -> validateGatewayReceiverMXBeanProxy(getMember(server1Site1.getVM()), true)); + locatorSite1 + .invoke(() -> validateGatewayReceiverMXBeanProxy(getMember(server2Site1.getVM()), true)); + + // create partition region on site #1 + csb = new CommandStringBuilder(CliStrings.CREATE_REGION); + csb.addOption(CliStrings.CREATE_REGION__REGION, "test1"); + csb.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, regionShortcut); + csb.addOption(CliStrings.CREATE_REGION__REDUNDANTCOPIES, redundancy); + + gfsh.executeAndAssertThat(csb.toString()).statusIsSuccess(); + + // Start client for site #1 + clientSite1 = + clusterStartupRule.startClientVM(8, c -> c.withLocatorConnection(locatorSite1.getPort())); + clientSite1.invoke(() -> { + ClusterStartupRule.clientCacheRule.createProxyRegion("test1"); + }); + + // start servers for site #2 + server1Site2 = clusterStartupRule.startServerVM(5, locatorSite2.getPort()); + server2Site2 = clusterStartupRule.startServerVM(6, locatorSite2.getPort()); + + // create gateway-sender on site #2 + connectGfshToSite(locatorSite2); + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "1") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, isParallel) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE, enablePersistenceParameter) + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1Site2.invoke(() -> verifySenderState("ln", true, false)); + server2Site2.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite2.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1Site2.getVM()), "ln", true, false)); + locatorSite2.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2Site2.getVM()), "ln", true, false)); + + // create partition region on site #2 + csb = new CommandStringBuilder(CliStrings.CREATE_REGION); + csb.addOption(CliStrings.CREATE_REGION__REGION, "test1"); + csb.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, regionShortcut); + csb.addOption(CliStrings.CREATE_REGION__GATEWAYSENDERID, "ln"); + csb.addOption(CliStrings.CREATE_REGION__REDUNDANTCOPIES, redundancy); + gfsh.executeAndAssertThat(csb.toString()).statusIsSuccess(); + + // Start client + clientSite2 = + clusterStartupRule.startClientVM(7, c -> c.withLocatorConnection(locatorSite2.getPort())); + clientSite2.invoke(() -> { + ClusterStartupRule.clientCacheRule.createProxyRegion("test1"); + }); + } + + void connectGfshToSite(MemberVM locator) throws Exception { + if (gfsh.isConnected()) { + gfsh.disconnect(); + } + gfsh.connectAndVerify(locator); + } + + void verifyGatewaySenderState(boolean isRunning, boolean isPaused) { + locatorSite2.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1Site2.getVM()), "ln", isRunning, + isPaused)); + locatorSite2.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2Site2.getVM()), "ln", isRunning, + isPaused)); + server1Site2.invoke(() -> verifySenderState("ln", isRunning, isPaused)); + server2Site2.invoke(() -> verifySenderState("ln", isRunning, isPaused)); + } + + private void executeGfshCommand(String cliCommand) { + String command = new CommandStringBuilder(cliCommand) + .addOption("id", "ln") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + } + + Set doPutsInRange(int start, int stop) { + Region region = + ClusterStartupRule.clientCacheRule.getCache().getRegion("test1"); + Set keys = new HashSet<>(); + for (int i = start; i < stop; i++) { + region.put(i + "key", i + "value"); + keys.add(i + "key"); + } + return keys; + } + + private void checkDataAvailable(Set keys) { + await() + .untilAsserted(() -> assertEquals(keys.size(), allKeysInRegion(keys))); + } + + private void checkDataNotAvailable(Set keys) { + assertNotEquals(keys.size(), allKeysInRegion(keys)); + } + + private int allKeysInRegion(Set keys) { + Region region = + ClusterStartupRule.clientCacheRule.getCache().getRegion("test1"); + Map data = region.getAll(keys); + int size = 0; + for (String dat : data.values()) { + if (dat != null) { + size++; + } + } + return size; + } + + public static GatewaySender getGatewaySender(String senderId) { + assertThat(ClusterStartupRule.getCache()).isNotNull(); + InternalCache internalCache = ClusterStartupRule.getCache(); + return internalCache.getGatewaySender(senderId); + } + + public static void checkQueueSize(String senderId, int numQueueEntries) { + await() + .untilAsserted(() -> testQueueSize(senderId, numQueueEntries)); + } + + public static void testQueueSize(String senderId, int numQueueEntries) { + GatewaySender sender = getGatewaySender(senderId); + int totalSize = 0; + Set queues = ((AbstractGatewaySender) sender).getQueues(); + if (sender.isParallel()) { + // Parallel gateway sender queues must be not null when recovered in stopped state + assertNotNull(queues); + for (RegionQueue q : queues) { + ConcurrentParallelGatewaySenderQueue prQ = (ConcurrentParallelGatewaySenderQueue) q; + totalSize += prQ.size(); + } + } else { + if (queues != null) { + for (RegionQueue q : queues) { + totalSize += q.size(); + } + } + } + assertEquals(numQueueEntries, totalSize); + } + + /* + * The batch dispatcher thread blocks and waits for configured time (batch-time-interval) to + * read new events. The batch-time-interval default value is 1000 milliseconds. So even if + * gateway-sender is paused it will still collect all events (for batch) received within these + * 1000 milliseconds and dispatch them. After 1000 milliseconds expire dispatcher thread will be + * actually paused. So it is necessary to wait for all dispatching threads to pause before + * sending new traffic. + */ + public static void waitAllDispatcherThreadsToPause(String senderId) { + await() + .untilAsserted(() -> testDispatcherThreadsToPause(senderId)); + } + + public static void testDispatcherThreadsToPause(String senderId) { + AbstractGatewaySender sender = (AbstractGatewaySender) getGatewaySender(senderId); + if (sender.isParallel()) { + ConcurrentParallelGatewaySenderEventProcessor abProc = + (ConcurrentParallelGatewaySenderEventProcessor) sender.getEventProcessor(); + List lproc = abProc.getProcessors(); + assertFalse(lproc.isEmpty()); + for (ParallelGatewaySenderEventProcessor serialProc : lproc) { + serialProc.waitForDispatcherToPause(); + } + } else { + ConcurrentSerialGatewaySenderEventProcessor abProc = + (ConcurrentSerialGatewaySenderEventProcessor) sender.getEventProcessor(); + List lproc = abProc.getProcessors(); + assertFalse(lproc.isEmpty()); + for (SerialGatewaySenderEventProcessor serialProc : lproc) { + serialProc.waitForDispatcherToPause(); + } + } + } +} diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/WANTestBase.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/WANTestBase.java index aaa8551c073b..a0e5d918786a 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/WANTestBase.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/WANTestBase.java @@ -43,6 +43,7 @@ import static org.apache.geode.test.util.ResourceUtils.createTempFileFromResource; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; +import static org.junit.Assert.assertEquals; import java.io.File; import java.io.IOException; @@ -3476,10 +3477,9 @@ public static void verifyTmpDroppedEventSize(String senderId, int size) { GatewaySender sender = getGatewaySender(senderId); AbstractGatewaySender ags = (AbstractGatewaySender) sender; - await().untilAsserted(() -> assertThat(ags.getTmpDroppedEventSize()).as( - "Expected tmpDroppedEvents size: " + size - + " but actual size: " + ags.getTmpDroppedEventSize()) - .isEqualTo(size)); + await().untilAsserted(() -> assertEquals("Expected tmpDroppedEvents size: " + size + + " but actual size: " + ags.getTempDroppedEventSize(), size, + ags.getTempDroppedEventSize())); } /** diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPersistenceEnabledGatewaySenderDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPersistenceEnabledGatewaySenderDUnitTest.java index 96093b70f5c9..46d761ded4e0 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPersistenceEnabledGatewaySenderDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPersistenceEnabledGatewaySenderDUnitTest.java @@ -184,8 +184,6 @@ protected SerializableRunnableIF createPartitionedRegionRunnable() { isOffHeap()); } - - /** * Enable persistence for GatewaySender. Pause the sender and do some puts in local region. Close * the local site and rebuild the region and sender from disk store. Dispatcher should not start @@ -480,7 +478,8 @@ public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart() { * all the events. */ @Test - public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart2() { + public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart2() + throws InterruptedException { // create locator on local site Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); // create locator on remote site @@ -558,25 +557,19 @@ public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart2() { logger.info("Created the senders back from the disk store."); - // create PR on local site - AsyncInvocation inv1 = vm4.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion1 = vm4.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv2 = vm5.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion2 = vm5.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv3 = vm6.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion3 = vm6.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv4 = vm7.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion4 = vm7.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - try { - inv1.join(); - inv2.join(); - inv3.join(); - inv4.join(); - } catch (InterruptedException e) { - e.printStackTrace(); - fail(); - } + createPersistentPartitionRegion1.await(); + createPersistentPartitionRegion2.await(); + createPersistentPartitionRegion3.await(); + createPersistentPartitionRegion4.await(); logger.info("Created back the partitioned regions"); @@ -631,7 +624,8 @@ public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart2() { * puts on the local region. Check if the remote site receives all the events. */ @Test - public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart_Scenario2() { + public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart_Scenario2() + throws InterruptedException { // create locator on local site Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); // create locator on remote site @@ -718,25 +712,19 @@ public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart_Scenario logger.info("Created the senders back from the disk store."); - // create PR on local site - AsyncInvocation inv1 = vm4.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion1 = vm4.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv2 = vm5.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion2 = vm5.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv3 = vm6.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion3 = vm6.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv4 = vm7.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion4 = vm7.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - try { - inv1.join(); - inv2.join(); - inv3.join(); - inv4.join(); - } catch (InterruptedException e) { - e.printStackTrace(); - fail(); - } + createPersistentPartitionRegion1.await(); + createPersistentPartitionRegion2.await(); + createPersistentPartitionRegion3.await(); + createPersistentPartitionRegion4.await(); logger.info("Created back the partitioned regions"); @@ -774,7 +762,8 @@ public void testPersistentPRWithGatewaySenderPersistenceEnabled_Restart_Scenario * if the remote site receives all the events. */ @Test - public void testPersistentPRWithPersistentGatewaySender_Restart_Bug44275() { + public void testPersistentPRWithPersistentGatewaySender_Restart_Bug44275() + throws InterruptedException { // create locator on local site Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); // create locator on remote site @@ -863,24 +852,19 @@ public void testPersistentPRWithPersistentGatewaySender_Restart_Bug44275() { logger.info("Created the senders back from the disk store."); // create PR on local site - AsyncInvocation inv1 = vm4.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion1 = vm4.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv2 = vm5.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion2 = vm5.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv3 = vm6.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion3 = vm6.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv4 = vm7.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion4 = vm7.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - try { - inv1.join(); - inv2.join(); - inv3.join(); - inv4.join(); - } catch (InterruptedException e) { - e.printStackTrace(); - fail(); - } + createPersistentPartitionRegion1.await(); + createPersistentPartitionRegion2.await(); + createPersistentPartitionRegion3.await(); + createPersistentPartitionRegion4.await(); logger.info("Created back the partitioned regions"); @@ -909,7 +893,8 @@ public void testPersistentPRWithPersistentGatewaySender_Restart_Bug44275() { * if the remote site receives all the events. */ @Test - public void testPersistentPRWithPersistentGatewaySender_Restart_DoOps() { + public void testPersistentPRWithPersistentGatewaySender_Restart_DoOps() + throws InterruptedException { // create locator on local site Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); // create locator on remote site @@ -997,14 +982,19 @@ public void testPersistentPRWithPersistentGatewaySender_Restart_DoOps() { logger.info("Created the senders back from the disk store."); // create PR on local site - vm4.invoke(() -> WANTestBase.createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, - 100, isOffHeap())); - vm5.invoke(() -> WANTestBase.createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, - 100, isOffHeap())); - vm6.invoke(() -> WANTestBase.createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, - 100, isOffHeap())); - vm7.invoke(() -> WANTestBase.createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, - 100, isOffHeap())); + AsyncInvocation createPersistentPartitionRegion1 = vm4.invokeAsync(() -> WANTestBase + .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); + AsyncInvocation createPersistentPartitionRegion2 = vm5.invokeAsync(() -> WANTestBase + .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); + AsyncInvocation createPersistentPartitionRegion3 = vm6.invokeAsync(() -> WANTestBase + .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); + AsyncInvocation createPersistentPartitionRegion4 = vm7.invokeAsync(() -> WANTestBase + .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); + + createPersistentPartitionRegion1.await(); + createPersistentPartitionRegion2.await(); + createPersistentPartitionRegion3.await(); + createPersistentPartitionRegion4.await(); logger.info("Created back the partitioned regions"); @@ -1041,7 +1031,7 @@ public void testPersistentPRWithPersistentGatewaySender_Restart_DoOps() { } @Test - public void testPersistentPR_Restart() { + public void testPersistentPR_Restart() throws InterruptedException { // create locator on local site Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); @@ -1077,35 +1067,19 @@ public void testPersistentPR_Restart() { logger.info("Created back the cache"); - // // create PR on local site - // vm4.invoke(WANTestBase.class, "createPersistentPartitionedRegion", - // new Object[] { testName, "ln", 1, 100, isOffHeap() }); - // vm5.invoke(WANTestBase.class, "createPersistentPartitionedRegion", - // new Object[] { testName, "ln", 1, 100, isOffHeap() }); - // vm6.invoke(WANTestBase.class, "createPersistentPartitionedRegion", - // new Object[] { testName, "ln", 1, 100, isOffHeap() }); - // vm7.invoke(WANTestBase.class, "createPersistentPartitionedRegion", - // new Object[] { testName, "ln", 1, 100, isOffHeap() }); - - // create PR on local site - AsyncInvocation inv1 = vm4.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion1 = vm4.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv2 = vm5.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion2 = vm5.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv3 = vm6.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion3 = vm6.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv4 = vm7.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion4 = vm7.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - try { - inv1.join(); - inv2.join(); - inv3.join(); - inv4.join(); - } catch (InterruptedException e) { - e.printStackTrace(); - fail(); - } + createPersistentPartitionRegion1.await(); + createPersistentPartitionRegion2.await(); + createPersistentPartitionRegion3.await(); + createPersistentPartitionRegion4.await(); logger.info("Created back the partitioned regions"); @@ -1362,7 +1336,8 @@ public void testNonPersistentPartitionedRegionWithGatewaySenderPersistenceEnable */ @Ignore("Bug50247") @Test - public void testPersistentPartitionedRegionWithGatewaySender_Restart() { + public void testPersistentPartitionedRegionWithGatewaySender_Restart() + throws InterruptedException { // create locator on local site Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1)); // create locator on remote site @@ -1452,25 +1427,19 @@ public void testPersistentPartitionedRegionWithGatewaySender_Restart() { logger.info("All the senders are now running..."); - // create PR on local site - AsyncInvocation inv1 = vm4.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion1 = vm4.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv2 = vm5.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion2 = vm5.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv3 = vm6.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion3 = vm6.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - AsyncInvocation inv4 = vm7.invokeAsync(() -> WANTestBase + AsyncInvocation createPersistentPartitionRegion4 = vm7.invokeAsync(() -> WANTestBase .createPersistentPartitionedRegion(getTestMethodName(), "ln", 1, 100, isOffHeap())); - try { - inv1.join(); - inv2.join(); - inv3.join(); - inv4.join(); - } catch (InterruptedException e) { - e.printStackTrace(); - fail(); - } + createPersistentPartitionRegion1.await(); + createPersistentPartitionRegion2.await(); + createPersistentPartitionRegion3.await(); + createPersistentPartitionRegion4.await(); logger.info("Created back the partitioned regions"); @@ -2217,7 +2186,6 @@ public void testpersistentWanGateway_restartSenderWithCleanQueuesDelayed_expectN waitForConditionInVM7.await(); - AsyncInvocation startSenderwithCleanQueuesInVM7 = vm7.invokeAsync(() -> startSenderwithCleanQueues("ln")); @@ -2241,7 +2209,6 @@ public void testpersistentWanGateway_restartSenderWithCleanQueuesDelayed_expectN vm3.invoke(() -> WANTestBase.validateRegionSize(getTestMethodName(), 0)); } - /** * When gateway senders starts to unqueue, stop gateway sender, and check that some evnts are * dispatched to receiving side, @@ -2327,8 +2294,6 @@ public void testPersistentPartitionedRegionWithGatewaySenderStartStopEventsDispa } - - /** * Enable persistence for GatewaySender. Pause the sender and do some puts in local region. Stop * GatewaySender. @@ -2429,7 +2394,6 @@ public void testpersistentWanGateway_restartSenderWithCleanQueuesDelayed_expectN vm3.invoke(() -> WANTestBase.validateRegionSize(getTestMethodName(), 0)); } - private static class BlockingDestroyRegionObserver extends DistributionMessageObserver { private final CountDownLatch startedBlocking = new CountDownLatch(1); @@ -2441,7 +2405,6 @@ public void beforeProcessMessage(ClusterDistributionManager dm, DistributionMess } } - /** * setIgnoreQueue has lots of callers by reflection *

diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPropagationLoopBackDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPropagationLoopBackDUnitTest.java index 646119c285d1..d9b4e2071609 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPropagationLoopBackDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANPropagationLoopBackDUnitTest.java @@ -365,9 +365,9 @@ public void testParallelPropagationLoopBack3SitesNtoNTopologyPutFromOneDS() { * Site-NY: dsid=1: senderId="ln": vm3, vm6 * NY site's sender's manual-start=true * - * Make sure the events are sent from LN to NY and will not be added into tmpDroppedEvents - * while normal events put from NY site can still be added to tmpDroppedEvents - * Start the sender, make sure the events in tmpDroppedEvents are sent to LN finally + * Verify that events aren't added to tmpDroppedEvents after gateway-sender is recovered in + * stopped state with manual-start=true, and that none of the events are sent to remote site + * after sender is started. */ @Test public void unstartedSenderShouldNotAddReceivedEventsIntoTmpDropped() { @@ -410,9 +410,9 @@ public void unstartedSenderShouldNotAddReceivedEventsIntoTmpDropped() { vm3.invoke(() -> WANTestBase.validateRegionSize(getTestMethodName() + "_PR", 100)); vm5.invoke(() -> WANTestBase.validateRegionSize(getTestMethodName() + "_PR", 100)); - // verify tmpDroppedEvents should be 100 at site-ln, because the sender is not started yet - vm3.invoke(() -> WANTestBase.verifyTmpDroppedEventSize("ln", 100)); - vm5.invoke(() -> WANTestBase.verifyTmpDroppedEventSize("ln", 100)); + // verify tmpDroppedEvents should be 0 at site-ln, because the sender is not started yet + vm3.invoke(() -> WANTestBase.verifyTmpDroppedEventSize("ln", 0)); + vm5.invoke(() -> WANTestBase.verifyTmpDroppedEventSize("ln", 0)); // verify site-ln has not received the events from site-ny yet vm2.invoke(() -> WANTestBase.validateRegionSize(getTestMethodName() + "_PR", 0)); diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/ClusterConfigStartStopPauseAndResumeGatewaySenderCommandDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/ClusterConfigStartStopPauseAndResumeGatewaySenderCommandDUnitTest.java new file mode 100644 index 000000000000..431905bf19da --- /dev/null +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/ClusterConfigStartStopPauseAndResumeGatewaySenderCommandDUnitTest.java @@ -0,0 +1,654 @@ +/* + * 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.wancommand; + +import static org.apache.geode.distributed.ConfigurationProperties.DISTRIBUTED_SYSTEM_ID; +import static org.apache.geode.distributed.ConfigurationProperties.GROUPS; +import static org.apache.geode.distributed.ConfigurationProperties.REMOTE_LOCATORS; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.getMember; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.startSender; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.validateGatewaySenderMXBeanProxy; +import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.verifySenderState; + +import java.io.Serializable; +import java.util.Properties; + +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import org.apache.geode.management.internal.cli.util.CommandStringBuilder; +import org.apache.geode.management.internal.i18n.CliStrings; +import org.apache.geode.test.dunit.rules.ClusterStartupRule; +import org.apache.geode.test.dunit.rules.MemberVM; +import org.apache.geode.test.junit.categories.WanTest; +import org.apache.geode.test.junit.rules.GfshCommandRule; + +@Category({WanTest.class}) +@RunWith(JUnitParamsRunner.class) +public class ClusterConfigStartStopPauseAndResumeGatewaySenderCommandDUnitTest + implements Serializable { + + @Rule + public ClusterStartupRule clusterStartupRule = new ClusterStartupRule(8); + + @Rule + public transient GfshCommandRule gfsh = new GfshCommandRule(); + + private MemberVM locatorSite1; + private MemberVM server1; + private MemberVM server2; + private MemberVM server3; + private MemberVM server4; + private MemberVM server5; + + @Before + public void before() throws Exception { + Properties props = new Properties(); + props.setProperty(DISTRIBUTED_SYSTEM_ID, "" + 1); + locatorSite1 = clusterStartupRule.startLocatorVM(1, props); + + props.setProperty(DISTRIBUTED_SYSTEM_ID, "" + 2); + props.setProperty(REMOTE_LOCATORS, "localhost[" + locatorSite1.getPort() + "]"); + clusterStartupRule.startLocatorVM(2, props); + + // Connect Gfsh to locator. + gfsh.connectAndVerify(locatorSite1); + } + + /** + * Test to validate that parallel and serial gateway-sender startup action is persisted within + * cluster configuration after "start, stop and pause gateway-sender" commands are executed. + */ + @Test + @Parameters({CliStrings.START_GATEWAYSENDER + ", true", + CliStrings.START_GATEWAYSENDER + ", false", + CliStrings.STOP_GATEWAYSENDER + ", true", CliStrings.STOP_GATEWAYSENDER + ", false", + CliStrings.PAUSE_GATEWAYSENDER + ", true", CliStrings.PAUSE_GATEWAYSENDER + ", false"}) + public void testStartStopPauseGatewaySenderWithClusterConfiguration(String cmd, + String isParallel) { + Integer locator1Port = locatorSite1.getPort(); + + // setup servers in Site #1 + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server2 = clusterStartupRule.startServerVM(4, locator1Port); + + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, isParallel) + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "" + getManualStartParameter(cmd)) + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + server2.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", + !getManualStartParameter(cmd), false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", + !getManualStartParameter(cmd), false)); + + command = new CommandStringBuilder(cmd) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK"); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + + server1.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + server2.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + + server1.stop(true); + server2.stop(true); + + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server2 = clusterStartupRule.startServerVM(4, locator1Port); + + server1.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + server2.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + } + + /** + * Test to validate that gateway-sender startup action is persisted withing cluster configuration + * after start, stop and pause gateway-sender command is executed for particular group. + */ + @Test + @Parameters({CliStrings.PAUSE_GATEWAYSENDER, CliStrings.START_GATEWAYSENDER, + CliStrings.STOP_GATEWAYSENDER}) + public void testStartStopGatewaySenderMultipleGroupClusterConfiguration(String cmd) { + int locator1Port = locatorSite1.getPort(); + + // setup servers in Site #1 + server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); + server2 = startServerWithGroups(4, "SenderGroup1", locator1Port); + server3 = startServerWithGroups(5, "SenderGroup1, SenderGroup2", locator1Port); + server4 = startServerWithGroups(6, "SenderGroup1, SenderGroup2", locator1Port); + server5 = startServerWithGroups(7, "SenderGroup3", locator1Port); + + // setup servers in Site #1 + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, "true") + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "" + getManualStartParameter(cmd)) + .addOption(CliStrings.GROUPS, "SenderGroup1,SenderGroup2,SenderGroup3") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + server2.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + server3.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + server4.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + server5.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", + !getManualStartParameter(cmd), false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", + !getManualStartParameter(cmd), false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", + !getManualStartParameter(cmd), false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server4.getVM()), "ln", + !getManualStartParameter(cmd), false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", + !getManualStartParameter(cmd), false)); + + command = new CommandStringBuilder(cmd) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1,SenderGroup2") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is updated.") + .containsOutput("Cluster configuration for group 'SenderGroup2' is updated.") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server4.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", + !getManualStartParameter(cmd), false)); + + server1.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + server2.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + server3.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + server4.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + server5.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + + server1.stop(false); + server3.stop(false); + server5.stop(false); + + server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); + server3 = startServerWithGroups(5, "SenderGroup1, SenderGroup2", locator1Port); + server5 = startServerWithGroups(7, "SenderGroup3", locator1Port); + + server1.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + server3.invoke(() -> verifySenderState("ln", shouldBeRunning(cmd), shouldBePaused(cmd))); + server5.invoke(() -> verifySenderState("ln", !getManualStartParameter(cmd), false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", + shouldBeRunning(cmd), shouldBePaused(cmd))); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", + !getManualStartParameter(cmd), false)); + } + + /** + * Test to validate that gateway-sender startup action is persisted withing cluster configuration + * after "resume gateway-sender" command is executed. + * This behavior is tested in a following way: + * + * 1. Create gateway-sender on all servers with manual-start set to true (gws will not be started + * automatically). + * 2. Start gateway-sender using "start gateway-sender command". + * 2. Pause gateway-sender using "pause gateway-sender command". + * 2. Resume gateway-senders using "resume gateway-sender command". + * 3. Stop and then start again all servers and verify that gateway-sender will reach paused + * state. + */ + @Test + public void testResumeGatewaySenderWithClusterConfigurationService() { + Integer locator1Port = locatorSite1.getPort(); + + // setup servers in Site #1 + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server2 = clusterStartupRule.startServerVM(4, locator1Port); + + // Create gateway-sender on all members + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, "true") + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "true") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + // Start gateway-sender on all members + command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK"); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server2.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, false)); + + // Pause gateway-sender on all members + command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER) + .addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, "ln") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK"); + + server1.invoke(() -> verifySenderState("ln", true, true)); + server2.invoke(() -> verifySenderState("ln", true, true)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, true)); + + // Resume gateway-sender on all members + command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER) + .addOption(CliStrings.RESUME_GATEWAYSENDER__ID, "ln") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK"); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server2.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, false)); + + server1.stop(false); + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server1.invoke(() -> verifySenderState("ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + } + + /** + * Test to validate that gateway-sender startup action is persisted withing cluster configuration + * after resume gateway-sender command is executed for particular group. + * This behavior is tested in a following way: + * + * 1. Create gateway-sender on all servers with manual-start set to true (gws will be started + * automatically). + * 2. Pause gateway-sender using "pause gateway-sender" command for wanted group of servers. + * 2. Resume gateway-sender using "resume gateway-sender" command for wanted group of servers. + * 3. Stop and then start again servers and verify that gateway-sender in wanted + * group of servers is started. + */ + @Test + public void testResumeGatewaySenderMultipleGroupClusterConfiguration() { + int locator1Port = locatorSite1.getPort(); + + server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); + server2 = startServerWithGroups(4, "SenderGroup1", locator1Port); + server3 = startServerWithGroups(5, "SenderGroup1, SenderGroup2", locator1Port); + server4 = startServerWithGroups(6, "SenderGroup2", locator1Port); + server5 = startServerWithGroups(7, "SenderGroup3", locator1Port); + + // setup servers in Site #1 + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, "true") + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "false") + .addOption(CliStrings.GROUPS, "SenderGroup1,SenderGroup2,SenderGroup3") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server2.invoke(() -> verifySenderState("ln", true, false)); + server3.invoke(() -> verifySenderState("ln", true, false)); + server4.invoke(() -> verifySenderState("ln", true, false)); + server5.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server4.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); + + + command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER) + .addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1,SenderGroup2") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is updated.") + .containsOutput("Cluster configuration for group 'SenderGroup2' is updated.") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); + + server1.invoke(() -> verifySenderState("ln", true, true)); + server2.invoke(() -> verifySenderState("ln", true, true)); + server3.invoke(() -> verifySenderState("ln", true, true)); + server4.invoke(() -> verifySenderState("ln", true, true)); + server5.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server4.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); + + + command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER) + .addOption(CliStrings.RESUME_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1,SenderGroup2") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is updated.") + .containsOutput("Cluster configuration for group 'SenderGroup2' is updated.") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server2.invoke(() -> verifySenderState("ln", true, false)); + server3.invoke(() -> verifySenderState("ln", true, false)); + server4.invoke(() -> verifySenderState("ln", true, false)); + server5.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server4.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); + + server1.stop(false); + server3.stop(false); + server5.stop(false); + + server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); + server3 = startServerWithGroups(5, "SenderGroup1, SenderGroup2", locator1Port); + server5 = startServerWithGroups(7, "SenderGroup3", locator1Port); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server3.invoke(() -> verifySenderState("ln", true, false)); + server5.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); + } + + + /** + * Test to validate that gateway-sender is persisted in cluster configuration after executing + * all actions. This behavior is tested in a following way: + * + * 1. Create gateway-sender on all servers with manual-start set to false (gws will be started + * automatically). + * 2. Pause gateway-sender using "pause gateway-sender" command. + * 3. Stop and then start again all servers and verify that gateway-sender remain paused. + * 4. Resume gateway-sender and verify it's state + * 5. Stop gateway-sender and verify it's state + * 6. Start gateway-sender and verify it's state + */ + @Test + public void testPauseGatewaySenderWithClusterConfiguration() { + Integer locator1Port = locatorSite1.getPort(); + // setup servers in Site #1 + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server2 = clusterStartupRule.startServerVM(4, locator1Port); + server3 = clusterStartupRule.startServerVM(5, locator1Port); + + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, "false") + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "false") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1.invoke(() -> startSender("ln")); + server2.invoke(() -> startSender("ln")); + server3.invoke(() -> startSender("ln")); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server2.invoke(() -> verifySenderState("ln", true, false)); + server3.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); + + command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER) + .addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, "ln") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, true)); + + server1.invoke(() -> verifySenderState("ln", true, true)); + server2.invoke(() -> verifySenderState("ln", true, true)); + server3.invoke(() -> verifySenderState("ln", true, true)); + + server1.stop(true); + server2.stop(true); + + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server2 = clusterStartupRule.startServerVM(4, locator1Port); + + server1.invoke(() -> verifySenderState("ln", true, true)); + server2.invoke(() -> verifySenderState("ln", true, true)); + server3.invoke(() -> verifySenderState("ln", true, true)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, true)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, true)); + + // Resume gateway-sender on all members + command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER) + .addOption(CliStrings.RESUME_GATEWAYSENDER__ID, "ln") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server2.invoke(() -> verifySenderState("ln", true, false)); + server3.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); + + // Stop gateway-sender on all members + command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER) + .addOption(CliStrings.STOP_GATEWAYSENDER__ID, "ln") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); + + server1.invoke(() -> verifySenderState("ln", false, false)); + server2.invoke(() -> verifySenderState("ln", false, false)); + server3.invoke(() -> verifySenderState("ln", false, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", false, false)); + + // Resume gateway-sender on all members + command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server2.invoke(() -> verifySenderState("ln", true, false)); + server3.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); + } + + private static boolean shouldBeRunning(String cmd) { + switch (cmd) { + case CliStrings.START_GATEWAYSENDER: + case CliStrings.PAUSE_GATEWAYSENDER: + case CliStrings.RESUME_GATEWAYSENDER: + return true; + default: + // CliStrings.STOP_GATEWAYSENDER + return false; + } + } + + private static boolean shouldBePaused(String cmd) { + switch (cmd) { + case CliStrings.START_GATEWAYSENDER: + case CliStrings.STOP_GATEWAYSENDER: + case CliStrings.RESUME_GATEWAYSENDER: + return false; + default: + // CliStrings.STOP_PAUSE_GATEWAYSENDER + return true; + } + } + + private static boolean getManualStartParameter(String cmd) { + switch (cmd) { + case CliStrings.START_GATEWAYSENDER: + return true; + default: + // CliStrings.PAUSE_GATEWAYSENDER + // CliStrings.STOP_GATEWAYSENDER + return false; + } + } + + private MemberVM startServerWithGroups(int index, String groups, int locPort) { + Properties props = new Properties(); + props.setProperty(GROUPS, groups); + return clusterStartupRule.startServerVM(index, props, locPort); + } +} diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/PauseGatewaySenderCommandDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/PauseGatewaySenderCommandDUnitTest.java index 5f189cf18bef..cc5f6f97ca02 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/PauseGatewaySenderCommandDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/PauseGatewaySenderCommandDUnitTest.java @@ -22,10 +22,8 @@ import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.startSender; import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.validateGatewaySenderMXBeanProxy; import static org.apache.geode.internal.cache.wan.wancommand.WANCommandUtils.verifySenderState; -import static org.assertj.core.api.Assertions.assertThat; import java.io.Serializable; -import java.util.List; import java.util.Properties; import org.junit.Before; @@ -34,17 +32,15 @@ import org.junit.experimental.categories.Category; import org.apache.geode.distributed.DistributedMember; -import org.apache.geode.management.cli.Result; -import org.apache.geode.management.internal.cli.result.CommandResult; -import org.apache.geode.management.internal.cli.result.model.TabularResultModel; +import org.apache.geode.management.internal.cli.util.CommandStringBuilder; import org.apache.geode.management.internal.i18n.CliStrings; import org.apache.geode.test.dunit.rules.ClusterStartupRule; import org.apache.geode.test.dunit.rules.MemberVM; +import org.apache.geode.test.junit.assertions.CommandResultAssert; import org.apache.geode.test.junit.categories.WanTest; import org.apache.geode.test.junit.rules.GfshCommandRule; @Category({WanTest.class}) -@SuppressWarnings("serial") public class PauseGatewaySenderCommandDUnitTest implements Serializable { @Rule @@ -54,7 +50,6 @@ public class PauseGatewaySenderCommandDUnitTest implements Serializable { public transient GfshCommandRule gfsh = new GfshCommandRule(); private MemberVM locatorSite1; - private MemberVM locatorSite2; private MemberVM server1; private MemberVM server2; private MemberVM server3; @@ -69,14 +64,14 @@ public void before() throws Exception { props.setProperty(DISTRIBUTED_SYSTEM_ID, "" + 2); props.setProperty(REMOTE_LOCATORS, "localhost[" + locatorSite1.getPort() + "]"); - locatorSite2 = clusterStartupRule.startLocatorVM(2, props); + clusterStartupRule.startLocatorVM(2, props); // Connect Gfsh to locator. gfsh.connectAndVerify(locatorSite1); } @Test - public void testPauseGatewaySender_ErrorConditions() throws Exception { + public void testPauseGatewaySender_ErrorConditions() { server1 = clusterStartupRule.startServerVM(3, locatorSite1.getPort()); DistributedMember vm1Member = getMember(server1.getVM()); String command = CliStrings.PAUSE_GATEWAYSENDER + " --" + CliStrings.PAUSE_GATEWAYSENDER__ID @@ -90,7 +85,7 @@ public void testPauseGatewaySender_ErrorConditions() throws Exception { * test to validate that the start gateway sender starts the gateway sender on a member */ @Test - public void testPauseGatewaySender_onMember() throws Exception { + public void testPauseGatewaySender_onMember() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -106,16 +101,16 @@ public void testPauseGatewaySender_onMember() throws Exception { () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); DistributedMember vm1Member = getMember(server1.getVM()); - String command = CliStrings.PAUSE_GATEWAYSENDER + " --" + CliStrings.PAUSE_GATEWAYSENDER__ID - + "=ln --" + CliStrings.MEMBER + "=" + vm1Member.getId(); - CommandResult cmdResult = gfsh.executeCommand(command); - assertThat(cmdResult).isNotNull(); + String command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER) + .addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, vm1Member.getId()) + .getCommandString(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.PAUSE_GATEWAYSENDER); - List messages = resultData.getValuesInColumn("Message"); - assertThat(messages.get(0)).contains("is paused on member"); + CommandResultAssert resultAssert = gfsh.executeAndAssertThat(command).statusIsSuccess(); + resultAssert.containsOutput( + "Configuration change is not persisted because the command is executed on specific member."); + resultAssert.hasTableSection().hasColumn("Message").asList().element(0).asString() + .contains("is paused on member"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); @@ -123,8 +118,54 @@ public void testPauseGatewaySender_onMember() throws Exception { server1.invoke(() -> verifySenderState("ln", true, true)); } + /** + * Test validates that cluster configuration is not updated when pause gateway-sender is executed + * per member. + */ @Test - public void testPauseGatewaySender() throws Exception { + public void testPauseGatewaySenderOnMemberWithClusterConfiguration() { + Integer locator1Port = locatorSite1.getPort(); + + // setup servers in Site #1 + server1 = clusterStartupRule.startServerVM(3, locator1Port); + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, "false") + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "false") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + + DistributedMember vm1Member = getMember(server1.getVM()); + command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER) + .addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, vm1Member.getId()) + .getCommandString(); + + CommandResultAssert resultAssert = gfsh.executeAndAssertThat(command).statusIsSuccess(); + resultAssert.containsOutput( + "Configuration change is not persisted because the command is executed on specific member."); + resultAssert.hasTableSection().hasColumn("Message").asList().element(0).asString() + .contains("is paused on member"); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); + server1.invoke(() -> verifySenderState("ln", true, true)); + + server1.stop(true); + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server1.invoke(() -> verifySenderState("ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + } + + @Test + public void testPauseGatewaySender() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -151,16 +192,14 @@ public void testPauseGatewaySender() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); - String command = - CliStrings.PAUSE_GATEWAYSENDER + " --" + CliStrings.PAUSE_GATEWAYSENDER__ID + "=ln"; - CommandResult cmdResult = gfsh.executeCommand(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER) + .addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, "ln") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.PAUSE_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); @@ -178,8 +217,8 @@ public void testPauseGatewaySender() throws Exception { * test to validate that the start gateway sender starts the gateway sender on a group of members */ @Test - public void testPauseGatewaySender_Group() throws Exception { - Integer locator1Port = locatorSite1.getPort(); + public void testPauseGatewaySender_Group() { + int locator1Port = locatorSite1.getPort(); // setup servers in Site #1 String groups = "SenderGroup1"; @@ -206,16 +245,15 @@ public void testPauseGatewaySender_Group() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); - String command = CliStrings.PAUSE_GATEWAYSENDER + " --" + CliStrings.PAUSE_GATEWAYSENDER__ID - + "=ln --" + CliStrings.GROUP + "=SenderGroup1"; - CommandResult cmdResult = gfsh.executeCommand(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER) + .addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.PAUSE_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); @@ -234,8 +272,8 @@ public void testPauseGatewaySender_Group() throws Exception { * to multiple groups */ @Test - public void testPauseGatewaySender_MultipleGroup() throws Exception { - Integer locator1Port = locatorSite1.getPort(); + public void testPauseGatewaySender_MultipleGroup() { + int locator1Port = locatorSite1.getPort(); // setup servers in Site #1 server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); @@ -273,16 +311,16 @@ public void testPauseGatewaySender_MultipleGroup() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); - String command = CliStrings.PAUSE_GATEWAYSENDER + " --" + CliStrings.PAUSE_GATEWAYSENDER__ID - + "=ln --" + CliStrings.GROUP + "=SenderGroup1,SenderGroup2"; - CommandResult cmdResult = gfsh.executeCommand(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER) + .addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1,SenderGroup2") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.PAUSE_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is not updated.") + .containsOutput("Cluster configuration for group 'SenderGroup2' is not updated.") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); @@ -302,7 +340,7 @@ public void testPauseGatewaySender_MultipleGroup() throws Exception { server5.invoke(() -> verifySenderState("ln", true, false)); } - private MemberVM startServerWithGroups(int index, String groups, int locPort) throws Exception { + private MemberVM startServerWithGroups(int index, String groups, int locPort) { Properties props = new Properties(); props.setProperty(GROUPS, groups); return clusterStartupRule.startServerVM(index, props, locPort); diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/ResumeGatewaySenderCommandDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/ResumeGatewaySenderCommandDUnitTest.java index 29f289a103cc..a161b1337c24 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/ResumeGatewaySenderCommandDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/ResumeGatewaySenderCommandDUnitTest.java @@ -38,14 +38,15 @@ import org.apache.geode.management.cli.Result; import org.apache.geode.management.internal.cli.result.CommandResult; import org.apache.geode.management.internal.cli.result.model.TabularResultModel; +import org.apache.geode.management.internal.cli.util.CommandStringBuilder; import org.apache.geode.management.internal.i18n.CliStrings; import org.apache.geode.test.dunit.rules.ClusterStartupRule; import org.apache.geode.test.dunit.rules.MemberVM; +import org.apache.geode.test.junit.assertions.CommandResultAssert; import org.apache.geode.test.junit.categories.WanTest; import org.apache.geode.test.junit.rules.GfshCommandRule; @Category({WanTest.class}) -@SuppressWarnings("serial") public class ResumeGatewaySenderCommandDUnitTest implements Serializable { @Rule @@ -55,7 +56,6 @@ public class ResumeGatewaySenderCommandDUnitTest implements Serializable { public transient GfshCommandRule gfsh = new GfshCommandRule(); private MemberVM locatorSite1; - private MemberVM locatorSite2; private MemberVM server1; private MemberVM server2; private MemberVM server3; @@ -70,14 +70,14 @@ public void before() throws Exception { props.setProperty(DISTRIBUTED_SYSTEM_ID, "" + 2); props.setProperty(REMOTE_LOCATORS, "localhost[" + locatorSite1.getPort() + "]"); - locatorSite2 = clusterStartupRule.startLocatorVM(2, props); + clusterStartupRule.startLocatorVM(2, props); // Connect Gfsh to locator. gfsh.connectAndVerify(locatorSite1); } @Test - public void testResumeGatewaySender_ErrorConditions() throws Exception { + public void testResumeGatewaySender_ErrorConditions() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -86,15 +86,17 @@ public void testResumeGatewaySender_ErrorConditions() throws Exception { server1.invoke(() -> createSender("ln", 2, false, 100, 400, false, false, null, true)); DistributedMember vm1Member = getMember(server1.getVM()); - String command = CliStrings.RESUME_GATEWAYSENDER + " --" + CliStrings.RESUME_GATEWAYSENDER__ID - + "=ln --" + CliStrings.MEMBER + "=" + vm1Member.getId() + " --" + CliStrings.GROUP - + "=SenderGroup1"; + String command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER) + .addOption(CliStrings.RESUME_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, vm1Member.getId()) + .addOption(CliStrings.GROUP, "SenderGroup1") + .getCommandString(); gfsh.executeAndAssertThat(command).statusIsError() .containsOutput(CliStrings.PROVIDE_EITHER_MEMBER_OR_GROUP_MESSAGE); } @Test - public void testResumeGatewaySender() throws Exception { + public void testResumeGatewaySender() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -129,16 +131,15 @@ public void testResumeGatewaySender() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, true)); - String command = - CliStrings.RESUME_GATEWAYSENDER + " --" + CliStrings.RESUME_GATEWAYSENDER__ID + "=ln"; - CommandResult cmdResult = gfsh.executeCommand(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + // Resume gateway-sender on all members + String command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER) + .addOption(CliStrings.RESUME_GATEWAYSENDER__ID, "ln") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.RESUME_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -156,7 +157,7 @@ public void testResumeGatewaySender() throws Exception { * test to validate that the start gateway sender starts the gateway sender on a member */ @Test - public void testResumeGatewaySender_onMember() throws Exception { + public void testResumeGatewaySender_onMember() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -174,18 +175,53 @@ public void testResumeGatewaySender_onMember() throws Exception { () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); DistributedMember vm1Member = getMember(server1.getVM()); - String command = CliStrings.RESUME_GATEWAYSENDER + " --" + CliStrings.RESUME_GATEWAYSENDER__ID - + "=ln --" + CliStrings.MEMBER + "=" + vm1Member.getId(); - CommandResult cmdResult = gfsh.executeCommand(command); - assertThat(cmdResult).isNotNull(); + String command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER) + .addOption(CliStrings.RESUME_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, vm1Member.getId()) + .getCommandString(); - String strCmdResult = cmdResult.toString(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + CommandResultAssert resultAssert = gfsh.executeAndAssertThat(command).statusIsSuccess(); + resultAssert.containsOutput( + "Configuration change is not persisted because the command is executed on specific member."); + resultAssert.hasTableSection().hasColumn("Message").asList().element(0).asString() + .contains("is resumed on member"); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.RESUME_GATEWAYSENDER); - List messages = resultData.getValuesInColumn("Message"); - assertThat(messages.get(0)).contains("is resumed on member"); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + + server1.invoke(() -> verifySenderState("ln", true, false)); + } + + /** + * test to validate that the start gateway sender starts the gateway sender on a member + */ + @Test + public void testResumeGatewaySenderOnMemberWithClusterConfigurationService() { + Integer locator1Port = locatorSite1.getPort(); + + // setup servers in Site #1 + server1 = clusterStartupRule.startServerVM(3, locator1Port); + + server1.invoke(() -> createSender("ln", 2, false, 100, 400, false, false, null, true)); + server1.invoke(() -> startSender("ln")); + server1.invoke(() -> verifySenderState("ln", true, false)); + server1.invoke(() -> pauseSender("ln")); + server1.invoke(() -> verifySenderState("ln", true, true)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, true)); + + DistributedMember vm1Member = getMember(server1.getVM()); + String command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER) + .addOption(CliStrings.RESUME_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, vm1Member.getId()) + .getCommandString(); + + CommandResultAssert resultAssert = gfsh.executeAndAssertThat(command).statusIsSuccess(); + resultAssert.containsOutput( + "Configuration change is not persisted because the command is executed on specific member."); + resultAssert.hasTableSection().hasColumn("Message").asList().element(0).asString() + .contains("is resumed on member"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -197,8 +233,8 @@ public void testResumeGatewaySender_onMember() throws Exception { * test to validate that the start gateway sender starts the gateway sender on a group of members */ @Test - public void testResumeGatewaySender_Group() throws Exception { - Integer locator1Port = locatorSite1.getPort(); + public void testResumeGatewaySender_Group() { + int locator1Port = locatorSite1.getPort(); // setup servers in Site #1 server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); @@ -260,8 +296,8 @@ public void testResumeGatewaySender_Group() throws Exception { * to multiple groups */ @Test - public void testResumeGatewaySender_MultipleGroup() throws Exception { - Integer locator1Port = locatorSite1.getPort(); + public void testResumeGatewaySender_MultipleGroup() { + int locator1Port = locatorSite1.getPort(); server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); server2 = startServerWithGroups(4, "SenderGroup1", locator1Port); @@ -310,16 +346,17 @@ public void testResumeGatewaySender_MultipleGroup() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, true)); - String command = CliStrings.RESUME_GATEWAYSENDER + " --" + CliStrings.RESUME_GATEWAYSENDER__ID - + "=ln --" + CliStrings.GROUP + "=SenderGroup1,SenderGroup2"; - CommandResult cmdResult = gfsh.executeCommand(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.RESUME_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); + String command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER) + .addOption(CliStrings.RESUME_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1,SenderGroup2") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is not updated.") + .containsOutput("Cluster configuration for group 'SenderGroup2' is not updated.") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -339,7 +376,7 @@ public void testResumeGatewaySender_MultipleGroup() throws Exception { server5.invoke(() -> verifySenderState("ln", true, true)); } - private MemberVM startServerWithGroups(int index, String groups, int locPort) throws Exception { + private MemberVM startServerWithGroups(int index, String groups, int locPort) { Properties props = new Properties(); props.setProperty(GROUPS, groups); return clusterStartupRule.startServerVM(index, props, locPort); diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/StartGatewaySenderCommandDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/StartGatewaySenderCommandDUnitTest.java index b41ac9f91676..a357b5d0b93d 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/StartGatewaySenderCommandDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/StartGatewaySenderCommandDUnitTest.java @@ -37,15 +37,16 @@ import org.apache.geode.management.cli.Result; import org.apache.geode.management.internal.cli.result.CommandResult; import org.apache.geode.management.internal.cli.result.model.TabularResultModel; +import org.apache.geode.management.internal.cli.util.CommandStringBuilder; import org.apache.geode.management.internal.i18n.CliStrings; import org.apache.geode.test.dunit.IgnoredException; import org.apache.geode.test.dunit.rules.ClusterStartupRule; import org.apache.geode.test.dunit.rules.MemberVM; +import org.apache.geode.test.junit.assertions.CommandResultAssert; import org.apache.geode.test.junit.categories.WanTest; import org.apache.geode.test.junit.rules.GfshCommandRule; @Category({WanTest.class}) -@SuppressWarnings("serial") public class StartGatewaySenderCommandDUnitTest implements Serializable { @Rule @@ -55,7 +56,6 @@ public class StartGatewaySenderCommandDUnitTest implements Serializable { public transient GfshCommandRule gfsh = new GfshCommandRule(); private MemberVM locatorSite1; - private MemberVM locatorSite2; private MemberVM server1; private MemberVM server2; private MemberVM server3; @@ -70,7 +70,7 @@ public void before() throws Exception { props.setProperty(DISTRIBUTED_SYSTEM_ID, "" + 2); props.setProperty(REMOTE_LOCATORS, "localhost[" + locatorSite1.getPort() + "]"); - locatorSite2 = clusterStartupRule.startLocatorVM(2, props); + clusterStartupRule.startLocatorVM(2, props); // Connect Gfsh to locator. gfsh.connectAndVerify(locatorSite1); @@ -90,15 +90,18 @@ public void testStartGatewaySender_ErrorConditions() { server1.invoke(() -> createSender("ln", 2, false, 100, 400, false, false, null, true)); DistributedMember vm1Member = getMember(server1.getVM()); - String command = CliStrings.START_GATEWAYSENDER + " --" + CliStrings.START_GATEWAYSENDER__ID - + "=ln --" + CliStrings.MEMBER + "=" + vm1Member.getId() + " --" + CliStrings.GROUP - + "=SenserGroup1"; + String command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, vm1Member.getId()) + .addOption(CliStrings.GROUP, "SenderGroup1") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsError() .containsOutput(CliStrings.PROVIDE_EITHER_MEMBER_OR_GROUP_MESSAGE); } @Test - public void testStartGatewaySender() throws Exception { + public void testStartGatewaySender() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -121,16 +124,14 @@ public void testStartGatewaySender() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", false, false)); - String command = - CliStrings.START_GATEWAYSENDER + " --" + CliStrings.START_GATEWAYSENDER__ID + "=ln"; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.START_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -144,11 +145,57 @@ public void testStartGatewaySender() throws Exception { server3.invoke(() -> verifySenderState("ln", true, false)); } + /** + * Test validates that cluster configuration is not updated when stop gateway-sender is executed + * per member. + */ + @Test + public void testStartGatewaySenderOnMemberWithClusterConfigurationService() { + Integer locator1Port = locatorSite1.getPort(); + + // setup servers in Site #1 + server1 = clusterStartupRule.startServerVM(3, locator1Port); + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, "false") + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "true") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1.invoke(() -> verifySenderState("ln", false, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); + + DistributedMember vm1Member = getMember(server1.getVM()); + command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, vm1Member.getId()) + .getCommandString(); + + CommandResultAssert resultAssert = gfsh.executeAndAssertThat(command).statusIsSuccess(); + resultAssert.containsOutput( + "Configuration change is not persisted because the command is executed on specific member."); + resultAssert.hasTableSection().hasColumn("Message").asList().element(0).asString() + .contains("is started on member"); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server1.stop(true); + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server1.invoke(() -> verifySenderState("ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); + } + /** * test to validate that the start gateway sender starts the gateway sender on a member */ @Test - public void testStartGatewaySender_onMember() throws Exception { + public void testStartGatewaySender_onMember() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -161,15 +208,16 @@ public void testStartGatewaySender_onMember() throws Exception { () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); DistributedMember vm1Member = getMember(server1.getVM()); - String command = CliStrings.START_GATEWAYSENDER + " --" + CliStrings.START_GATEWAYSENDER__ID - + "=ln --" + CliStrings.MEMBER + "=" + vm1Member.getId(); - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); + String command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, vm1Member.getId()) + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.START_GATEWAYSENDER); - List messages = resultData.getValuesInColumn("Message"); - assertThat(messages.get(0)).contains("is started on member"); + CommandResultAssert resultAssert = gfsh.executeAndAssertThat(command).statusIsSuccess(); + resultAssert.containsOutput( + "Configuration change is not persisted because the command is executed on specific member."); + resultAssert.hasTableSection().hasColumn("Message").asList().element(0).asString() + .contains("is started on member"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -181,8 +229,8 @@ public void testStartGatewaySender_onMember() throws Exception { * test to validate that the start gateway sender starts the gateway sender on a group of members */ @Test - public void testStartGatewaySender_Group() throws Exception { - Integer locator1Port = locatorSite1.getPort(); + public void testStartGatewaySender_Group() { + int locator1Port = locatorSite1.getPort(); // setup servers in Site #1 server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); @@ -204,16 +252,15 @@ public void testStartGatewaySender_Group() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", false, false)); - String command = CliStrings.START_GATEWAYSENDER + " --" + CliStrings.START_GATEWAYSENDER__ID - + "=ln --" + CliStrings.GROUP + "=SenderGroup1"; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.START_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -232,8 +279,8 @@ public void testStartGatewaySender_Group() throws Exception { * to multiple groups */ @Test - public void testStartGatewaySender_MultipleGroup() throws Exception { - Integer locator1Port = locatorSite1.getPort(); + public void testStartGatewaySender_MultipleGroup() { + int locator1Port = locatorSite1.getPort(); // setup servers in Site #1 server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); @@ -265,15 +312,15 @@ public void testStartGatewaySender_MultipleGroup() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", false, false)); - String command = CliStrings.START_GATEWAYSENDER + " --" + CliStrings.START_GATEWAYSENDER__ID - + "=ln --" + CliStrings.GROUP + "=SenderGroup1,SenderGroup2"; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1,SenderGroup2") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.START_GATEWAYSENDER); - assertThat(resultData.getValuesInColumn("Result")) + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is not updated.") + .containsOutput("Cluster configuration for group 'SenderGroup2' is not updated.") + .hasTableSection().hasColumn("Result") .containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); locatorSite1.invoke( @@ -298,8 +345,8 @@ public void testStartGatewaySender_MultipleGroup() throws Exception { * Test to validate the test scenario when one of the member ion group does not have the sender. */ @Test - public void testStartGatewaySender_Group_MissingSenderFromGroup() throws Exception { - Integer locator1Port = locatorSite1.getPort(); + public void testStartGatewaySender_Group_MissingSenderFromGroup() { + int locator1Port = locatorSite1.getPort(); // setup servers in Site #1 server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); @@ -317,15 +364,15 @@ public void testStartGatewaySender_Group_MissingSenderFromGroup() throws Excepti locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", false, false)); - String command = CliStrings.START_GATEWAYSENDER + " --" + CliStrings.START_GATEWAYSENDER__ID - + "=ln --" + CliStrings.GROUP + "=SenderGroup1"; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); + String command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.START_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("Error", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("Error", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -337,7 +384,7 @@ public void testStartGatewaySender_Group_MissingSenderFromGroup() throws Excepti } @Test - public void testStartGatewaySender_clean_queues() throws Exception { + public void testStartGatewaySender_clean_queues() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -360,17 +407,15 @@ public void testStartGatewaySender_clean_queues() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", false, false)); - String command = - CliStrings.START_GATEWAYSENDER + " --" + CliStrings.START_GATEWAYSENDER__ID + "=ln --" - + CliStrings.START_GATEWAYSENDER__CLEAN_QUEUE; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.START_GATEWAYSENDER__CLEAN_QUEUE, "true") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.START_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -385,7 +430,7 @@ public void testStartGatewaySender_clean_queues() throws Exception { } @Test - public void testStartGatewaySender_clean_queues_false() throws Exception { + public void testStartGatewaySender_clean_queues_false() { Integer locator1Port = locatorSite1.getPort(); // setup servers in Site #1 @@ -408,17 +453,15 @@ public void testStartGatewaySender_clean_queues_false() throws Exception { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", false, false)); - String command = - CliStrings.START_GATEWAYSENDER + " --" + CliStrings.START_GATEWAYSENDER__ID + "=ln --" - + CliStrings.START_GATEWAYSENDER__CLEAN_QUEUE + "=false"; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER) + .addOption(CliStrings.START_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.START_GATEWAYSENDER__CLEAN_QUEUE, "false") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.START_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); @@ -470,10 +513,8 @@ public void testStartGatewaySender_clean_queues_sender_on_one_server_allready_st .getTableSection(CliStrings.REJECT_START_GATEWAYSENDER_REASON); List status = resultData.getValuesInColumn("Result"); assertThat(status).containsExactlyInAnyOrder("Error"); - } - @Test public void testStartGatewaySender_clean_queues_on_one_member() throws Exception { Integer locator1Port = locatorSite1.getPort(); @@ -521,15 +562,13 @@ public void testStartGatewaySender_clean_queues_on_one_member() throws Exception server3.invoke(() -> verifySenderState("ln", false, false)); } - - private CommandResult executeCommandWithIgnoredExceptions(String command) throws Exception { try (IgnoredException ie = IgnoredException.addIgnoredException("Could not connect")) { return gfsh.executeCommand(command); } } - private MemberVM startServerWithGroups(int index, String groups, int locPort) throws Exception { + private MemberVM startServerWithGroups(int index, String groups, int locPort) { Properties props = new Properties(); props.setProperty(GROUPS, groups); return clusterStartupRule.startServerVM(index, props, locPort); diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/StopGatewaySenderCommandDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/StopGatewaySenderCommandDUnitTest.java index 3379551c77e4..6509a3a5c2cf 100644 --- a/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/StopGatewaySenderCommandDUnitTest.java +++ b/geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/wancommand/StopGatewaySenderCommandDUnitTest.java @@ -38,10 +38,12 @@ import org.apache.geode.management.cli.Result; import org.apache.geode.management.internal.cli.result.CommandResult; import org.apache.geode.management.internal.cli.result.model.TabularResultModel; +import org.apache.geode.management.internal.cli.util.CommandStringBuilder; import org.apache.geode.management.internal.i18n.CliStrings; import org.apache.geode.test.dunit.IgnoredException; import org.apache.geode.test.dunit.rules.ClusterStartupRule; import org.apache.geode.test.dunit.rules.MemberVM; +import org.apache.geode.test.junit.assertions.CommandResultAssert; import org.apache.geode.test.junit.categories.WanTest; import org.apache.geode.test.junit.rules.GfshCommandRule; @@ -55,7 +57,6 @@ public class StopGatewaySenderCommandDUnitTest implements Serializable { public transient GfshCommandRule gfsh = new GfshCommandRule(); private MemberVM locatorSite1; - private MemberVM locatorSite2; private MemberVM server1; private MemberVM server2; private MemberVM server3; @@ -70,7 +71,7 @@ public void before() throws Exception { props.setProperty(DISTRIBUTED_SYSTEM_ID, "" + 2); props.setProperty(REMOTE_LOCATORS, "localhost[" + locatorSite1.getPort() + "]"); - locatorSite2 = clusterStartupRule.startLocatorVM(2, props); + clusterStartupRule.startLocatorVM(2, props); // Connect Gfsh to locator. gfsh.connectAndVerify(locatorSite1); @@ -86,9 +87,13 @@ public void testStopGatewaySender_ErrorConditions() { server1.invoke(() -> createSender("ln", 2, false, 100, 400, false, false, null, true)); DistributedMember server1DM = getMember(server1.getVM()); - String command = CliStrings.STOP_GATEWAYSENDER + " --" + CliStrings.STOP_GATEWAYSENDER__ID - + "=ln --" + CliStrings.MEMBER + "=" + server1DM.getId() + " --" + CliStrings.GROUP - + "=SenderGroup1"; + + String command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER) + .addOption(CliStrings.STOP_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, server1DM.getId()) + .addOption(CliStrings.GROUP, "SenderGroup1") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsError() .containsOutput(CliStrings.PROVIDE_EITHER_MEMBER_OR_GROUP_MESSAGE); } @@ -97,10 +102,11 @@ public void testStopGatewaySender_ErrorConditions() { public void testStopGatewaySender() { Integer locator1Port = locatorSite1.getPort(); + Properties props = new Properties(); // setup servers in Site #1 (London) - server1 = clusterStartupRule.startServerVM(3, locator1Port); - server2 = clusterStartupRule.startServerVM(4, locator1Port); - server3 = clusterStartupRule.startServerVM(5, locator1Port); + server1 = clusterStartupRule.startServerVM(3, props, locator1Port); + server2 = clusterStartupRule.startServerVM(4, props, locator1Port); + server3 = clusterStartupRule.startServerVM(5, props, locator1Port); server1.invoke(() -> createSender("ln", 2, false, 100, 400, false, false, null, true)); server2.invoke(() -> createSender("ln", 2, false, 100, 400, false, false, null, true)); @@ -121,16 +127,14 @@ public void testStopGatewaySender() { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); - String command = - CliStrings.STOP_GATEWAYSENDER + " --" + CliStrings.STOP_GATEWAYSENDER__ID + "=ln"; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER) + .addOption(CliStrings.STOP_GATEWAYSENDER__ID, "ln") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.STOP_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'cluster' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); @@ -196,17 +200,17 @@ public void testStopGatewaySender_onMember() { () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); DistributedMember server1DM = getMember(server1.getVM()); - String command = CliStrings.STOP_GATEWAYSENDER + " --" + CliStrings.STOP_GATEWAYSENDER__ID - + "=ln --" + CliStrings.MEMBER + "=" + server1DM.getId(); - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER) + .addOption(CliStrings.STOP_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, server1DM.getId()) + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.STOP_GATEWAYSENDER); - List messages = resultData.getValuesInColumn("Message"); - assertThat(messages.get(0)).contains("is stopped on member"); + CommandResultAssert resultAssert = gfsh.executeAndAssertThat(command).statusIsSuccess(); + resultAssert.containsOutput( + "Configuration change is not persisted because the command is executed on specific member."); + resultAssert.hasTableSection().hasColumn("Message").asList().element(0).asString() + .contains("is stopped on member"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); @@ -214,6 +218,55 @@ public void testStopGatewaySender_onMember() { server1.invoke(() -> verifySenderState("ln", false, false)); } + /** + * Test validates that cluster configuration is not updated when stop gateway-sender is executed + * per member. + */ + @Test + public void testStopGatewaySenderOnMemberWithClusterConfigurationService() { + Integer locator1Port = locatorSite1.getPort(); + + // setup servers in Site #1 (London) + server1 = clusterStartupRule.startServerVM(3, locator1Port); + + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, "true") + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "false") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + + DistributedMember server1DM = getMember(server1.getVM()); + + command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER) + .addOption(CliStrings.STOP_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.MEMBER, server1DM.getId()) + .getCommandString(); + + CommandResultAssert resultAssert = gfsh.executeAndAssertThat(command).statusIsSuccess(); + resultAssert.containsOutput( + "Configuration change is not persisted because the command is executed on specific member."); + resultAssert.hasTableSection().hasColumn("Message").asList().element(0).asString() + .contains("is stopped on member"); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); + + server1.invoke(() -> verifySenderState("ln", false, false)); + + server1.stop(true); + server1 = clusterStartupRule.startServerVM(3, locator1Port); + server1.invoke(() -> verifySenderState("ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + } + /** * test to validate that the start gateway sender starts the gateway sender on a group of members */ @@ -245,16 +298,15 @@ public void testStopGatewaySender_Group() { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); - String command = CliStrings.STOP_GATEWAYSENDER + " --" + CliStrings.STOP_GATEWAYSENDER__ID - + "=ln --" + CliStrings.GROUP + "=SenderGroup1"; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER) + .addOption(CliStrings.STOP_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.STOP_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is not updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); @@ -312,16 +364,16 @@ public void testStopGatewaySender_MultipleGroup() { locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); - String command = CliStrings.STOP_GATEWAYSENDER + " --" + CliStrings.STOP_GATEWAYSENDER__ID - + "=ln --" + CliStrings.GROUP + "=SenderGroup1,SenderGroup2"; - CommandResult cmdResult = executeCommandWithIgnoredExceptions(command); - assertThat(cmdResult).isNotNull(); - assertThat(cmdResult.getStatus()).isSameAs(Result.Status.OK); + String command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER) + .addOption(CliStrings.STOP_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1,SenderGroup2") + .getCommandString(); - TabularResultModel resultData = cmdResult.getResultData() - .getTableSection(CliStrings.STOP_GATEWAYSENDER); - List status = resultData.getValuesInColumn("Result"); - assertThat(status).containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is not updated.") + .containsOutput("Cluster configuration for group 'SenderGroup2' is not updated.") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); locatorSite1.invoke( () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); @@ -452,6 +504,102 @@ public void testStopGatewaySender_OneSenderNotAvailable() { server2.invoke(() -> verifySenderState("ln", false, false)); } + /** + * Test to validate that gateway-sender state is persisted withing cluster configuration + * after stop gateway-sender command is executed for particular group. + * This behavior is tested in a following way: + * + * 1. Create gateway-sender on all servers with manual-start set to false (gws will be started + * automatically). + * 2. Stop gateway-senders using "stop gateway-sender" command for wanted group of servers. + * 3. Stop and then start again servers and verify that gateway-sender in wanted + * group of servers remained stopped. + */ + @Test + public void testStopGatewaySenderMultipleGroupClusterConfiguration() { + int locator1Port = locatorSite1.getPort(); + + // setup servers in Site #1 (London) + server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); + server2 = startServerWithGroups(4, "SenderGroup1", locator1Port); + server3 = startServerWithGroups(5, "SenderGroup1, SenderGroup2", locator1Port); + server4 = startServerWithGroups(6, "SenderGroup2", locator1Port); + server5 = startServerWithGroups(7, "SenderGroup3", locator1Port); + + String command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER) + .addOption(CliStrings.CREATE_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID, "2") + .addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, "false") + .addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, "false") + .addOption(CliStrings.GROUPS, "SenderGroup1,SenderGroup2,SenderGroup3") + .getCommandString(); + gfsh.executeAndAssertThat(command).statusIsSuccess(); + + server1.invoke(() -> verifySenderState("ln", true, false)); + server2.invoke(() -> verifySenderState("ln", true, false)); + server3.invoke(() -> verifySenderState("ln", true, false)); + server4.invoke(() -> verifySenderState("ln", true, false)); + server5.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server4.getVM()), "ln", true, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); + + command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER) + .addOption(CliStrings.STOP_GATEWAYSENDER__ID, "ln") + .addOption(CliStrings.GROUP, "SenderGroup1,SenderGroup2") + .getCommandString(); + + gfsh.executeAndAssertThat(command).statusIsSuccess() + .containsOutput("Cluster configuration for group 'SenderGroup1' is updated") + .containsOutput("Cluster configuration for group 'SenderGroup2' is updated") + .hasTableSection().hasColumn("Result") + .containsExactlyInAnyOrder("OK", "OK", "OK", "OK"); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server2.getVM()), "ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server4.getVM()), "ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); + + server1.invoke(() -> verifySenderState("ln", false, false)); + server2.invoke(() -> verifySenderState("ln", false, false)); + server3.invoke(() -> verifySenderState("ln", false, false)); + server4.invoke(() -> verifySenderState("ln", false, false)); + server5.invoke(() -> verifySenderState("ln", true, false)); + + server1.stop(false); + server3.stop(false); + server5.stop(false); + + server1 = startServerWithGroups(3, "SenderGroup1", locator1Port); + server3 = startServerWithGroups(5, "SenderGroup1, SenderGroup2", locator1Port); + server5 = startServerWithGroups(7, "SenderGroup3", locator1Port); + + server1.invoke(() -> verifySenderState("ln", false, false)); + server3.invoke(() -> verifySenderState("ln", false, false)); + server5.invoke(() -> verifySenderState("ln", true, false)); + + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server1.getVM()), "ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server3.getVM()), "ln", false, false)); + locatorSite1.invoke( + () -> validateGatewaySenderMXBeanProxy(getMember(server5.getVM()), "ln", true, false)); + } + private CommandResult executeCommandWithIgnoredExceptions(String command) { try (IgnoredException ie = IgnoredException.addIgnoredException("Could not connect")) { return gfsh.executeCommand(command); 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 533caf7af05e..22d90c2d5d04 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 @@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.File; import java.io.IOException; import java.util.HashSet; import java.util.Set; @@ -31,6 +32,8 @@ import org.apache.geode.cache.Cache; import org.apache.geode.cache.CacheFactory; +import org.apache.geode.cache.DiskStore; +import org.apache.geode.cache.DiskStoreFactory; import org.apache.geode.cache.Region; import org.apache.geode.cache.RegionFactory; import org.apache.geode.cache.RegionShortcut; @@ -378,6 +381,14 @@ public void test_ValidateSerialGatewaySenderAttributes() { @Test public void test_ValidateParallelGatewaySenderAttributes() { cache = new CacheFactory().set(MCAST_PORT, "0").create(); + File directory = + new File("TKSender" + "_disk_" + System.currentTimeMillis()); + directory.mkdir(); + File[] dirs1 = new File[] {directory}; + DiskStoreFactory dsf = cache.createDiskStoreFactory(); + dsf.setDiskDirs(dirs1); + DiskStore diskStore = dsf.create("FORNY"); + GatewaySenderFactory fact = cache.createGatewaySenderFactory(); fact.setParallel(true); fact.setManualStart(true); diff --git a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/GatewaySenderFactoryImpl.java b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/GatewaySenderFactoryImpl.java index 42e522c2b228..ac5e1672e409 100644 --- a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/GatewaySenderFactoryImpl.java +++ b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/GatewaySenderFactoryImpl.java @@ -25,6 +25,7 @@ import org.apache.geode.cache.wan.GatewaySender; import org.apache.geode.cache.wan.GatewaySender.OrderPolicy; import org.apache.geode.cache.wan.GatewaySenderFactory; +import org.apache.geode.cache.wan.GatewaySenderStartupAction; import org.apache.geode.cache.wan.GatewayTransportFilter; import org.apache.geode.cache.wan.internal.parallel.ParallelGatewaySenderImpl; import org.apache.geode.cache.wan.internal.serial.SerialGatewaySenderImpl; @@ -34,6 +35,7 @@ import org.apache.geode.internal.cache.wan.AsyncEventQueueConfigurationException; import org.apache.geode.internal.cache.wan.GatewaySenderAttributes; import org.apache.geode.internal.cache.wan.GatewaySenderException; +import org.apache.geode.internal.cache.wan.InternalGatewaySender; import org.apache.geode.internal.cache.wan.InternalGatewaySenderFactory; import org.apache.geode.internal.cache.xmlcache.CacheCreation; import org.apache.geode.internal.cache.xmlcache.ParallelGatewaySenderCreation; @@ -138,6 +140,13 @@ public GatewaySenderFactory setBatchSize(int batchSize) { return this; } + @Override + public GatewaySenderFactory setStartupAction( + GatewaySenderStartupAction gatewaySenderStartupAction) { + this.attrs.setStartupAction(gatewaySenderStartupAction); + return this; + } + @Override public GatewaySenderFactory setBatchTimeInterval(int batchTimeInterval) { attrs.setBatchTimeInterval(batchTimeInterval); @@ -286,12 +295,9 @@ public GatewaySender create(String id, int remoteDSId) { } if (cache instanceof GemFireCacheImpl) { sender = new ParallelGatewaySenderImpl(cache, statisticsClock, attrs); - cache.addGatewaySender(sender); - - if (!attrs.isManualStart()) { - sender.start(); - } - } else if (cache instanceof CacheCreation) { + this.cache.addGatewaySender(sender); + executeConfiguredStartupActionOnGatewaySender(sender); + } else if (this.cache instanceof CacheCreation) { sender = new ParallelGatewaySenderCreation(cache, attrs); cache.addGatewaySender(sender); } @@ -313,11 +319,9 @@ public GatewaySender create(String id, int remoteDSId) { } if (cache instanceof GemFireCacheImpl) { sender = new SerialGatewaySenderImpl(cache, statisticsClock, attrs); - cache.addGatewaySender(sender); - if (!attrs.isManualStart()) { - sender.start(); - } - } else if (cache instanceof CacheCreation) { + this.cache.addGatewaySender(sender); + executeConfiguredStartupActionOnGatewaySender(sender); + } else if (this.cache instanceof CacheCreation) { sender = new SerialGatewaySenderCreation(cache, attrs); cache.addGatewaySender(sender); } @@ -325,6 +329,31 @@ public GatewaySender create(String id, int remoteDSId) { return sender; } + /** + * This method executes configured startup-action at startup of gateway-sender. + * + * @see GatewaySenderStartupAction + */ + private void executeConfiguredStartupActionOnGatewaySender(GatewaySender sender) { + InternalGatewaySender internalGatewaySender = (InternalGatewaySender) sender; + GatewaySenderStartupAction startupAction = + internalGatewaySender.calculateStartupActionForGatewaySender(); + + switch (startupAction) { + case START: + internalGatewaySender.start(); + break; + case PAUSE: + internalGatewaySender.setStartEventProcessor(true); + internalGatewaySender.start(); + internalGatewaySender.setStartEventProcessor(false); + break; + default: + // GatewaySenderStartupAction.STOP and/or manual-start == true + internalGatewaySender.recoverInStoppedState(); + } + } + @Override public GatewaySender create(String id) { attrs.setId(id); @@ -418,5 +447,6 @@ public void configureGatewaySender(GatewaySender senderCreation) { attrs.setGroupTransactionEvents(senderCreation.mustGroupTransactionEvents()); attrs.setEnforceThreadsConnectSameReceiver( senderCreation.getEnforceThreadsConnectSameReceiver()); + attrs.setStartupAction(((InternalGatewaySender) senderCreation).getStartupAction()); } } diff --git a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/ParallelGatewaySenderImpl.java b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/ParallelGatewaySenderImpl.java index 7079f485aa85..0b99c2982f02 100644 --- a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/ParallelGatewaySenderImpl.java +++ b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/ParallelGatewaySenderImpl.java @@ -14,8 +14,12 @@ */ package org.apache.geode.cache.wan.internal.parallel; +import java.util.HashSet; +import java.util.Set; + import org.apache.logging.log4j.Logger; +import org.apache.geode.cache.Region; import org.apache.geode.cache.asyncqueue.AsyncEventListener; import org.apache.geode.cache.wan.GatewayEventFilter; import org.apache.geode.cache.wan.GatewayTransportFilter; @@ -28,6 +32,7 @@ import org.apache.geode.internal.cache.EntryEventImpl; import org.apache.geode.internal.cache.EventID; import org.apache.geode.internal.cache.InternalCache; +import org.apache.geode.internal.cache.InternalRegion; import org.apache.geode.internal.cache.PartitionedRegionHelper; import org.apache.geode.internal.cache.UpdateAttributesProcessor; import org.apache.geode.internal.cache.ha.ThreadIdentifier; @@ -55,6 +60,43 @@ public void start() { start(false); } + @Override + public void recoverInStoppedState() { + this.getLifeCycleLock().writeLock().lock(); + try { + if (eventProcessor != null) { + // Already recovered in stopped state + return; + } + + Set> targetRs = new HashSet<>(); + for (InternalRegion pr : this.getCache().getApplicationRegions()) { + if (pr.getAllGatewaySenderIds().contains(this.getId())) { + targetRs.add(pr); + } + } + if (targetRs.isEmpty()) { + // Don't do anything if data region for which gateway-sender is configured is not + // available. Gateway-sender queue will be recovered during creation of that data region. + return; + } + + eventProcessor = + new RemoteConcurrentParallelGatewaySenderEventProcessor(this, getThreadMonitorObj(), + false, true); + + InternalDistributedSystem system = + (InternalDistributedSystem) this.cache.getDistributedSystem(); + system.handleResourceEvent(ResourceEvent.GATEWAYSENDER_STOP, this); + + logger.info("Stopped {}", this); + + processTempDroppedEvents(); + } finally { + this.getLifeCycleLock().writeLock().unlock(); + } + } + @Override public void startWithCleanQueue() { start(true); @@ -83,7 +125,7 @@ private void start(boolean cleanQueues) { */ eventProcessor = new RemoteConcurrentParallelGatewaySenderEventProcessor(this, getThreadMonitorObj(), - cleanQueues); + cleanQueues, false); if (isStartEventProcessorInPausedState()) { pauseEvenIfProcessorStopped(); } diff --git a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/RemoteConcurrentParallelGatewaySenderEventProcessor.java b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/RemoteConcurrentParallelGatewaySenderEventProcessor.java index 1da39c36c281..4242f0c50203 100644 --- a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/RemoteConcurrentParallelGatewaySenderEventProcessor.java +++ b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/RemoteConcurrentParallelGatewaySenderEventProcessor.java @@ -34,20 +34,21 @@ public class RemoteConcurrentParallelGatewaySenderEventProcessor extends ConcurrentParallelGatewaySenderEventProcessor { public RemoteConcurrentParallelGatewaySenderEventProcessor(AbstractGatewaySender sender, - ThreadsMonitoring tMonitoring, boolean cleanQueues) { - super(sender, tMonitoring, cleanQueues); + ThreadsMonitoring tMonitoring, boolean cleanQueues, boolean recoverQueuesOnly) { + super(sender, tMonitoring, cleanQueues, recoverQueuesOnly); } @Override protected void createProcessors(int dispatcherThreads, Set> targetRs, - boolean cleanQueues) { + boolean cleanQueues, boolean recoverQueuesOnly) { processors = new RemoteParallelGatewaySenderEventProcessor[sender.getDispatcherThreads()]; if (logger.isDebugEnabled()) { logger.debug("Creating GatewaySenderEventProcessor"); } for (int i = 0; i < sender.getDispatcherThreads(); i++) { processors[i] = new RemoteParallelGatewaySenderEventProcessor(sender, targetRs, i, - sender.getDispatcherThreads(), getThreadMonitorObj(), cleanQueues); + sender.getDispatcherThreads(), getThreadMonitorObj(), cleanQueues, + recoverQueuesOnly); } } diff --git a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/RemoteParallelGatewaySenderEventProcessor.java b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/RemoteParallelGatewaySenderEventProcessor.java index 03166275ed8f..ae80b8d5617b 100644 --- a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/RemoteParallelGatewaySenderEventProcessor.java +++ b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/parallel/RemoteParallelGatewaySenderEventProcessor.java @@ -35,8 +35,8 @@ public class RemoteParallelGatewaySenderEventProcessor extends ParallelGatewaySe */ protected RemoteParallelGatewaySenderEventProcessor(AbstractGatewaySender sender, Set> userRegions, int id, int nDispatcher, ThreadsMonitoring tMonitoring, - boolean cleanQueues) { - super(sender, id, nDispatcher, tMonitoring, cleanQueues); + boolean cleanQueues, boolean recoverQueuesOnly) { + super(sender, id, nDispatcher, tMonitoring, cleanQueues, recoverQueuesOnly); } @Override diff --git a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/serial/RemoteConcurrentSerialGatewaySenderEventProcessor.java b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/serial/RemoteConcurrentSerialGatewaySenderEventProcessor.java index 45967538f26a..74152d54f267 100644 --- a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/serial/RemoteConcurrentSerialGatewaySenderEventProcessor.java +++ b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/serial/RemoteConcurrentSerialGatewaySenderEventProcessor.java @@ -32,7 +32,7 @@ public RemoteConcurrentSerialGatewaySenderEventProcessor(AbstractGatewaySender s } @Override - protected void initializeMessageQueue(String id, boolean cleanQueues) { + protected void initializeMessageQueue(String id, boolean cleanQueues, boolean isStopped) { for (int i = 0; i < sender.getDispatcherThreads(); i++) { processors.add(new RemoteSerialGatewaySenderEventProcessor(sender, id + "." + i, getThreadMonitorObj(), cleanQueues)); diff --git a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/serial/SerialGatewaySenderImpl.java b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/serial/SerialGatewaySenderImpl.java index f32d5b9a0507..e74c61626c80 100644 --- a/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/serial/SerialGatewaySenderImpl.java +++ b/geode-wan/src/main/java/org/apache/geode/cache/wan/internal/serial/SerialGatewaySenderImpl.java @@ -58,6 +58,9 @@ public void start() { start(false); } + @Override + public void recoverInStoppedState() {} + @Override public void startWithCleanQueue() { start(true); @@ -108,8 +111,7 @@ private void start(boolean cleanQueues) { InternalDistributedSystem system = cache.getInternalDistributedSystem(); system.handleResourceEvent(ResourceEvent.GATEWAYSENDER_START, this); - logger - .info("Started {}", this); + logger.info("Started {}", this); enqueueTempEvents(); } finally {