You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2018/11/19 18:14:36 UTC

[geode] branch develop updated: GEODE-5971: Refactor AlterAsycnEventQueueCommand to use SingleGfshCommand (#2854)

This is an automated email from the ASF dual-hosted git repository.

jensdeppe pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 333cde1  GEODE-5971: Refactor AlterAsycnEventQueueCommand to use SingleGfshCommand (#2854)
333cde1 is described below

commit 333cde15e7734bdbf8d760ff1d14d33f3765da90
Author: Aditya Anchuri <aa...@pivotal.io>
AuthorDate: Mon Nov 19 10:14:27 2018 -0800

    GEODE-5971: Refactor AlterAsycnEventQueueCommand to use SingleGfshCommand (#2854)
    
    Co-authored-by: Jens Deppe <jd...@pivotal.io>
    Co-authored-by: Kenneth Howe <kh...@pivotal.io>
---
 .../integrationTest/resources/assembly_content.txt |   1 +
 .../jdbc/internal/cli/CreateDataSourceCommand.java |   3 +-
 .../jdbc/internal/cli/CreateMappingCommand.java    |   8 +-
 .../jdbc/internal/cli/DestroyMappingCommand.java   |   4 +-
 .../internal/cli/CreateDataSourceCommandTest.java  |   4 +-
 .../internal/cli/CreateMappingCommandTest.java     |  26 +--
 .../AlterAsyncEventQueueCommandDUnitTest.java      |   2 -
 .../geode/cache/configuration/CacheConfig.java     |   3 +-
 .../InternalConfigurationPersistenceService.java   |  13 +-
 .../geode/management/cli/SingleGfshCommand.java    |  18 +--
 .../cli/UpdateAllConfigurationGroupsMarker.java    |  27 ++++
 .../cli/commands/AlterAsyncEventQueueCommand.java  | 144 +++++++++--------
 .../internal/cli/commands/ConfigurePDXCommand.java |   3 +-
 .../cli/commands/CreateAsyncEventQueueCommand.java |   3 +-
 .../cli/commands/CreateDefinedIndexesCommand.java  |   5 +-
 .../cli/commands/CreateDiskStoreCommand.java       |   3 +-
 .../cli/commands/CreateGatewayReceiverCommand.java |   3 +-
 .../cli/commands/CreateGatewaySenderCommand.java   |   3 +-
 .../internal/cli/commands/CreateIndexCommand.java  |   3 +-
 .../cli/commands/CreateJndiBindingCommand.java     |   3 +-
 .../cli/commands/DestroyDiskStoreCommand.java      |   3 +-
 .../commands/DestroyGatewayReceiverCommand.java    |   4 +-
 .../cli/commands/DestroyGatewaySenderCommand.java  |   3 +-
 .../internal/cli/commands/DestroyIndexCommand.java |   3 +-
 .../cli/commands/DestroyJndiBindingCommand.java    |   3 +-
 .../internal/cli/remote/CommandExecutor.java       |  60 +++++--
 .../commands/AlterAsyncEventQueueCommandTest.java  | 180 +++++----------------
 .../cli/commands/ConfigurePDXCommandTest.java      |  10 +-
 .../commands/CreateDefinedIndexesCommandTest.java  |   2 +-
 .../cli/commands/CreateJndiBindingCommandTest.java |   4 +-
 .../commands/DestroyJndiBindingCommandTest.java    |   4 +-
 .../internal/cli/remote/CommandExecutorTest.java   | 102 ++++++++++--
 32 files changed, 373 insertions(+), 284 deletions(-)

diff --git a/geode-assembly/src/integrationTest/resources/assembly_content.txt b/geode-assembly/src/integrationTest/resources/assembly_content.txt
index 841006e..1206bcb 100644
--- a/geode-assembly/src/integrationTest/resources/assembly_content.txt
+++ b/geode-assembly/src/integrationTest/resources/assembly_content.txt
@@ -849,6 +849,7 @@ javadoc/org/apache/geode/management/cli/GfshCommand.html
 javadoc/org/apache/geode/management/cli/Result.Status.html
 javadoc/org/apache/geode/management/cli/Result.html
 javadoc/org/apache/geode/management/cli/SingleGfshCommand.html
+javadoc/org/apache/geode/management/cli/UpdateAllConfigurationGroupsMarker.html
 javadoc/org/apache/geode/management/cli/package-frame.html
 javadoc/org/apache/geode/management/cli/package-summary.html
 javadoc/org/apache/geode/management/cli/package-tree.html
diff --git a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommand.java b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommand.java
index 345d182..c0c94b7 100644
--- a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommand.java
+++ b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommand.java
@@ -143,8 +143,9 @@ public class CreateDataSourceCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object element) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object element) {
     config.getJndiBindings().add((JndiBindingsType.JndiBinding) element);
+    return true;
   }
 
   public static class PoolProperty {
diff --git a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommand.java b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommand.java
index deca66d..8a726f7 100644
--- a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommand.java
+++ b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommand.java
@@ -181,7 +181,7 @@ public class CreateMappingCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig cacheConfig, Object element) {
+  public boolean updateConfigForGroup(String group, CacheConfig cacheConfig, Object element) {
     Object[] arguments = (Object[]) element;
     RegionMapping regionMapping = (RegionMapping) arguments[0];
     boolean synchronous = (Boolean) arguments[1];
@@ -189,14 +189,17 @@ public class CreateMappingCommand extends SingleGfshCommand {
     String queueName = createAsyncEventQueueName(regionName);
     RegionConfig regionConfig = findRegionConfig(cacheConfig, regionName);
     if (regionConfig == null) {
-      return;
+      return false;
     }
+
     RegionAttributesType attributes = getRegionAttributes(regionConfig);
     addMappingToRegion(regionMapping, regionConfig);
     if (!synchronous) {
       createAsyncQueue(cacheConfig, attributes, queueName);
     }
     alterRegion(queueName, attributes, synchronous);
+
+    return true;
   }
 
   private void alterRegion(String queueName, RegionAttributesType attributes, boolean synchronous) {
@@ -242,6 +245,7 @@ public class CreateMappingCommand extends SingleGfshCommand {
       asyncEventQueueList += queueName;
       attributes.setAsyncEventQueueIds(asyncEventQueueList);
     }
+
   }
 
   private void setCacheLoader(RegionAttributesType attributes) {
diff --git a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommand.java b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommand.java
index f85bf44..ccdace0 100644
--- a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommand.java
+++ b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommand.java
@@ -61,7 +61,7 @@ public class DestroyMappingCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig cacheConfig, Object configObject) {
+  public boolean updateConfigForGroup(String group, CacheConfig cacheConfig, Object configObject) {
     String region = (String) configObject;
     RegionMapping existingCacheElement = cacheConfig.findCustomRegionElement("/" + region,
         RegionMapping.ELEMENT_ID, RegionMapping.class);
@@ -73,6 +73,8 @@ public class DestroyMappingCommand extends SingleGfshCommand {
           .filter(regionConfig -> regionConfig.getName().equals(region))
           .forEach(
               regionConfig -> regionConfig.getCustomRegionElements().remove(existingCacheElement));
+      return true;
     }
+    return false;
   }
 }
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommandTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommandTest.java
index 6f5f340..f6fcdab 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommandTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommandTest.java
@@ -230,7 +230,7 @@ public class CreateDataSourceCommandTest {
         .containsOutput("Changes to configuration for group 'cluster' are persisted.");
 
     verify(clusterConfigService).updateCacheConfig(any(), any());
-    verify(command).updateClusterConfig(eq("cluster"), eq(cacheConfig), any());
+    verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), any());
   }
 
   @Test
@@ -304,7 +304,7 @@ public class CreateDataSourceCommandTest {
             "Tried creating jndi binding \"name\" on \"server1\"");
 
     verify(clusterConfigService).updateCacheConfig(any(), any());
-    verify(command).updateClusterConfig(eq("cluster"), eq(cacheConfig), any());
+    verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), any());
 
     ArgumentCaptor<CreateJndiBindingFunction> function =
         ArgumentCaptor.forClass(CreateJndiBindingFunction.class);
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandTest.java
index 1513f91..7fb820a 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandTest.java
@@ -328,7 +328,7 @@ public class CreateMappingCommandTest {
   public void updateClusterConfigWithNoRegionsDoesNotThrowException() {
     when(cacheConfig.getRegions()).thenReturn(Collections.emptyList());
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
   }
 
   @Test
@@ -339,7 +339,7 @@ public class CreateMappingCommandTest {
     list.add(matchingRegion);
     when(cacheConfig.getRegions()).thenReturn(list);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     assertThat(listCacheElements.size()).isEqualTo(1);
     assertThat(listCacheElements).contains(mapping);
@@ -355,7 +355,7 @@ public class CreateMappingCommandTest {
     list.add(nonMatchingRegion);
     when(cacheConfig.getRegions()).thenReturn(list);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     assertThat(listCacheElements).isEmpty();
   }
@@ -370,7 +370,7 @@ public class CreateMappingCommandTest {
     List<CacheConfig.AsyncEventQueue> queueList = new ArrayList<>();
     when(cacheConfig.getAsyncEventQueues()).thenReturn(queueList);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     assertThat(queueList.size()).isEqualTo(1);
     String queueName = CreateMappingCommand.createAsyncEventQueueName(regionName);
@@ -393,7 +393,7 @@ public class CreateMappingCommandTest {
     when(matchingRegionAttributes.getDataPolicy())
         .thenReturn(RegionAttributesDataPolicy.PARTITION);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     assertThat(queueList.get(0).isParallel()).isTrue();
   }
@@ -408,7 +408,7 @@ public class CreateMappingCommandTest {
     List<CacheConfig.AsyncEventQueue> queueList = new ArrayList<>();
     when(cacheConfig.getAsyncEventQueues()).thenReturn(queueList);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     verify(matchingRegionAttributes).setCacheLoader(any());
   }
@@ -424,7 +424,7 @@ public class CreateMappingCommandTest {
     when(cacheConfig.getAsyncEventQueues()).thenReturn(queueList);
     when(matchingRegionAttributes.getAsyncEventQueueIds()).thenReturn(null);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
     verify(matchingRegionAttributes).setAsyncEventQueueIds(argument.capture());
@@ -443,7 +443,7 @@ public class CreateMappingCommandTest {
     when(cacheConfig.getAsyncEventQueues()).thenReturn(queueList);
     when(matchingRegionAttributes.getAsyncEventQueueIds()).thenReturn("");
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
     verify(matchingRegionAttributes).setAsyncEventQueueIds(argument.capture());
@@ -462,7 +462,7 @@ public class CreateMappingCommandTest {
     when(cacheConfig.getAsyncEventQueues()).thenReturn(queueList);
     when(matchingRegionAttributes.getAsyncEventQueueIds()).thenReturn("q1,q2");
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
     verify(matchingRegionAttributes).setAsyncEventQueueIds(argument.capture());
@@ -483,7 +483,7 @@ public class CreateMappingCommandTest {
     String existingQueues = "q1," + queueName + ",q2";
     when(matchingRegionAttributes.getAsyncEventQueueIds()).thenReturn(existingQueues);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     verify(matchingRegionAttributes, never()).setAsyncEventQueueIds(any());
   }
@@ -497,7 +497,7 @@ public class CreateMappingCommandTest {
     list.add(matchingRegion);
     when(cacheConfig.getRegions()).thenReturn(list);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     verify(matchingRegionAttributes).setCacheWriter(any());
   }
@@ -514,7 +514,7 @@ public class CreateMappingCommandTest {
     when(cacheConfig.getAsyncEventQueues()).thenReturn(queueList);
     when(matchingRegionAttributes.getAsyncEventQueueIds()).thenReturn("q1,q2");
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     verify(matchingRegionAttributes, never()).setAsyncEventQueueIds(any());
   }
@@ -530,7 +530,7 @@ public class CreateMappingCommandTest {
     List<CacheConfig.AsyncEventQueue> queueList = new ArrayList<>();
     when(cacheConfig.getAsyncEventQueues()).thenReturn(queueList);
 
-    createRegionMappingCommand.updateClusterConfig(null, cacheConfig, arguments);
+    createRegionMappingCommand.updateConfigForGroup(null, cacheConfig, arguments);
 
     assertThat(queueList).isEmpty();
   }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandDUnitTest.java
index 292b70a..ba42d27 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandDUnitTest.java
@@ -49,8 +49,6 @@ public class AlterAsyncEventQueueCommandDUnitTest {
     gfsh.connectAndVerify(locator);
   }
 
-
-
   @Test
   public void testAlterAsyncEventQueue() throws Exception {
     gfsh.executeAndAssertThat("create async-event-queue --id=queue1 --group=group1 --listener="
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 285184f..c1c59f8 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
@@ -20,7 +20,6 @@ package org.apache.geode.cache.configuration;
 
 import static org.apache.geode.cache.configuration.CacheElement.findElement;
 
-import java.io.Serializable;
 import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.List;
@@ -1108,7 +1107,7 @@ public class CacheConfig {
   @XmlAccessorType(XmlAccessType.FIELD)
   @XmlType(name = "",
       propOrder = {"gatewayEventFilters", "gatewayEventSubstitutionFilter", "asyncEventListener"})
-  public static class AsyncEventQueue implements Serializable {
+  public static class AsyncEventQueue implements CacheElement {
 
     @XmlElement(name = "gateway-event-filter", namespace = "http://geode.apache.org/schema/cache")
     protected List<DeclarableType> gatewayEventFilters;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalConfigurationPersistenceService.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalConfigurationPersistenceService.java
index 8dbf19a..74aba94 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalConfigurationPersistenceService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalConfigurationPersistenceService.java
@@ -148,6 +148,14 @@ public class InternalConfigurationPersistenceService implements ConfigurationPer
     jaxbService.validateWithLocalCacheXSD();
   }
 
+  @TestingOnly
+  InternalConfigurationPersistenceService() {
+    configDirPath = null;
+    configDiskDirPath = null;
+    cache = null;
+    jaxbService = new JAXBService(CacheConfig.class);
+  }
+
   public InternalConfigurationPersistenceService(InternalCache cache, Class<?>... xsdClasses)
       throws IOException {
     this.cache = cache;
@@ -716,6 +724,10 @@ public class InternalConfigurationPersistenceService implements ConfigurationPer
     return getConfigurationRegion().getAll(keys);
   }
 
+  public Set<String> getGroups() {
+    return getConfigurationRegion().keySet();
+  }
+
   /**
    * Returns the path of Shared configuration directory
    *
@@ -987,5 +999,4 @@ public class InternalConfigurationPersistenceService implements ConfigurationPer
       unlockSharedConfiguration();
     }
   }
-
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/cli/SingleGfshCommand.java b/geode-core/src/main/java/org/apache/geode/management/cli/SingleGfshCommand.java
index edcf3e9..5cdb40d 100644
--- a/geode-core/src/main/java/org/apache/geode/management/cli/SingleGfshCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/cli/SingleGfshCommand.java
@@ -28,20 +28,18 @@ import org.apache.geode.cache.configuration.CacheConfig;
 public abstract class SingleGfshCommand extends GfshCommand {
 
   /**
-   * implement this method for updating the cluster configuration of the group
+   * Implement this method for updating the configuration of a given group
    *
-   * the implementation should update the passed in config object with appropriate changes
-   * if for any reason config can't be updated. throw a RuntimeException stating the reason.
+   * the implementation should update the passed in config object with appropriate changes if for
+   * any reason config can't be updated. throw a RuntimeException stating the reason.
    *
-   * @param group the group name of the cluster config, never null
+   * @param group the name of the group to update cluster config for
    * @param config the configuration object, never null
    * @param configObject the return value of CommandResult.getConfigObject. CommandResult is the
-   *        return
-   *        value of your command method.
-   *
-   *        it should throw some RuntimeException if update failed.
+   *        return value of your command method.
+   * @return a boolean indicating whether a change to the cluster configuration was persisted.
    */
-  public void updateClusterConfig(String group, CacheConfig config, Object configObject) {
-    // Default is a no-op
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
+    return false;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/cli/UpdateAllConfigurationGroupsMarker.java b/geode-core/src/main/java/org/apache/geode/management/cli/UpdateAllConfigurationGroupsMarker.java
new file mode 100644
index 0000000..1964528
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/cli/UpdateAllConfigurationGroupsMarker.java
@@ -0,0 +1,27 @@
+/*
+ * 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.cli;
+
+/**
+ * This interface is implemented by gfsh commands that can potentially update the configuration for
+ * all groups (including the cluster-wide group, "cluster").
+ * <p/>
+ * If the command implements UpdateAllConfigurationGroupsMarker, CommandExecutor builds the list of
+ * groups that will be updated to include all groups (including the "cluster" group). Otherwise,
+ * the list of groups are those specified with a command <code>--group</code> option, or "cluster"
+ * if there * is no <code>--group</code> option.
+ */
+public interface UpdateAllConfigurationGroupsMarker {
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommand.java
index 505200f..4fc8a0f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommand.java
@@ -22,27 +22,27 @@ import static org.apache.geode.management.internal.cli.i18n.CliStrings.IFEXISTS;
 import static org.apache.geode.management.internal.cli.i18n.CliStrings.IFEXISTS_HELP;
 
 import java.io.IOException;
+import java.util.List;
+import java.util.Set;
 
 import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.transform.TransformerException;
 
+import org.apache.commons.lang3.StringUtils;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
-import org.apache.geode.cache.Region;
+import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.cache.configuration.CacheElement;
 import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
 import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.SingleGfshCommand;
+import org.apache.geode.management.cli.UpdateAllConfigurationGroupsMarker;
 import org.apache.geode.management.internal.cli.AbstractCliAroundInterceptor;
 import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.exceptions.EntityNotFoundException;
 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.configuration.domain.Configuration;
-import org.apache.geode.management.internal.configuration.utils.XmlUtils;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission;
 
@@ -50,7 +50,8 @@ import org.apache.geode.security.ResourcePermission;
  * this command currently only updates the cluster configuration. Requires server restart to pick up
  * the changes.
  */
-public class AlterAsyncEventQueueCommand extends InternalGfshCommand {
+public class AlterAsyncEventQueueCommand extends SingleGfshCommand implements
+    UpdateAllConfigurationGroupsMarker {
 
   public static final String GROUP_STATUS_SECTION = "group-status";
   static final String COMMAND_NAME = "alter async-event-queue";
@@ -68,7 +69,6 @@ public class AlterAsyncEventQueueCommand extends InternalGfshCommand {
   static final String MAXIMUM_QUEUE_MEMORY_HELP =
       CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY__HELP;
 
-
   @CliCommand(value = COMMAND_NAME, help = COMMAND_HELP)
   @CliMetaData(
       interceptor = "org.apache.geode.management.internal.cli.commands.AlterAsyncEventQueueCommand$Interceptor")
@@ -81,82 +81,94 @@ public class AlterAsyncEventQueueCommand extends InternalGfshCommand {
       @CliOption(key = MAX_QUEUE_MEMORY, help = MAXIMUM_QUEUE_MEMORY_HELP) Integer maxQueueMemory,
       @CliOption(key = IFEXISTS, help = IFEXISTS_HELP, specifiedDefaultValue = "true",
           unspecifiedDefaultValue = "false") boolean ifExists)
-      throws IOException, SAXException, ParserConfigurationException, TransformerException {
+      throws IOException, SAXException, ParserConfigurationException, TransformerException,
+      EntityNotFoundException {
 
     // need not check if any running servers has this async-event-queue. A server with this queue id
     // may be shutdown, but we still need to update Cluster Configuration.
-    InternalConfigurationPersistenceService service =
-        (InternalConfigurationPersistenceService) getConfigurationPersistenceService();
-
-    if (service == null) {
+    if (getConfigurationPersistenceService() == null) {
       return ResultModel.createError("Cluster Configuration Service is not available. "
           + "Please connect to a locator with running Cluster Configuration Service.");
     }
 
-    boolean locked = service.lockSharedConfiguration();
-    if (!locked) {
-      return ResultModel.createCommandProcessingError("Unable to lock the cluster configuration.");
+    if (findAEQ(id) == null) {
+      String message = String.format("Can not find an async event queue with id '%s'.", id);
+      throw new EntityNotFoundException(message, ifExists);
+    }
+
+    CacheConfig.AsyncEventQueue aeqConfiguration = new CacheConfig.AsyncEventQueue();
+    aeqConfiguration.setId(id);
+
+    if (batchSize != null) {
+      aeqConfiguration.setBatchSize(batchSize + "");
+    }
+    if (batchTimeInterval != null) {
+      aeqConfiguration.setBatchTimeInterval(batchTimeInterval + "");
+    }
+    if (maxQueueMemory != null) {
+      aeqConfiguration.setMaximumQueueMemory(maxQueueMemory + "");
     }
 
     ResultModel result = new ResultModel();
-    TabularResultModel tableData = result.addTable(GROUP_STATUS_SECTION);
-    boolean xmlUpdated = false;
-    try {
-      Region<String, Configuration> configRegion = service.getConfigurationRegion();
-      for (String group : configRegion.keySet()) {
-        Configuration config = configRegion.get(group);
-        if (config.getCacheXmlContent() == null) {
-          // skip to the next group
-          continue;
+    result.addInfo().addLine("Please restart the servers to apply any changed configuration");
+    result.setConfigObject(aeqConfiguration);
+
+    return result;
+  }
+
+  CacheConfig.AsyncEventQueue findAEQ(String aeqId) {
+    CacheConfig.AsyncEventQueue queue = null;
+    InternalConfigurationPersistenceService ccService =
+        (InternalConfigurationPersistenceService) this.getConfigurationPersistenceService();
+    if (ccService == null) {
+      return null;
+    }
+
+    Set<String> groups = ccService.getGroups();
+    for (String group : groups) {
+      queue =
+          CacheElement.findElement(ccService.getCacheConfig(group).getAsyncEventQueues(), aeqId);
+      if (queue != null) {
+        return queue;
+      }
+    }
+    return queue;
+  }
+
+  @Override
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
+
+    boolean aeqConfigsHaveBeenUpdated = false;
+
+    List<CacheConfig.AsyncEventQueue> queues = config.getAsyncEventQueues();
+    if (queues.isEmpty()) {
+      return false;
+    }
+
+    CacheConfig.AsyncEventQueue aeqConfiguration =
+        ((CacheConfig.AsyncEventQueue) configObject);
+
+    String aeqId = aeqConfiguration.getId();
+
+    for (CacheConfig.AsyncEventQueue queue : queues) {
+      if (aeqId.equals(queue.getId())) {
+        if (StringUtils.isNotBlank(aeqConfiguration.getBatchSize())) {
+          queue.setBatchSize(aeqConfiguration.getBatchSize());
         }
 
-        Document document = XmlUtils.createDocumentFromXml(config.getCacheXmlContent());
-        NodeList nodeList = document.getElementsByTagName("async-event-queue");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-          Element item = (Element) nodeList.item(i);
-          String queueId = item.getAttribute("id");
-          if (!id.equals(queueId)) {
-            // skip to the next async-event-queue found in this xml
-            continue;
-          }
-          // this node is the async-event-queue with the correct id
-          if (batchSize != null) {
-            item.setAttribute(BATCH_SIZE, batchSize + "");
-          }
-          if (batchTimeInterval != null) {
-            item.setAttribute(BATCH_TIME_INTERVAL, batchTimeInterval + "");
-          }
-          if (maxQueueMemory != null) {
-            item.setAttribute(MAXIMUM_QUEUE_MEMORY, maxQueueMemory + "");
-          }
-          // each group should have only one queue with this id defined
-          tableData.accumulate("Group", group);
-          tableData.accumulate("Status", "Cluster Configuration Updated");
-          xmlUpdated = true;
-          break;
+        if (StringUtils.isNotBlank(aeqConfiguration.getBatchTimeInterval())) {
+          queue.setBatchTimeInterval(aeqConfiguration.getBatchTimeInterval());
         }
 
-        if (xmlUpdated) {
-          String newXml = XmlUtils.prettyXml(document.getFirstChild());
-          config.setCacheXmlContent(newXml);
-          configRegion.put(group, config);
+        if (StringUtils.isNotBlank(aeqConfiguration.getMaximumQueueMemory())) {
+          queue.setMaximumQueueMemory(aeqConfiguration.getMaximumQueueMemory());
         }
+        aeqConfigsHaveBeenUpdated = true;
       }
-    } finally {
-      service.unlockSharedConfiguration();
-    }
 
-    if (!xmlUpdated) {
-      String message = String.format("Can not find an async event queue with id '%s'.", id);
-      throw new EntityNotFoundException(message, ifExists);
     }
+    return aeqConfigsHaveBeenUpdated;
 
-    // some configurations are changed, print out the warning message as well.
-    tableData.setFooter(System.lineSeparator()
-        + "These changes won't take effect on the running servers. " + System.lineSeparator()
-        + "Please restart the servers in these groups for the changes to take effect.");
-
-    return result;
   }
 
   public static class Interceptor extends AbstractCliAroundInterceptor {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommand.java
index c6abb0f..f2da2a8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommand.java
@@ -109,8 +109,9 @@ public class ConfigurePDXCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object configObject) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
     config.setPdx((PdxType) configObject);
+    return true;
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommand.java
index c9fea50..51a4fae 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommand.java
@@ -139,7 +139,8 @@ public class CreateAsyncEventQueueCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object configObject) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
     config.getAsyncEventQueues().add((CacheConfig.AsyncEventQueue) configObject);
+    return true;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommand.java
index 31a589a..a93e623 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommand.java
@@ -73,10 +73,10 @@ public class CreateDefinedIndexesCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object configObject) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
     Set<RegionConfig.Index> updatedIndexes = (Set<RegionConfig.Index>) configObject;
     if (updatedIndexes == null) {
-      return;
+      return false;
     }
 
     for (RegionConfig.Index index : updatedIndexes) {
@@ -87,6 +87,7 @@ public class CreateDefinedIndexesCommand extends SingleGfshCommand {
 
       regionConfig.getIndexes().add(index);
     }
+    return true;
   }
 
   RegionConfig getValidRegionConfig(String regionPath, CacheConfig cacheConfig) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
index f4db0ad..87f84a2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
@@ -153,8 +153,9 @@ public class CreateDiskStoreCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object configObject) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
     DiskStoreType diskStoreType = (DiskStoreType) configObject;
     config.getDiskStores().add(diskStoreType);
+    return true;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateGatewayReceiverCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateGatewayReceiverCommand.java
index a02fbb7..bf53499 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateGatewayReceiverCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateGatewayReceiverCommand.java
@@ -102,8 +102,9 @@ public class CreateGatewayReceiverCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object configObject) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
     config.setGatewayReceiver((CacheConfig.GatewayReceiver) configObject);
+    return true;
   }
 
   private CacheConfig.GatewayReceiver buildConfiguration(Boolean manualStart, Integer startPort,
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateGatewaySenderCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateGatewaySenderCommand.java
index 078054a..6f8921a 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateGatewaySenderCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateGatewaySenderCommand.java
@@ -142,8 +142,9 @@ public class CreateGatewaySenderCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object configObject) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
     config.getGatewaySenders().add((CacheConfig.GatewaySender) configObject);
+    return true;
   }
 
   private boolean verifyAllCurrentVersion(Set<DistributedMember> members) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommand.java
index 7f45366..191d2ac 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommand.java
@@ -102,7 +102,7 @@ public class CreateIndexCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object element) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object element) {
     RegionConfig.Index index = (RegionConfig.Index) element;
     String regionPath = getValidRegionName(index.getFromClause(), config);
 
@@ -111,5 +111,6 @@ public class CreateIndexCommand extends SingleGfshCommand {
       throw new EntityNotFoundException("Region " + index.getFromClause() + " not found.");
     }
     regionConfig.getIndexes().add(index);
+    return true;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
index 2095195..bf89c89 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
@@ -180,8 +180,9 @@ public class CreateJndiBindingCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object element) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object element) {
     config.getJndiBindings().add((JndiBindingsType.JndiBinding) element);
+    return true;
   }
 
   public enum DATASOURCE_TYPE {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyDiskStoreCommand.java
index e266355..f93077b 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyDiskStoreCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyDiskStoreCommand.java
@@ -67,8 +67,9 @@ public class DestroyDiskStoreCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object configObject) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object configObject) {
     String diskStoreName = (String) configObject;
     CacheElement.removeElement(config.getDiskStores(), diskStoreName);
+    return true;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyGatewayReceiverCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyGatewayReceiverCommand.java
index 60ef451..85a65fa 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyGatewayReceiverCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyGatewayReceiverCommand.java
@@ -69,9 +69,11 @@ public class DestroyGatewayReceiverCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object element) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object element) {
     if (config.getGatewayReceiver() != null) {
       config.setGatewayReceiver(null);
+      return true;
     }
+    return false;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyGatewaySenderCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyGatewaySenderCommand.java
index dd77207..fd1a4fe 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyGatewaySenderCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyGatewaySenderCommand.java
@@ -71,7 +71,8 @@ public class DestroyGatewaySenderCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object id) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object id) {
     config.getGatewaySenders().removeIf((sender) -> sender.getId().equals(id));
+    return true;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyIndexCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyIndexCommand.java
index 346a426..23b8200 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyIndexCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyIndexCommand.java
@@ -91,7 +91,7 @@ public class DestroyIndexCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object element) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object element) {
     RegionConfig.Index indexFromCommand = (RegionConfig.Index) element;
     String indexName = indexFromCommand.getName();
 
@@ -117,6 +117,7 @@ public class DestroyIndexCommand extends SingleGfshCommand {
         CacheElement.removeElement(r.getIndexes(), indexName);
       }
     }
+    return true;
   }
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java
index b2769d8..9ba0b92 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java
@@ -84,7 +84,8 @@ public class DestroyJndiBindingCommand extends SingleGfshCommand {
   }
 
   @Override
-  public void updateClusterConfig(String group, CacheConfig config, Object element) {
+  public boolean updateConfigForGroup(String group, CacheConfig config, Object element) {
     CacheElement.removeElement(config.getJndiBindings(), (String) element);
+    return true;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java
index ace6a86..d695a26 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java
@@ -14,19 +14,28 @@
  */
 package org.apache.geode.management.internal.cli.remote;
 
+import static org.apache.geode.management.internal.cli.commands.AlterAsyncEventQueueCommand.GROUP_STATUS_SECTION;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.logging.log4j.Logger;
 import org.springframework.util.ReflectionUtils;
 
 import org.apache.geode.SystemFailure;
-import org.apache.geode.distributed.ConfigurationPersistenceService;
+import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.cli.SingleGfshCommand;
+import org.apache.geode.management.cli.UpdateAllConfigurationGroupsMarker;
 import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.exceptions.EntityNotFoundException;
 import org.apache.geode.management.internal.cli.exceptions.UserErrorException;
 import org.apache.geode.management.internal.cli.result.model.InfoResultModel;
 import org.apache.geode.management.internal.cli.result.model.ResultModel;
+import org.apache.geode.management.internal.cli.result.model.TabularResultModel;
 import org.apache.geode.security.NotAuthorizedException;
 
 /**
@@ -98,14 +107,18 @@ public class CommandExecutor {
     }
   }
 
+  protected Object callInvokeMethod(Object command, GfshParseResult parseResult) {
+    return ReflectionUtils.invokeMethod(parseResult.getMethod(), command,
+        parseResult.getArguments());
+  }
+
   protected Object invokeCommand(Object command, GfshParseResult parseResult) {
     // if no command instance is passed in, use the one in the parseResult
     if (command == null) {
       command = parseResult.getInstance();
     }
 
-    Object result =
-        ReflectionUtils.invokeMethod(parseResult.getMethod(), command, parseResult.getArguments());
+    Object result = callInvokeMethod(command, parseResult);
 
     if (!(command instanceof SingleGfshCommand)) {
       return result;
@@ -119,7 +132,8 @@ public class CommandExecutor {
 
     // if command result is ok, we will need to see if we need to update cluster configuration
     InfoResultModel infoResultModel = resultModel.addInfo(ResultModel.INFO_SECTION);
-    ConfigurationPersistenceService ccService = gfshCommand.getConfigurationPersistenceService();
+    InternalConfigurationPersistenceService ccService =
+        (InternalConfigurationPersistenceService) gfshCommand.getConfigurationPersistenceService();
     if (ccService == null) {
       infoResultModel.addLine(SERVICE_NOT_RUNNING_CHANGE_NOT_PERSISTED);
       return resultModel;
@@ -130,17 +144,40 @@ public class CommandExecutor {
       return resultModel;
     }
 
+    List<String> groupsToUpdate;
     String groupInput = parseResult.getParamValueAsString("group");
-    if (groupInput == null) {
-      groupInput = "cluster";
+    TabularResultModel table = null;
+
+    if (!StringUtils.isBlank(groupInput)) {
+      groupsToUpdate = Arrays.asList(groupInput.split(","));
+    } else if (gfshCommand instanceof UpdateAllConfigurationGroupsMarker) {
+      groupsToUpdate = ccService.getGroups().stream().collect(Collectors.toList());
+      table = resultModel.addTable(GROUP_STATUS_SECTION);
+      table.setColumnHeader("Group", "Status");
+    } else {
+      groupsToUpdate = Arrays.asList("cluster");
     }
-    String[] groups = groupInput.split(",");
-    for (String group : groups) {
+
+    final TabularResultModel finalTable = table;
+    for (String group : groupsToUpdate) {
       ccService.updateCacheConfig(group, cc -> {
         try {
-          gfshCommand.updateClusterConfig(group, cc, resultModel.getConfigObject());
-          infoResultModel
-              .addLine("Changes to configuration for group '" + group + "' are persisted.");
+          if (gfshCommand.updateConfigForGroup(group, cc, resultModel.getConfigObject())) {
+            if (finalTable != null) {
+              finalTable.addRow(group, "Cluster Configuration Updated");
+            } else {
+              infoResultModel
+                  .addLine("Changes to configuration for group '" + group + "' are persisted.");
+            }
+          } else {
+            if (finalTable != null) {
+              finalTable.addRow(group, "Cluster Configuration not updated");
+
+            } else {
+              infoResultModel
+                  .addLine("No changes were made to the configuration for group '" + group + "'");
+            }
+          }
         } catch (Exception e) {
           String message = "failed to update cluster config for " + group;
           logger.error(message, e);
@@ -152,6 +189,7 @@ public class CommandExecutor {
         return cc;
       });
     }
+
     return resultModel;
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
index 15e9285..6d33c93 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
@@ -15,36 +15,26 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
-import static org.apache.geode.management.internal.cli.commands.AlterAsyncEventQueueCommand.BATCH_SIZE;
-import static org.apache.geode.management.internal.cli.commands.AlterAsyncEventQueueCommand.BATCH_TIME_INTERVAL;
-import static org.apache.geode.management.internal.cli.commands.AlterAsyncEventQueueCommand.ID;
-import static org.apache.geode.management.internal.cli.commands.AlterAsyncEventQueueCommand.MAXIMUM_QUEUE_MEMORY;
-import static org.apache.geode.management.internal.cli.commands.AlterAsyncEventQueueCommand.MAX_QUEUE_MEMORY;
-import static org.assertj.core.api.Assertions.assertThat;
+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.verify;
-import static org.mockito.Mockito.when;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.stream.Collectors;
+import java.util.HashSet;
+import java.util.Set;
 
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NodeList;
 
 import org.apache.geode.cache.Region;
+import org.apache.geode.cache.configuration.CacheConfig;
 import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
-import org.apache.geode.management.internal.configuration.domain.Configuration;
-import org.apache.geode.management.internal.configuration.utils.XmlUtils;
+import org.apache.geode.internal.cache.AbstractRegion;
 import org.apache.geode.test.junit.rules.GfshParserRule;
 
-
 public class AlterAsyncEventQueueCommandTest {
 
   @ClassRule
@@ -52,27 +42,33 @@ public class AlterAsyncEventQueueCommandTest {
 
   private AlterAsyncEventQueueCommand command;
   private InternalConfigurationPersistenceService service;
-  private Region<String, Configuration> configRegion;
+  private Region configRegion;
+  private Set<String> groupSet = new HashSet<>();
 
   @Before
   public void before() throws Exception {
     command = spy(AlterAsyncEventQueueCommand.class);
-    service = mock(InternalConfigurationPersistenceService.class);
-    doReturn(service).when(command).getConfigurationPersistenceService();
-    configRegion = mock(Region.class);
-    when(service.getConfigurationRegion()).thenReturn(configRegion);
-    when(service.lockSharedConfiguration()).thenReturn(true);
-
-    when(configRegion.keySet())
-        .thenReturn(Arrays.stream("group1,group2".split(",")).collect(Collectors.toSet()));
-    Configuration configuration1 = new Configuration("group1");
-    configuration1.setCacheXmlContent(getCacheXml("queue1"));
-    when(configRegion.get("group1")).thenReturn(configuration1);
+    service = spy(InternalConfigurationPersistenceService.class);
+    configRegion = mock(AbstractRegion.class);
 
-    Configuration configuration2 = new Configuration("group2");
-    configuration2.setCacheXmlContent(getCacheXml("queue2"));
-    when(configRegion.get("group2")).thenReturn(configuration2);
+    doReturn(service).when(command).getConfigurationPersistenceService();
 
+    groupSet.add("group1");
+    groupSet.add("group2");
+    doReturn(groupSet).when(service).getGroups();
+
+    CacheConfig config = new CacheConfig();
+    CacheConfig.AsyncEventQueue aeq1 = new CacheConfig.AsyncEventQueue();
+    aeq1.setId("queue1");
+
+    config.getAsyncEventQueues().add(aeq1);
+    doReturn(config).when(service).getCacheConfig("group1");
+    doReturn(new CacheConfig()).when(service).getCacheConfig("group2");
+    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());
   }
 
   @Test
@@ -92,8 +88,6 @@ public class AlterAsyncEventQueueCommandTest {
     gfsh.executeAndAssertThat(command, "alter async-event-queue --id=test --batch-size=100")
         .statusIsError().containsOutput("Can not find an async event queue");
 
-    verify(service).lockSharedConfiguration();
-    verify(service).unlockSharedConfiguration();
   }
 
   @Test
@@ -102,8 +96,6 @@ public class AlterAsyncEventQueueCommandTest {
         "alter async-event-queue --id=test --batch-size=100 --if-exists").statusIsSuccess()
         .containsOutput("Skipping: Can not find an async event queue with id");
 
-    verify(service).lockSharedConfiguration();
-    verify(service).unlockSharedConfiguration();
   }
 
   @Test
@@ -115,129 +107,39 @@ public class AlterAsyncEventQueueCommandTest {
 
   @Test
   public void queueIdNotFoundInTheMap() throws Exception {
-    Configuration configuration = new Configuration("group");
-    configuration.setCacheXmlContent(getCacheXml("queue1", "queue2"));
-    configRegion.put("group", configuration);
-
-    gfsh.executeAndAssertThat(command, "alter async-event-queue --batch-size=100 --id=queue")
+    gfsh.executeAndAssertThat(command,
+        "alter async-event-queue --batch-size=100 --id=queue")
         .statusIsError().containsOutput("Can not find an async event queue");
 
-    verify(service).lockSharedConfiguration();
-    verify(service).unlockSharedConfiguration();
-  }
-
-  @Test
-  public void cannotLockClusterConfiguration() throws Exception {
-    when(service.lockSharedConfiguration()).thenReturn(false);
-    gfsh.executeAndAssertThat(command, "alter async-event-queue --batch-size=100 --id=queue")
-        .statusIsError().containsOutput("Unable to lock the cluster configuration");
   }
 
   @Test
   public void queueIdFoundInTheMap_updateBatchSize() throws Exception {
     gfsh.executeAndAssertThat(command, "alter async-event-queue --batch-size=100 --id=queue1")
-        .statusIsSuccess().tableHasRowCount("Group", 1)
+        .statusIsSuccess().tableHasRowCount("Group", 2)
         .tableHasRowWithValues("Group", "Status", "group1", "Cluster Configuration Updated")
-        .containsOutput("Please restart the servers");
-
-    // verify that the xml is updated
-    Element element =
-        findAsyncEventQueueElement(configRegion.get("group1").getCacheXmlContent(), 0);
-    assertThat(element.getAttribute(ID)).isEqualTo("queue1");
-    assertThat(element.getAttribute(BATCH_SIZE)).isEqualTo("100");
-    assertThat(element.getAttribute(BATCH_TIME_INTERVAL)).isEqualTo("");
-    assertThat(element.getAttribute(MAX_QUEUE_MEMORY)).isEqualTo("");
-
-    verify(service).lockSharedConfiguration();
-    verify(service).unlockSharedConfiguration();
+        .containsOutput("Please restart the servers to apply any changed configuration");
   }
 
   @Test
   public void queueIdFoundInTheMap_updateBatchTimeInterval() throws Exception {
     gfsh.executeAndAssertThat(command,
-        "alter async-event-queue --batch-time-interval=100 --id=queue1").statusIsSuccess()
-        .tableHasRowCount("Group", 1)
+        "alter async-event-queue --batch-time-interval=100 --id=queue1")
+        .statusIsSuccess()
+        .tableHasRowCount("Group", 2)
         .tableHasRowWithValues("Group", "Status", "group1", "Cluster Configuration Updated")
-        .containsOutput("Please restart the servers");
-
-    // verify that the xml is updated
-    Element element =
-        findAsyncEventQueueElement(configRegion.get("group1").getCacheXmlContent(), 0);
-    assertThat(element.getAttribute(ID)).isEqualTo("queue1");
-    assertThat(element.getAttribute(BATCH_SIZE)).isEqualTo("");
-    assertThat(element.getAttribute(BATCH_TIME_INTERVAL)).isEqualTo("100");
-    assertThat(element.getAttribute(MAX_QUEUE_MEMORY)).isEqualTo("");
+        .containsOutput("Please restart the servers to apply any changed configuration");
 
-    verify(service).lockSharedConfiguration();
-    verify(service).unlockSharedConfiguration();
+    gfsh.executeAndAssertThat(command,
+        "alter async-event-queue --batch-time-interval=100 --id=queue1").statusIsSuccess()
+        .containsOutput("Please restart the servers");
   }
 
   @Test
   public void queueIdFoundInTheMap_updateMaxMemory() throws Exception {
     gfsh.executeAndAssertThat(command, "alter async-event-queue --max-queue-memory=100 --id=queue1")
-        .statusIsSuccess().tableHasRowCount("Group", 1)
+        .statusIsSuccess().tableHasRowCount("Group", 2)
         .tableHasRowWithValues("Group", "Status", "group1", "Cluster Configuration Updated")
-        .containsOutput("Please restart the servers");
-
-    // verify that the xml is updated
-    Element element =
-        findAsyncEventQueueElement(configRegion.get("group1").getCacheXmlContent(), 0);
-    assertThat(element.getAttribute(ID)).isEqualTo("queue1");
-    assertThat(element.getAttribute(BATCH_SIZE)).isEqualTo("");
-    assertThat(element.getAttribute(BATCH_TIME_INTERVAL)).isEqualTo("");
-    assertThat(element.getAttribute(MAXIMUM_QUEUE_MEMORY)).isEqualTo("100");
-
-    verify(service).lockSharedConfiguration();
-    verify(service).unlockSharedConfiguration();
-  }
-
-  @Test
-  public void multipleQueuesInClusterConfig() throws Exception {
-    when(configRegion.keySet()).thenReturn(Collections.singleton("group"));
-    Configuration configuration = new Configuration("group");
-    configuration.setCacheXmlContent(getCacheXml("queue1", "queue2"));
-    when(configRegion.get("group")).thenReturn(configuration);
-
-    gfsh.executeAndAssertThat(command, "alter async-event-queue --batch-size=100 --id=queue1")
-        .statusIsSuccess().tableHasRowCount("Group", 1)
-        .tableHasRowWithValues("Group", "Status", "group", "Cluster Configuration Updated")
-        .containsOutput("Please restart the servers");
-
-    // verify that queue1's xml is updated
-    Element element = findAsyncEventQueueElement(configRegion.get("group").getCacheXmlContent(), 0);
-    assertThat(element.getAttribute(ID)).isEqualTo("queue1");
-    assertThat(element.getAttribute(BATCH_SIZE)).isEqualTo("100");
-    assertThat(element.getAttribute(BATCH_TIME_INTERVAL)).isEqualTo("");
-    assertThat(element.getAttribute(MAX_QUEUE_MEMORY)).isEqualTo("");
-
-    // verify that queue2's xml is untouched
-    element = findAsyncEventQueueElement(configRegion.get("group").getCacheXmlContent(), 1);
-    assertThat(element.getAttribute(ID)).isEqualTo("queue2");
-    assertThat(element.getAttribute(BATCH_SIZE)).isEqualTo("");
-    assertThat(element.getAttribute(BATCH_TIME_INTERVAL)).isEqualTo("");
-    assertThat(element.getAttribute(MAX_QUEUE_MEMORY)).isEqualTo("");
-
-    verify(service).lockSharedConfiguration();
-    verify(service).unlockSharedConfiguration();
-  }
-
-  private Element findAsyncEventQueueElement(String xml, int index) throws Exception {
-    Document document = XmlUtils.createDocumentFromXml(xml);
-    NodeList nodeList = document.getElementsByTagName("async-event-queue");
-    return (Element) nodeList.item(index);
-  }
-
-  private String getAsyncEventQueueXml(String queueId) {
-    String xml = "<async-event-queue dispatcher-threads=\"1\" id=\"" + queueId + "\">\n"
-        + "    <async-event-listener>\n"
-        + "      <class-name>org.apache.geode.internal.cache.wan.MyAsyncEventListener</class-name>\n"
-        + "    </async-event-listener>\n" + "  </async-event-queue>\n";
-    return xml;
-  }
-
-  private String getCacheXml(String... queueIds) {
-    String xml = "<cache>\n" + Arrays.stream(queueIds).map(x -> getAsyncEventQueueXml(x))
-        .collect(Collectors.joining("\n")) + "</cache>";
-    return xml;
+        .containsOutput("Please restart the servers to apply any changed configuration");
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandTest.java
index e190127..c8ca243 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandTest.java
@@ -33,8 +33,8 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
-import org.apache.geode.distributed.ConfigurationPersistenceService;
 import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.test.junit.rules.GfshParserRule;
 
@@ -43,7 +43,7 @@ public class ConfigurePDXCommandTest {
 
   private InternalCache cache;
   private ConfigurePDXCommand command;
-  private ConfigurationPersistenceService clusterConfigurationService;
+  private InternalConfigurationPersistenceService clusterConfigurationService;
 
   @Rule
   public GfshParserRule gfshParserRule = new GfshParserRule();
@@ -52,7 +52,7 @@ public class ConfigurePDXCommandTest {
   public void setUp() throws Exception {
     cache = mock(InternalCache.class);
     command = spy(ConfigurePDXCommand.class);
-    clusterConfigurationService = mock(ConfigurationPersistenceService.class);
+    clusterConfigurationService = mock(InternalConfigurationPersistenceService.class);
 
     doReturn(cache).when(command).getCache();
     doReturn(Collections.emptySet()).when(command).getAllNormalMembers();
@@ -97,7 +97,7 @@ public class ConfigurePDXCommandTest {
         .statusIsError().containsOutput("Error while processing command")
         .containsOutput("Can't create ReflectionBasedAutoSerializer.");
 
-    verify(command, times(0)).updateClusterConfig(any(), any(), any());
+    verify(command, times(0)).updateConfigForGroup(any(), any(), any());
   }
 
   @Test
@@ -109,7 +109,7 @@ public class ConfigurePDXCommandTest {
         .statusIsError().containsOutput(
             "The autoserializer cannot support both portable and non-portable classes at the same time.");
 
-    verify(command, times(0)).updateClusterConfig(any(), any(), any());
+    verify(command, times(0)).updateConfigForGroup(any(), any(), any());
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandTest.java
index 2fb692d..13e2b0f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandTest.java
@@ -114,7 +114,7 @@ public class CreateDefinedIndexesCommandTest {
     result = gfshParser.executeCommandWithInstance(command, "create defined indexes");
 
     assertThat(result.getStatus()).isEqualTo(OK);
-    verify(command, Mockito.times(0)).updateClusterConfig(any(), any(), any());
+    verify(command, Mockito.times(0)).updateConfigForGroup(any(), any(), any());
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
index ffb5c12..4bdff14 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
@@ -224,7 +224,7 @@ public class CreateJndiBindingCommandTest {
         .containsOutput("Changes to configuration for group 'cluster' are persisted.");
 
     verify(clusterConfigService).updateCacheConfig(any(), any());
-    verify(command).updateClusterConfig(eq("cluster"), eq(cacheConfig), any());
+    verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), any());
   }
 
   @Test
@@ -297,7 +297,7 @@ public class CreateJndiBindingCommandTest {
             "Tried creating jndi binding \"name\" on \"server1\"");
 
     verify(clusterConfigService).updateCacheConfig(any(), any());
-    verify(command).updateClusterConfig(eq("cluster"), eq(cacheConfig), any());
+    verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), any());
 
     ArgumentCaptor<CreateJndiBindingFunction> function =
         ArgumentCaptor.forClass(CreateJndiBindingFunction.class);
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
index 4e27189..9de3810 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
@@ -133,7 +133,7 @@ public class DestroyJndiBindingCommandTest {
         .containsOutput("Changes to configuration for group 'cluster' are persisted.");
 
     verify(ccService).updateCacheConfig(any(), any());
-    verify(command).updateClusterConfig(eq("cluster"), eq(cacheConfig), any());
+    verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), any());
   }
 
   @Test
@@ -195,7 +195,7 @@ public class DestroyJndiBindingCommandTest {
         .tableHasColumnOnlyWithValues("Message", "Jndi binding \"name\" destroyed on \"server1\"");
 
     assertThat(cacheConfig.getJndiBindings().isEmpty()).isTrue();
-    verify(command).updateClusterConfig(eq("cluster"), eq(cacheConfig), any());
+    verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), any());
 
     ArgumentCaptor<DestroyJndiBindingFunction> function =
         ArgumentCaptor.forClass(DestroyJndiBindingFunction.class);
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/CommandExecutorTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/CommandExecutorTest.java
index fb21824..a8e2567 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/CommandExecutorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/CommandExecutorTest.java
@@ -18,15 +18,30 @@ package org.apache.geode.management.internal.cli.remote;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.withSettings;
+
+import java.util.HashSet;
+import java.util.Set;
 
 import org.junit.Before;
 import org.junit.Test;
 
+import org.apache.geode.cache.Region;
+import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
+import org.apache.geode.internal.cache.AbstractRegion;
 import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.cli.SingleGfshCommand;
+import org.apache.geode.management.cli.UpdateAllConfigurationGroupsMarker;
 import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.exceptions.EntityNotFoundException;
 import org.apache.geode.management.internal.cli.exceptions.UserErrorException;
@@ -34,15 +49,31 @@ import org.apache.geode.management.internal.cli.result.model.ResultModel;
 import org.apache.geode.security.NotAuthorizedException;
 
 public class CommandExecutorTest {
-  GfshParseResult parseResult;
-  CommandExecutor executor;
-  Result result;
+  private GfshParseResult parseResult;
+  private CommandExecutor executor;
+  private ResultModel result;
+  private SingleGfshCommand testCommand;
+  private InternalConfigurationPersistenceService ccService;
+  private Region configRegion;
+
 
   @Before
   public void before() {
     parseResult = mock(GfshParseResult.class);
-    result = mock(Result.class);
+    result = new ResultModel();
     executor = spy(CommandExecutor.class);
+    testCommand = mock(SingleGfshCommand.class,
+        withSettings().extraInterfaces(UpdateAllConfigurationGroupsMarker.class));
+    ccService = spy(InternalConfigurationPersistenceService.class);
+    configRegion = mock(AbstractRegion.class);
+
+
+    doReturn(ccService).when(testCommand).getConfigurationPersistenceService();
+    doCallRealMethod().when(ccService).updateCacheConfig(any(), any());
+    doReturn(true).when(ccService).lockSharedConfiguration();
+    doNothing().when(ccService).unlockSharedConfiguration();
+    doReturn(configRegion).when(ccService).getConfigurationRegion();
+
   }
 
 
@@ -69,7 +100,6 @@ public class CommandExecutorTest {
 
   @Test
   public void anyRuntimeExceptionGetsCaught() throws Exception {
-    ;
     doThrow(new RuntimeException("my message here")).when(executor).invokeCommand(any(), any());
     Object thisResult = executor.execute(parseResult);
     assertThat(((ResultModel) thisResult).getStatus()).isEqualTo(Result.Status.ERROR);
@@ -86,7 +116,6 @@ public class CommandExecutorTest {
 
   @Test
   public void anyIllegalArgumentExceptionGetsCaught() throws Exception {
-    ;
     doThrow(new IllegalArgumentException("my message here")).when(executor).invokeCommand(any(),
         any());
     Object thisResult = executor.execute(parseResult);
@@ -96,7 +125,6 @@ public class CommandExecutorTest {
 
   @Test
   public void anyIllegalStateExceptionGetsCaught() throws Exception {
-    ;
     doThrow(new IllegalStateException("my message here")).when(executor).invokeCommand(any(),
         any());
     Object thisResult = executor.execute(parseResult);
@@ -106,7 +134,6 @@ public class CommandExecutorTest {
 
   @Test
   public void anyUserErrorExceptionGetsCaught() throws Exception {
-    ;
     doThrow(new UserErrorException("my message here")).when(executor).invokeCommand(any(), any());
     Object thisResult = executor.execute(parseResult);
     assertThat(((ResultModel) thisResult).getStatus()).isEqualTo(Result.Status.ERROR);
@@ -115,7 +142,6 @@ public class CommandExecutorTest {
 
   @Test
   public void anyEntityNotFoundException_statusOK() throws Exception {
-    ;
     doThrow(new EntityNotFoundException("my message here", true)).when(executor)
         .invokeCommand(any(), any());
     Object thisResult = executor.execute(parseResult);
@@ -125,11 +151,67 @@ public class CommandExecutorTest {
 
   @Test
   public void anyEntityNotFoundException_statusERROR() throws Exception {
-    ;
     doThrow(new EntityNotFoundException("my message here")).when(executor).invokeCommand(any(),
         any());
     Object thisResult = executor.execute(parseResult);
     assertThat(((ResultModel) thisResult).getStatus()).isEqualTo(Result.Status.ERROR);
     assertThat(thisResult.toString()).contains("my message here");
   }
+
+  @Test
+  public void invokeCommandWithUpdateAllConfigsInterface_multipleGroupOptionSpecifiedWhenSingleConfiguredGroups_CallsUpdateConfigForGroupTwice() {
+    Set<String> configuredGroups = new HashSet<>();
+    configuredGroups.add("group1");
+    when(parseResult.getParamValueAsString("group")).thenReturn("Group1,Group2");
+    doReturn(result).when(executor).callInvokeMethod(any(), any());
+    doReturn(configuredGroups).when(ccService).getGroups();
+
+    Object thisResult = executor.invokeCommand(testCommand, parseResult);
+
+    verify(testCommand, times(1)).updateConfigForGroup(eq("Group1"), any(), any());
+    verify(testCommand, times(1)).updateConfigForGroup(eq("Group2"), any(), any());
+  }
+
+  @Test
+  public void invokeCommandWithUpdateAllConfigsInterface_singleGroupOptionSpecifiedWhenMultipleConfiguredGroups_CallsUpdateConfigForGroup() {
+    Set<String> configuredGroups = new HashSet<>();
+    configuredGroups.add("group1");
+    configuredGroups.add("group2");
+    when(parseResult.getParamValueAsString("group")).thenReturn("group1");
+    doReturn(result).when(executor).callInvokeMethod(any(), any());
+    doReturn(configuredGroups).when(ccService).getGroups();
+
+    Object thisResult = executor.invokeCommand(testCommand, parseResult);
+
+    verify(testCommand, times(1)).updateConfigForGroup(eq("group1"), any(), any());
+  }
+
+  @Test
+  public void invokeCommandWithUpdateAllConfigsInterface_noGroupOptionSpecifiedWhenSingleConfiguredGroups_CallsUpdateConfigForGroup() {
+    Set<String> configuredGroups = new HashSet<>();
+    configuredGroups.add("group1");
+    when(parseResult.getParamValueAsString("group")).thenReturn(null);
+    doReturn(result).when(executor).callInvokeMethod(any(), any());
+    doReturn(configuredGroups).when(ccService).getGroups();
+
+    Object thisResult = executor.invokeCommand(testCommand, parseResult);
+
+    verify(testCommand, times(1)).updateConfigForGroup(eq("group1"), any(), any());
+  }
+
+  @Test
+  public void invokeCommandWithOutUpdateAllConfigsInterface_noGroupOptionSpecifiedWhenSingleConfiguredGroups_CallsUpdateConfigForCluster() {
+    testCommand = mock(SingleGfshCommand.class);
+    doReturn(ccService).when(testCommand).getConfigurationPersistenceService();
+
+    Set<String> configuredGroups = new HashSet<>();
+    configuredGroups.add("group1");
+    when(parseResult.getParamValueAsString("group")).thenReturn(null);
+    doReturn(result).when(executor).callInvokeMethod(any(), any());
+    doReturn(configuredGroups).when(ccService).getGroups();
+
+    Object thisResult = executor.invokeCommand(testCommand, parseResult);
+
+    verify(testCommand, times(1)).updateConfigForGroup(eq("cluster"), any(), any());
+  }
 }