You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2019/01/09 15:46:19 UTC

[geode] branch develop updated: GEODE-5971: refactor AlterRegionCommand to use RegionConfig object (#3054)

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

jinmeiliao 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 f12055a  GEODE-5971: refactor AlterRegionCommand to use RegionConfig object (#3054)
f12055a is described below

commit f12055ae3ae4b1f4731c0447af0c4cb9abdd4159
Author: jinmeiliao <ji...@pivotal.io>
AuthorDate: Wed Jan 9 07:46:05 2019 -0800

    GEODE-5971: refactor AlterRegionCommand to use RegionConfig object (#3054)
    
    * GEODE-5971: refactor AlterRegionCommand to use RegionConfig object
    
    Co-authored-by: Kenneth Howe <kh...@pivotal.io>
    Co-authored-by: Aditya Anchuri <aa...@pivotal.io>
    Co-authored-by: Peter Tran <pt...@pivotal.io>
    
    * refactor AlterRegionCommand to use ResultModel and RegionConfig
    * only alter region in the same group as the region is created on
    * when cluster configuration is enabled, if region does not exist in CC, the command would error out.
    * The command would no longer alter region not created by gfsh command
    * add more unit tests
---
 .../cli/commands/AlterRegionCommandDUnitTest.java  |  73 +++-
 .../internal/beans/RegionMBeanAttributesTest.java  |   2 +-
 .../AlterRegionCommandIntegrationTest.java         |  35 +-
 .../org/apache/geode/cache/ExpirationAction.java   |  36 +-
 .../apache/geode/cache/ExpirationAttributes.java   |   1 -
 .../geode/cache/configuration/DeclarableType.java  |  36 +-
 .../geode/cache/configuration/ObjectType.java      |  15 +
 .../geode/cache/configuration/ParameterType.java   |   2 +
 .../cache/configuration/RegionAttributesType.java  |  64 ++-
 .../geode/cache/configuration/RegionConfig.java    |   6 +-
 .../internal/cli/commands/AlterRegionCommand.java  | 261 ++++++++++---
 .../cli/functions/RegionAlterFunction.java         | 214 ++++------
 .../cli/functions/RegionCreateFunction.java        |  12 +-
 .../internal/cli/remote/CommandExecutor.java       |   4 +-
 .../sanctioned-geode-core-serializables.txt        |   1 +
 .../cache/configuration/DeclarableTypeTest.java    |  13 +
 .../configuration/RegionAttributesTypeTest.java    |  93 +++++
 .../cache/configuration/RegionConfigTest.java      |  29 ++
 .../cli/commands/AlterRegionCommandTest.java       | 435 ++++++++++++++++++++-
 .../internal/cli/domain/ClassNameTest.java         |   4 +-
 .../cli/functions/RegionAlterFunctionTest.java     | 278 +++++++++++++
 .../LuceneClusterConfigurationDUnitTest.java       |  17 +-
 22 files changed, 1398 insertions(+), 233 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
index c500911..8e13972 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
@@ -33,10 +33,17 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 
+import org.apache.geode.cache.EvictionAlgorithm;
+import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.ExpirationAction;
 import org.apache.geode.cache.ExpirationAttributes;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.cache.configuration.CacheElement;
+import org.apache.geode.cache.configuration.EnumActionDestroyOverflow;
+import org.apache.geode.cache.configuration.RegionAttributesType;
+import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.test.compiler.ClassBuilder;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
@@ -84,7 +91,7 @@ public class AlterRegionCommandDUnitTest {
   }
 
   @Test
-  public void alterRegionResetCacheListeners() throws IOException {
+  public void alterRegionResetCacheListeners() throws Exception {
     gfsh.executeAndAssertThat("create region --name=regionA --type=PARTITION").statusIsSuccess();
 
     String listenerABC =
@@ -103,11 +110,19 @@ public class AlterRegionCommandDUnitTest {
               "com.cadrdunit.RegionAlterCacheListenerC");
     }, server1, server2, server3);
 
-    // remove listener on server1
+    // alter region on a group instead of "cluster"
     gfsh.executeAndAssertThat("alter region --group=group1 --name=regionA --cache-listener=''")
-        .statusIsSuccess().tableHasRowCount("Member", 1).tableHasRowWithValues("Member", "Status",
-            "server-1", "Region \"/regionA\" altered on \"server-1\"");
-
+        .statusIsError()
+        .hasInfoSection()
+        .hasOutput().contains("/regionA does not exist in group group1");
+
+    // since this region exists on "cluster" group, we can only alter it with a "cluster" group
+    gfsh.executeAndAssertThat("alter region --name=regionA --cache-listener=''")
+        .statusIsSuccess()
+        .hasTableSection().hasRowSize(3)
+        .hasRow(0)
+        .containsExactly("server-1", "OK", "Region regionA altered");
+    // remove listener on server1
     server1.invoke(() -> {
       RegionAttributes attributes =
           ClusterStartupRule.getCache().getRegion("regionA").getAttributes();
@@ -190,12 +205,12 @@ public class AlterRegionCommandDUnitTest {
     gfsh.executeAndAssertThat(
         "alter region --name=regionA --entry-idle-time-expiration-action=invalidate")
         .statusIsError()
-        .containsOutput("ERROR: Cannot set idle timeout when statistics are disabled.");
+        .containsOutput("Cannot set idle timeout when statistics are disabled.");
 
     gfsh.executeAndAssertThat(
         "alter region --name=regionA --entry-idle-time-custom-expiry=com.cadrdunit.RegionAlterCustomExpiry")
         .statusIsError()
-        .containsOutput("ERROR: Cannot set idle timeout when statistics are disabled.");
+        .containsOutput("Cannot set idle timeout when statistics are disabled.");
   }
 
   @Test
@@ -231,6 +246,50 @@ public class AlterRegionCommandDUnitTest {
     });
   }
 
+  @Test
+  public void alterEvictionMaxOnRegionWithoutEvictionAttributesHasNoEffect() throws Exception {
+    gfsh.executeAndAssertThat("create region --name=regionA --type=REPLICATE").statusIsSuccess();
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/regionA", 3);
+
+    gfsh.executeAndAssertThat("alter region --name=regionA --eviction-max=20").statusIsSuccess();
+
+    locator.invoke(() -> {
+      CacheConfig config = ClusterStartupRule.getLocator().getConfigurationPersistenceService()
+          .getCacheConfig("cluster");
+      RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), "regionA");
+      RegionAttributesType.EvictionAttributes evictionAttributes =
+          regionConfig.getRegionAttributes().getEvictionAttributes();
+      assertThat(evictionAttributes).isNull();
+    });
+
+    server1.invoke(() -> {
+      Region region = ClusterStartupRule.getCache().getRegion("/regionA");
+      EvictionAttributes evictionAttributes = region.getAttributes().getEvictionAttributes();
+      assertThat(evictionAttributes.getAlgorithm()).isEqualTo(EvictionAlgorithm.NONE);
+    });
+  }
+
+  @Test
+  public void alterRegionWithEvictionMaxOnRegionWithEviction() throws Exception {
+    gfsh.executeAndAssertThat(
+        "create region --name=regionA --type=REPLICATE --eviction-entry-count=20 --eviction-action=local-destroy")
+        .statusIsSuccess();
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/regionA", 3);
+
+    gfsh.executeAndAssertThat("alter region --name=regionA --eviction-max=30").statusIsSuccess();
+
+    locator.invoke(() -> {
+      CacheConfig config = ClusterStartupRule.getLocator().getConfigurationPersistenceService()
+          .getCacheConfig("cluster");
+      RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), "regionA");
+      RegionAttributesType.EvictionAttributes evictionAttributes =
+          regionConfig.getRegionAttributes().getEvictionAttributes();
+      assertThat(evictionAttributes.getLruEntryCount().getMaximum()).isEqualTo("30");
+      assertThat(evictionAttributes.getLruEntryCount().getAction()).isEqualTo(
+          EnumActionDestroyOverflow.LOCAL_DESTROY);
+    });
+  }
+
   private static void deployJarFilesForRegionAlter() throws IOException {
     ClassBuilder classBuilder = new ClassBuilder();
     final File jarFile1 = new File(temporaryFolder.getRoot(), "testAlterRegion1.jar");
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/beans/RegionMBeanAttributesTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/beans/RegionMBeanAttributesTest.java
index 5a2c10e..3eb1e26 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/beans/RegionMBeanAttributesTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/beans/RegionMBeanAttributesTest.java
@@ -85,6 +85,6 @@ public class RegionMBeanAttributesTest {
     gfsh.executeAndAssertThat("alter region --name=/FOO --async-event-queue-id=").statusIsSuccess();
 
     eventQueueIds = bean.listRegionAttributes().getAsyncEventQueueIds();
-    assertThat(eventQueueIds).containsExactly("");
+    assertThat(eventQueueIds).isNotNull().isEmpty();
   }
 }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandIntegrationTest.java
index b8d040f..653004b 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandIntegrationTest.java
@@ -15,11 +15,14 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.assertj.core.api.Assertions.assertThat;
+
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.test.junit.categories.RegionsTest;
 import org.apache.geode.test.junit.rules.GfshCommandRule;
@@ -29,7 +32,8 @@ import org.apache.geode.test.junit.rules.ServerStarterRule;
 public class AlterRegionCommandIntegrationTest {
   @ClassRule
   public static ServerStarterRule server =
-      new ServerStarterRule().withJMXManager().withRegion(RegionShortcut.REPLICATE, "REPLICATED");
+      new ServerStarterRule().withJMXManager().withRegion(RegionShortcut.REPLICATE, "REPLICATED")
+          .withRegion(RegionShortcut.PARTITION, "PARTITION");
 
   @ClassRule
   public static GfshCommandRule gfsh = new GfshCommandRule();
@@ -42,7 +46,7 @@ public class AlterRegionCommandIntegrationTest {
   @Test
   public void validateGroup() throws Exception {
     gfsh.executeAndAssertThat("alter region --name=/REPLICATED --group=unknown").statusIsError()
-        .containsOutput("Group(s) \"[unknown]\" are invalid.");
+        .containsOutput("No Members Found");
   }
 
   @Test
@@ -71,4 +75,31 @@ public class AlterRegionCommandIntegrationTest {
     gfsh.executeAndAssertThat("alter region --name=/REPLICATED --eviction-max=-1").statusIsError()
         .containsOutput("Specify 0 or a positive integer value for eviction-max");
   }
+
+
+  @Test
+  public void alterRegionWithGatewaySender() throws Exception {
+    Region region = server.getCache().getRegion("/REPLICATED");
+    region.getAttributesMutator().addGatewaySenderId("1");
+    gfsh.executeAndAssertThat("alter region --name=REPLICATED --gateway-sender-id='1,2'")
+        .statusIsSuccess();
+    assertThat(region.getAttributes().getGatewaySenderIds()).containsExactly("1", "2");
+
+    gfsh.executeAndAssertThat("alter region --name=REPLICATED --gateway-sender-id=''")
+        .statusIsSuccess();
+    assertThat(region.getAttributes().getGatewaySenderIds()).isNotNull().isEmpty();
+  }
+
+  @Test
+  public void alterRegionWithAsyncEventQueue() throws Exception {
+    Region region = server.getCache().getRegion("/REPLICATED");
+    region.getAttributesMutator().addAsyncEventQueueId("1");
+    gfsh.executeAndAssertThat("alter region --name=REPLICATED --async-event-queue-id='1,2'")
+        .statusIsSuccess();
+    assertThat(region.getAttributes().getAsyncEventQueueIds()).containsExactly("1", "2");
+
+    gfsh.executeAndAssertThat("alter region --name=REPLICATED --async-event-queue-id=''")
+        .statusIsSuccess();
+    assertThat(region.getAttributes().getAsyncEventQueueIds()).isNotNull().isEmpty();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/ExpirationAction.java b/geode-core/src/main/java/org/apache/geode/cache/ExpirationAction.java
index f7167f3..7d3e67f 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/ExpirationAction.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/ExpirationAction.java
@@ -120,6 +120,11 @@ public class ExpirationAction implements Serializable {
     return this.name;
   }
 
+  /**
+   * converts to strings used in cache.xml
+   *
+   * @return strings used in cache.xml
+   */
   public String toXmlString() {
     switch (this.name) {
       case "INVALIDATE":
@@ -135,19 +140,26 @@ public class ExpirationAction implements Serializable {
     }
   }
 
-  public static ExpirationAction fromString(String s) {
-    int matchValue = -1;
-    for (int i = 0; i < VALUES.length; i++) {
-      if (VALUES[i].toString().equals(s)) {
-        matchValue = i;
-        break;
-      }
-    }
-    if (matchValue != -1) {
-      return VALUES[matchValue];
+  /**
+   * converts allowed values in cache.xml into ExpirationAction
+   *
+   * @param xmlValue the values allowed are: invalidate, destroy, local-invalidate, local-destroy
+   * @return the corresponding ExpirationAction
+   * @throws IllegalArgumentException for all other invalid strings.
+   */
+  public static ExpirationAction fromXmlString(String xmlValue) {
+    switch (xmlValue) {
+      case "invalidate":
+        return INVALIDATE;
+      case "destroy":
+        return DESTROY;
+      case "local-destroy":
+        return LOCAL_DESTROY;
+      case "local-invalidate":
+        return LOCAL_INVALIDATE;
+      default:
+        throw new IllegalArgumentException("invalid expiration action: " + xmlValue);
     }
-
-    return null;
   }
 
   // The 4 declarations below are necessary for serialization
diff --git a/geode-core/src/main/java/org/apache/geode/cache/ExpirationAttributes.java b/geode-core/src/main/java/org/apache/geode/cache/ExpirationAttributes.java
index c97da56..31026ca 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/ExpirationAttributes.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/ExpirationAttributes.java
@@ -91,7 +91,6 @@ public class ExpirationAttributes implements DataSerializable {
     }
   }
 
-
   /**
    * Returns the number of seconds before a region or value expires.
    *
diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/DeclarableType.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/DeclarableType.java
index 32c6498..47e4837 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/configuration/DeclarableType.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/DeclarableType.java
@@ -29,7 +29,12 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlType;
 
+import org.apache.commons.lang3.StringUtils;
+
 import org.apache.geode.annotations.Experimental;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Declarable;
+import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.management.internal.cli.domain.ClassName;
 
 /**
@@ -66,10 +71,14 @@ public class DeclarableType extends ClassNameType implements Serializable {
   @XmlElement(name = "parameter", namespace = "http://geode.apache.org/schema/cache")
   protected List<ParameterType> parameters;
 
+  // used to remove a Declarable through gfsh command
+  // e.g. alter region --name=regionA --cache-loader=''
+  public static DeclarableType EMPTY = new DeclarableType("");
+
   public DeclarableType() {}
 
   public DeclarableType(String className) {
-    this.className = className;
+    this(className, (Properties) null);
   }
 
   public DeclarableType(String className, String jsonProperties) {
@@ -77,7 +86,12 @@ public class DeclarableType extends ClassNameType implements Serializable {
   }
 
   public DeclarableType(String className, Properties properties) {
+    if (StringUtils.isBlank(className)) {
+      return;
+    }
+
     this.className = className;
+
     if (properties != null) {
       parameters = properties.stringPropertyNames().stream()
           .map(k -> new ParameterType(k, properties.getProperty(k))).collect(Collectors.toList());
@@ -109,7 +123,7 @@ public class DeclarableType extends ClassNameType implements Serializable {
    */
   public List<ParameterType> getParameters() {
     if (parameters == null) {
-      parameters = new ArrayList<ParameterType>();
+      parameters = new ArrayList<>();
     }
     return this.parameters;
   }
@@ -140,4 +154,22 @@ public class DeclarableType extends ClassNameType implements Serializable {
     return className + "{"
         + parameters.stream().map(Objects::toString).collect(Collectors.joining(",")) + "}";
   }
+
+  public <T> T newInstance(Cache cache) {
+    try {
+      Class<T> loadedClass = (Class<T>) ClassPathLoader.getLatest().forName(className);
+      T object = loadedClass.newInstance();
+      if (object instanceof Declarable) {
+        Declarable declarable = (Declarable) object;
+        Properties initProperties = new Properties();
+        for (ParameterType parameter : parameters) {
+          initProperties.put(parameter.getName(), parameter.newInstance(cache));
+        }
+        declarable.initialize(cache, initProperties);
+      }
+      return object;
+    } catch (Exception e) {
+      throw new RuntimeException("Error instantiating class: <" + className + ">", e);
+    }
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/ObjectType.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/ObjectType.java
index 2767815..60bbfdc 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/configuration/ObjectType.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/ObjectType.java
@@ -25,6 +25,8 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlType;
 
+import org.apache.geode.cache.Cache;
+
 /**
  * <p>
  * Java class for anonymous complex type.
@@ -45,6 +47,7 @@ import javax.xml.bind.annotation.XmlType;
  * &lt;/complexType>
  * </pre>
  *
+ * ObjectType represents either a string or an object represented by the DeclarableType
  *
  */
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -131,4 +134,16 @@ public class ObjectType implements Serializable {
     }
     return "";
   }
+
+  public Object newInstance(Cache cache) {
+    if (string != null) {
+      return string;
+    }
+
+    if (declarable != null) {
+      return declarable.newInstance(cache);
+    }
+
+    return null;
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/ParameterType.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/ParameterType.java
index 7b56f1c..e27e009 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/configuration/ParameterType.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/ParameterType.java
@@ -53,6 +53,7 @@ import org.apache.geode.annotations.Experimental;
  * &lt;/complexType>
  * </pre>
  *
+ * This represents a key/value pair, name is the key, another string or object is the value
  *
  */
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -117,4 +118,5 @@ public class ParameterType extends ObjectType {
   public String toString() {
     return name + ":" + super.toString();
   }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionAttributesType.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionAttributesType.java
index 2fa33d9..18265d0 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionAttributesType.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionAttributesType.java
@@ -20,7 +20,12 @@ package org.apache.geode.cache.configuration;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -28,7 +33,10 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlType;
 
+import org.apache.commons.lang3.StringUtils;
+
 import org.apache.geode.annotations.Experimental;
+import org.apache.geode.cache.ExpirationAction;
 
 
 /**
@@ -929,6 +937,18 @@ public class RegionAttributesType implements Serializable {
   }
 
   /**
+   * turn the comma separated ids into a set of id
+   */
+  public Set<String> getGatewaySenderIdsAsSet() {
+    if (gatewaySenderIds == null) {
+      return null;
+    }
+    return Arrays.stream(gatewaySenderIds.split(","))
+        .filter(StringUtils::isNotBlank)
+        .collect(Collectors.toSet());
+  }
+
+  /**
    * Sets the value of the gatewaySenderIds property.
    *
    * allowed object is
@@ -951,6 +971,18 @@ public class RegionAttributesType implements Serializable {
   }
 
   /**
+   * turn the comma separated id into a set of ids
+   */
+  public Set<String> getAsyncEventQueueIdsAsSet() {
+    if (asyncEventQueueIds == null) {
+      return null;
+    }
+    return Arrays.stream(asyncEventQueueIds.split(","))
+        .filter(StringUtils::isNotBlank)
+        .collect(Collectors.toSet());
+  }
+
+  /**
    * Sets the value of the asyncEventQueueIds property.
    *
    * allowed object is
@@ -1441,7 +1473,7 @@ public class RegionAttributesType implements Serializable {
    */
   @XmlAccessorType(XmlAccessType.FIELD)
   @XmlType(name = "", propOrder = {"asynchronousWrites", "synchronousWrites"})
-  public static class DiskWriteAttributes {
+  public static class DiskWriteAttributes implements Serializable {
 
     @XmlElement(name = "asynchronous-writes", namespace = "http://geode.apache.org/schema/cache")
     protected RegionAttributesType.DiskWriteAttributes.AsynchronousWrites asynchronousWrites;
@@ -1648,6 +1680,36 @@ public class RegionAttributesType implements Serializable {
         required = true)
     protected ExpirationAttributesDetail expirationAttributes = new ExpirationAttributesDetail();
 
+    public ExpirationAttributesType() {}
+
+    public ExpirationAttributesType(Integer timeout, ExpirationAction action, String expiry,
+        Properties iniProps) {
+      expirationAttributes.setTimeout(Objects.toString(timeout, null));
+      if (action != null) {
+        expirationAttributes.setAction(action.toXmlString());
+      }
+      if (expiry != null) {
+        expirationAttributes.setCustomExpiry(new DeclarableType(expiry, iniProps));
+      }
+    }
+
+    /**
+     * @return true if timeout or action is specified
+     */
+    public boolean hasTimoutOrAction() {
+      return (getTimeout() != null || getAction() != null);
+    }
+
+    /**
+     * @return true if custom expiry class is specified
+     */
+    public boolean hasCustomExpiry() {
+      return getCustomExpiry() != null;
+    }
+
+    /**
+     * @return the custom expiry declarable
+     */
     public DeclarableType getCustomExpiry() {
       return expirationAttributes.getCustomExpiry();
     }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
index 805255c..969e938 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
@@ -316,7 +316,11 @@ public class RegionConfig implements CacheElement {
    *
    */
   public void setName(String value) {
-    this.name = value;
+    if (value != null && value.startsWith("/")) {
+      this.name = value.substring(1);
+    } else {
+      this.name = value;
+    }
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommand.java
index 9c24efe..2a751ba 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommand.java
@@ -14,9 +14,12 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
+import org.apache.commons.lang3.StringUtils;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
@@ -25,26 +28,31 @@ import org.apache.geode.cache.CacheLoader;
 import org.apache.geode.cache.CacheWriter;
 import org.apache.geode.cache.CustomExpiry;
 import org.apache.geode.cache.ExpirationAction;
+import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.cache.configuration.CacheElement;
+import org.apache.geode.cache.configuration.DeclarableType;
+import org.apache.geode.cache.configuration.RegionAttributesType;
+import org.apache.geode.cache.configuration.RegionAttributesType.EvictionAttributes;
+import org.apache.geode.cache.configuration.RegionAttributesType.ExpirationAttributesType;
+import org.apache.geode.cache.configuration.RegionConfig;
+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.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.cli.SingleGfshCommand;
 import org.apache.geode.management.internal.cli.domain.ClassName;
+import org.apache.geode.management.internal.cli.exceptions.EntityNotFoundException;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.cli.functions.RegionAlterFunction;
-import org.apache.geode.management.internal.cli.functions.RegionFunctionArgs;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.configuration.domain.XmlEntity;
+import org.apache.geode.management.internal.cli.result.model.ResultModel;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
 
-public class AlterRegionCommand extends InternalGfshCommand {
+public class AlterRegionCommand extends SingleGfshCommand {
   @CliCommand(value = CliStrings.ALTER_REGION, help = CliStrings.ALTER_REGION__HELP)
   @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION)
-  public Result alterRegion(
+  public ResultModel alterRegion(
       @CliOption(key = CliStrings.ALTER_REGION__REGION, mandatory = true,
           optionContext = ConverterHint.REGION_PATH,
           help = CliStrings.ALTER_REGION__REGION__HELP) String regionPath,
@@ -88,37 +96,83 @@ public class AlterRegionCommand extends InternalGfshCommand {
       @CliOption(key = CliStrings.ALTER_REGION__GATEWAYSENDERID, specifiedDefaultValue = "",
           help = CliStrings.ALTER_REGION__GATEWAYSENDERID__HELP) String[] gatewaySenderIds,
       @CliOption(key = CliStrings.ALTER_REGION__CLONINGENABLED, specifiedDefaultValue = "true",
-          unspecifiedDefaultValue = "false",
-          help = CliStrings.ALTER_REGION__CLONINGENABLED__HELP) boolean cloningEnabled,
+          help = CliStrings.ALTER_REGION__CLONINGENABLED__HELP) Boolean cloningEnabled,
       @CliOption(key = CliStrings.ALTER_REGION__EVICTIONMAX, specifiedDefaultValue = "0",
           help = CliStrings.ALTER_REGION__EVICTIONMAX__HELP) Integer evictionMax) {
-    Result result;
-
     authorize(Resource.DATA, Operation.MANAGE, regionPath);
 
-    InternalCache cache = (InternalCache) getCache();
-
-    if (groups != null) {
-      RegionCommandsUtils.validateGroups(cache, groups);
-    }
-
-    RegionFunctionArgs regionFunctionArgs = new RegionFunctionArgs();
-    regionFunctionArgs.setRegionPath(regionPath);
-    regionFunctionArgs.setEntryExpirationIdleTime(entryExpirationIdleTime,
-        entryExpirationIdleTimeAction);
-    regionFunctionArgs.setEntryExpirationTTL(entryExpirationTTL, entryExpirationTTLAction);
-    regionFunctionArgs.setEntryIdleTimeCustomExpiry(entryIdleTimeCustomExpiry);
-    regionFunctionArgs.setEntryTTLCustomExpiry(entryTTLCustomExpiry);
-    regionFunctionArgs.setRegionExpirationIdleTime(regionExpirationIdleTime,
-        regionExpirationIdleTimeAction);
-    regionFunctionArgs.setRegionExpirationTTL(regionExpirationTTL, regionExpirationTTLAction);
-    regionFunctionArgs.setCacheListeners(cacheListeners);
-    regionFunctionArgs.setCacheLoader(cacheLoader);
-    regionFunctionArgs.setCacheWriter(cacheWriter);
-    regionFunctionArgs.setAsyncEventQueueIds(asyncEventQueueIds);
-    regionFunctionArgs.setGatewaySenderIds(gatewaySenderIds);
-    regionFunctionArgs.setCloningEnabled(cloningEnabled);
-    regionFunctionArgs.setEvictionMax(evictionMax);
+    Set<DistributedMember> targetMembers = findMembers(groups, null);
+
+    if (targetMembers.isEmpty()) {
+      return ResultModel.createError(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+    }
+
+    ConfigurationPersistenceService ccService = getConfigurationPersistenceService();
+
+    if (groups == null) {
+      groups = new String[] {"cluster"};
+    }
+
+    // Check that the specified region is in all the groups (normally we should pass in just one
+    // group for alter region command
+    if (ccService != null) {
+      for (String group : groups) {
+        CacheConfig clusterConfig = ccService.getCacheConfig(group);
+        RegionConfig regionConfig = null;
+        if (clusterConfig != null) {
+          // we always know that regionPath starts with a "/", so we need to strip it out before we
+          // pass it in to look for the regionConfig
+          regionConfig =
+              CacheElement.findElement(clusterConfig.getRegions(), regionPath.substring(1));
+        }
+
+        if (regionConfig == null) {
+          throw new EntityNotFoundException(
+              String.format("%s does not exist in group %s", regionPath, group));
+        }
+      }
+    }
+
+    RegionConfig deltaConfig = new RegionConfig();
+    deltaConfig.setName(regionPath);
+    RegionAttributesType regionAttributesType = new RegionAttributesType();
+    deltaConfig.setRegionAttributes(regionAttributesType);
+    regionAttributesType.setEntryIdleTime(getExpirationAttributes(entryExpirationIdleTime,
+        entryExpirationIdleTimeAction, entryIdleTimeCustomExpiry));
+    regionAttributesType.setEntryTimeToLive(getExpirationAttributes(entryExpirationTTL,
+        entryExpirationTTLAction, entryTTLCustomExpiry));
+    regionAttributesType.setRegionIdleTime(
+        getExpirationAttributes(regionExpirationIdleTime, regionExpirationIdleTimeAction, null));
+    regionAttributesType.setRegionTimeToLive(
+        getExpirationAttributes(regionExpirationTTL, regionExpirationTTLAction, null));
+    if (cacheLoader != null) {
+      regionAttributesType.setCacheLoader(
+          new DeclarableType(cacheLoader.getClassName(), cacheLoader.getInitProperties()));
+    }
+
+    if (cacheWriter != null) {
+      regionAttributesType.setCacheWriter(
+          new DeclarableType(cacheLoader.getClassName(), cacheLoader.getInitProperties()));
+    }
+
+    if (cacheListeners != null) {
+      regionAttributesType.getCacheListeners().addAll(
+          Arrays.stream(cacheListeners)
+              .map(l -> new DeclarableType(l.getClassName(), l.getInitProperties()))
+              .collect(Collectors.toList()));
+    }
+
+    if (gatewaySenderIds != null) {
+      regionAttributesType.setGatewaySenderIds(StringUtils.join(gatewaySenderIds, ","));
+    }
+
+    if (asyncEventQueueIds != null) {
+      regionAttributesType.setAsyncEventQueueIds(StringUtils.join(asyncEventQueueIds, ","));
+    }
+
+    if (cloningEnabled != null) {
+      regionAttributesType.setCloningEnabled(cloningEnabled);
+    }
 
     if (evictionMax != null && evictionMax < 0) {
       throw new IllegalArgumentException(CliStrings.format(
@@ -126,23 +180,136 @@ public class AlterRegionCommand extends InternalGfshCommand {
           evictionMax));
     }
 
-    Set<DistributedMember> targetMembers = findMembers(groups, null);
-
-    if (targetMembers.isEmpty()) {
-      return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+    if (evictionMax != null) {
+      EvictionAttributes evictionAttributes =
+          new EvictionAttributes();
+      EvictionAttributes.LruEntryCount lruEntryCount =
+          new EvictionAttributes.LruEntryCount();
+      lruEntryCount.setMaximum(evictionMax.toString());
+      evictionAttributes.setLruEntryCount(lruEntryCount);
+      regionAttributesType.setEvictionAttributes(evictionAttributes);
     }
 
     List<CliFunctionResult> regionAlterResults =
-        executeAndGetFunctionResult(new RegionAlterFunction(), regionFunctionArgs, targetMembers);
-    result = ResultBuilder.buildResult(regionAlterResults);
+        executeAndGetFunctionResult(new RegionAlterFunction(), deltaConfig, targetMembers);
+    ResultModel result = ResultModel.createMemberStatusResult(regionAlterResults);
+    result.setConfigObject(deltaConfig);
+    return result;
+  }
+
+  public boolean updateConfigForGroup(String group, CacheConfig cacheConfig, Object configObject) {
+    RegionConfig deltaConfig = (RegionConfig) configObject;
+    RegionConfig existingConfig = CacheElement.findElement(cacheConfig.getRegions(),
+        deltaConfig.getId());
+
+    RegionAttributesType deltaAttributes = deltaConfig.getRegionAttributes();
+    RegionAttributesType existingAttributes = existingConfig.getRegionAttributes();
 
-    XmlEntity xmlEntity = findXmlEntity(regionAlterResults);
-    if (xmlEntity != null) {
-      persistClusterConfiguration(result,
-          () -> ((InternalConfigurationPersistenceService) getConfigurationPersistenceService())
-              .addXmlEntity(xmlEntity, groups));
+    existingAttributes.setEntryIdleTime(
+        combine(existingAttributes.getEntryIdleTime(), deltaAttributes.getEntryIdleTime()));
+    existingAttributes.setEntryTimeToLive(
+        combine(existingAttributes.getEntryTimeToLive(), deltaAttributes.getEntryTimeToLive()));
+    existingAttributes.setRegionIdleTime(
+        combine(existingAttributes.getRegionIdleTime(), deltaAttributes.getRegionIdleTime()));
+    existingAttributes.setRegionTimeToLive(
+        combine(existingAttributes.getRegionTimeToLive(), deltaAttributes.getRegionTimeToLive()));
+
+    if (deltaAttributes.getCacheLoader() != null) {
+      if (deltaAttributes.getCacheLoader().equals(DeclarableType.EMPTY)) {
+        existingAttributes.setCacheLoader(null);
+      } else {
+        existingAttributes.setCacheLoader(deltaAttributes.getCacheLoader());
+      }
     }
-    return result;
+
+    if (deltaAttributes.getCacheWriter() != null) {
+      if (deltaAttributes.getCacheWriter().equals(DeclarableType.EMPTY)) {
+        existingAttributes.setCacheWriter(null);
+      } else {
+        existingAttributes.setCacheWriter(deltaAttributes.getCacheWriter());
+      }
+    }
+
+    if (!deltaAttributes.getCacheListeners().isEmpty()) {
+      existingAttributes.getCacheListeners().clear();
+      // only add the new cache listeners to the list when it's an EMPTY cache listener
+      if (!deltaAttributes.getCacheListeners().get(0).equals(DeclarableType.EMPTY)) {
+        existingAttributes.getCacheListeners().addAll(deltaAttributes.getCacheListeners());
+      }
+    }
+
+    if (deltaAttributes.getGatewaySenderIds() != null) {
+      existingAttributes.setGatewaySenderIds(deltaAttributes.getGatewaySenderIds());
+    }
+
+    if (deltaAttributes.getAsyncEventQueueIds() != null) {
+      existingAttributes.setAsyncEventQueueIds(deltaAttributes.getAsyncEventQueueIds());
+    }
+
+    if (deltaAttributes.isCloningEnabled() != null) {
+      existingAttributes.setCloningEnabled(deltaAttributes.isCloningEnabled());
+    }
+
+    EvictionAttributes evictionAttributes = deltaAttributes.getEvictionAttributes();
+    if (evictionAttributes != null) {
+      // we only set the max in the delta's lruEntryCount in the alter region command
+      String newMax = evictionAttributes.getLruEntryCount().getMaximum();
+      EvictionAttributes existingEviction = existingAttributes.getEvictionAttributes();
+
+      // we only alter the max value if there is an existing eviction attributes
+      if (existingEviction != null) {
+        if (existingEviction.getLruEntryCount() != null) {
+          existingEviction.getLruEntryCount().setMaximum(newMax);
+        }
+
+        if (existingEviction.getLruMemorySize() != null) {
+          existingEviction.getLruMemorySize().setMaximum(newMax);
+        }
+      }
+    }
+    return true;
+  }
+
+  ExpirationAttributesType getExpirationAttributes(Integer timeout,
+      ExpirationAction action, ClassName expiry) {
+    if (timeout == null && action == null && expiry == null) {
+      return null;
+    }
+    if (expiry != null) {
+      return new ExpirationAttributesType(timeout, action,
+          expiry.getClassName(), expiry.getInitProperties());
+    } else {
+      return new ExpirationAttributesType(timeout, action, null, null);
+    }
+  }
+
+  // this is a helper method to combine the existing with the delta ExpirationAttributesType
+  ExpirationAttributesType combine(ExpirationAttributesType existing,
+      ExpirationAttributesType delta) {
+    if (delta == null) {
+      return existing;
+    }
+
+    if (existing == null) {
+      existing = new ExpirationAttributesType();
+      existing.setAction(ExpirationAction.INVALIDATE.toXmlString());
+      existing.setTimeout("0");
+    }
+
+    if (delta.getTimeout() != null) {
+      existing.setTimeout(delta.getTimeout());
+    }
+    if (delta.getAction() != null) {
+      existing.setAction(delta.getAction());
+    }
+    if (delta.getCustomExpiry() != null) {
+      if (delta.getCustomExpiry().equals(DeclarableType.EMPTY)) {
+        existing.setCustomExpiry(null);
+      } else {
+        existing.setCustomExpiry(delta.getCustomExpiry());
+      }
+    }
+    return existing;
   }
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionAlterFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionAlterFunction.java
index b93e752..e8e94bb 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionAlterFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionAlterFunction.java
@@ -15,43 +15,40 @@
 package org.apache.geode.management.internal.cli.functions;
 
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
+import java.util.function.Consumer;
 
-import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.logging.log4j.Logger;
 
-import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.AttributesMutator;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheListener;
-import org.apache.geode.cache.CacheLoader;
-import org.apache.geode.cache.CacheWriter;
 import org.apache.geode.cache.CustomExpiry;
-import org.apache.geode.cache.Region;
+import org.apache.geode.cache.ExpirationAction;
+import org.apache.geode.cache.ExpirationAttributes;
+import org.apache.geode.cache.configuration.DeclarableType;
+import org.apache.geode.cache.configuration.RegionAttributesType;
+import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.cache.execute.ResultSender;
 import org.apache.geode.internal.cache.AbstractRegion;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.execute.InternalFunction;
 import org.apache.geode.internal.cache.partitioned.PRLocallyDestroyedException;
-import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.domain.ClassName;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.RegionPath;
-import org.apache.geode.management.internal.configuration.domain.XmlEntity;
+import org.apache.geode.management.cli.CliFunction;
+import org.apache.geode.management.cli.Result;
 
 /**
  * Function used by the 'alter region' gfsh command to alter a region on each member.
  *
  * @since GemFire 8.0
  */
-public class RegionAlterFunction implements InternalFunction {
+public class RegionAlterFunction extends CliFunction<RegionConfig> {
   private static final Logger logger = LogService.getLogger();
 
   private static final long serialVersionUID = -4846425364943216425L;
+  private static final String NULLSTR = "null";
 
   @Override
   public boolean isHA() {
@@ -59,134 +56,56 @@ public class RegionAlterFunction implements InternalFunction {
   }
 
   @Override
-  public void execute(FunctionContext context) {
-    ResultSender<Object> resultSender = context.getResultSender();
-
+  public CliFunctionResult executeFunction(FunctionContext<RegionConfig> context) {
     Cache cache = ((InternalCache) context.getCache()).getCacheForProcessingClientRequests();
-    String memberNameOrId =
-        CliUtil.getMemberNameOrId(cache.getDistributedSystem().getDistributedMember());
-
-    RegionFunctionArgs regionAlterArgs = (RegionFunctionArgs) context.getArguments();
-    try {
-      Region<?, ?> alteredRegion = alterRegion(cache, regionAlterArgs);
-      XmlEntity xmlEntity = new XmlEntity(CacheXml.REGION, "name", alteredRegion.getName());
-      resultSender.lastResult(new CliFunctionResult(memberNameOrId, xmlEntity,
-          CliStrings.format(CliStrings.ALTER_REGION__MSG__REGION_0_ALTERED_ON_1,
-              new Object[] {alteredRegion.getFullPath(), memberNameOrId})));
-
-    } catch (IllegalStateException e) {
-      logger.error(e.getMessage(), e);
-
-      resultSender.lastResult(new CliFunctionResult(memberNameOrId, false, e.getMessage()));
-    } catch (IllegalArgumentException e) {
-      logger.error(e.getMessage(), e);
-
-      resultSender.lastResult(new CliFunctionResult(memberNameOrId, false, e.getMessage()));
-    } catch (VirtualMachineError e) {
-      SystemFailure.initiateFailure(e);
-      throw e;
-
-    } catch (Throwable th) {
-      SystemFailure.checkFailure();
-      logger.error(th.getMessage(), th);
-
-      String exceptionMsg = th.getMessage();
-      if (exceptionMsg == null) {
-        exceptionMsg = ExceptionUtils.getStackTrace(th);
-      }
-      resultSender.lastResult(new CliFunctionResult(memberNameOrId, false, exceptionMsg));
-    }
+    RegionConfig deltaConfig = context.getArguments();
+    alterRegion(cache, deltaConfig);
+    return new CliFunctionResult(context.getMemberName(), Result.Status.OK,
+        String.format("Region %s altered", deltaConfig.getName()));
   }
 
-  private <K, V> Region<?, ?> alterRegion(Cache cache, RegionFunctionArgs regionAlterArgs) {
-    final String regionPathString = regionAlterArgs.getRegionPath();
+  void alterRegion(Cache cache, RegionConfig deltaConfig) {
+    final String regionPathString = deltaConfig.getName();
 
-    RegionPath regionPath = new RegionPath(regionPathString);
     AbstractRegion region = (AbstractRegion) cache.getRegion(regionPathString);
     if (region == null) {
-      throw new IllegalArgumentException(CliStrings.format(
-          CliStrings.ALTER_REGION__MSG__REGION_DOES_NOT_EXIST_0, new Object[] {regionPath}));
+      throw new IllegalArgumentException(String.format(
+          "Region does not exist: %s", regionPathString));
     }
 
+    RegionAttributesType regionAttributes = deltaConfig.getRegionAttributes();
     AttributesMutator mutator = region.getAttributesMutator();
 
-    if (regionAlterArgs.getCloningEnabled() != null) {
-      mutator.setCloningEnabled(regionAlterArgs.getCloningEnabled());
+    if (regionAttributes.isCloningEnabled() != null) {
+      mutator.setCloningEnabled(regionAttributes.isCloningEnabled());
       if (logger.isDebugEnabled()) {
         logger.debug("Region successfully altered - cloning");
       }
     }
 
-    if (regionAlterArgs.getEvictionMax() != null) {
-      mutator.getEvictionAttributesMutator().setMaximum(regionAlterArgs.getEvictionMax());
+    if (regionAttributes.getEvictionAttributes() != null) {
+      mutator.getEvictionAttributesMutator().setMaximum(Integer
+          .parseInt(regionAttributes.getEvictionAttributes().getLruEntryCount().getMaximum()));
       if (logger.isDebugEnabled()) {
         logger.debug("Region successfully altered - eviction attributes max");
       }
     }
 
     // Alter expiration attributes
-    final RegionFunctionArgs.ExpirationAttrs newEntryExpirationIdleTime =
-        regionAlterArgs.getEntryExpirationIdleTime();
-    if (newEntryExpirationIdleTime != null && newEntryExpirationIdleTime.isTimeOrActionSet()) {
-      mutator.setEntryIdleTimeout(
-          newEntryExpirationIdleTime.getExpirationAttributes(region.getEntryIdleTimeout()));
-      if (logger.isDebugEnabled()) {
-        logger.debug("Region successfully altered - entry idle timeout");
-      }
-    }
-
-    final RegionFunctionArgs.ExpirationAttrs newEntryExpirationTTL =
-        regionAlterArgs.getEntryExpirationTTL();
-    if (newEntryExpirationTTL != null && newEntryExpirationTTL.isTimeOrActionSet()) {
-      mutator.setEntryTimeToLive(
-          newEntryExpirationTTL.getExpirationAttributes(region.getEntryTimeToLive()));
-      if (logger.isDebugEnabled()) {
-        logger.debug("Region successfully altered - entry TTL");
-      }
-    }
-
-    final ClassName<CustomExpiry> entryIdleCustomExpiry =
-        regionAlterArgs.getEntryIdleTimeCustomExpiry();
-    if (entryIdleCustomExpiry != null) {
-      if (entryIdleCustomExpiry.equals(ClassName.EMPTY)) {
-        mutator.setCustomEntryIdleTimeout(null);
-      } else {
-        mutator.setCustomEntryIdleTimeout(entryIdleCustomExpiry.newInstance(cache));
-      }
-    }
-
-    final ClassName<CustomExpiry> entryTTLCustomExpiry = regionAlterArgs.getEntryTTLCustomExpiry();
-    if (entryTTLCustomExpiry != null) {
-      if (entryTTLCustomExpiry.equals(ClassName.EMPTY)) {
-        mutator.setCustomEntryTimeToLive(null);
-      } else {
-        mutator.setCustomEntryTimeToLive(entryTTLCustomExpiry.newInstance(cache));
-      }
-    }
-
-    final RegionFunctionArgs.ExpirationAttrs newRegionExpirationIdleTime =
-        regionAlterArgs.getRegionExpirationIdleTime();
-    if (newRegionExpirationIdleTime != null && newRegionExpirationIdleTime.isTimeOrActionSet()) {
-      mutator.setRegionIdleTimeout(
-          newRegionExpirationIdleTime.getExpirationAttributes(region.getRegionIdleTimeout()));
-      if (logger.isDebugEnabled()) {
-        logger.debug("Region successfully altered - region idle timeout");
-      }
-    }
-
-    final RegionFunctionArgs.ExpirationAttrs newRegionExpirationTTL =
-        regionAlterArgs.getRegionExpirationTTL();
-    if (newRegionExpirationTTL != null && newRegionExpirationTTL.isTimeOrActionSet()) {
-      mutator.setRegionTimeToLive(
-          newRegionExpirationTTL.getExpirationAttributes(region.getRegionTimeToLive()));
-      if (logger.isDebugEnabled()) {
-        logger.debug("Region successfully altered - region TTL");
-      }
-    }
+    updateExpirationAttributes(cache, regionAttributes.getEntryIdleTime(),
+        region.getEntryIdleTimeout(), p -> mutator.setEntryIdleTimeout(p),
+        p -> mutator.setCustomEntryIdleTimeout(p));
+    updateExpirationAttributes(cache, regionAttributes.getEntryTimeToLive(),
+        region.getEntryTimeToLive(), p -> mutator.setEntryTimeToLive(p),
+        p -> mutator.setCustomEntryTimeToLive(p));
+    updateExpirationAttributes(cache, regionAttributes.getRegionIdleTime(),
+        region.getRegionIdleTimeout(), p -> mutator.setRegionIdleTimeout(p), null);
+    updateExpirationAttributes(cache, regionAttributes.getRegionTimeToLive(),
+        region.getRegionTimeToLive(), p -> mutator.setRegionTimeToLive(p), null);
 
-    final Set<String> newGatewaySenderIds = regionAlterArgs.getGatewaySenderIds();
-    final Set<String> newAsyncEventQueueIds = regionAlterArgs.getAsyncEventQueueIds();
 
+    final Set<String> newGatewaySenderIds = regionAttributes.getGatewaySenderIdsAsSet();
+    final Set<String> newAsyncEventQueueIds = regionAttributes.getAsyncEventQueueIdsAsSet();
 
     if (region instanceof PartitionedRegion) {
       Set<String> senderIds = new HashSet<>();
@@ -255,10 +174,10 @@ public class RegionAlterFunction implements InternalFunction {
     }
 
     // Alter Cache Listeners
-    final Set<ClassName<CacheListener>> newCacheListeners = regionAlterArgs.getCacheListeners();
+    final List<DeclarableType> newCacheListeners = regionAttributes.getCacheListeners();
 
     // user specified a new set of cache listeners
-    if (newCacheListeners != null) {
+    if (!newCacheListeners.isEmpty()) {
       // remove the old ones, even if the new set includes the same class name, the init properties
       // might be different
       CacheListener[] oldCacheListeners = region.getCacheListeners();
@@ -267,8 +186,8 @@ public class RegionAlterFunction implements InternalFunction {
       }
 
       // Add new cache listeners
-      for (ClassName<CacheListener> newCacheListener : newCacheListeners) {
-        if (!newCacheListener.equals(ClassName.EMPTY)) {
+      for (DeclarableType newCacheListener : newCacheListeners) {
+        if (!newCacheListener.equals(DeclarableType.EMPTY)) {
           mutator.addCacheListener(newCacheListener.newInstance(cache));
         }
       }
@@ -277,9 +196,9 @@ public class RegionAlterFunction implements InternalFunction {
       }
     }
 
-    final ClassName<CacheLoader> cacheLoader = regionAlterArgs.getCacheLoader();
+    final DeclarableType cacheLoader = regionAttributes.getCacheLoader();
     if (cacheLoader != null) {
-      if (cacheLoader.equals(ClassName.EMPTY)) {
+      if (cacheLoader.equals(DeclarableType.EMPTY)) {
         mutator.setCacheLoader(null);
       } else {
         mutator.setCacheLoader(cacheLoader.newInstance(cache));
@@ -290,9 +209,9 @@ public class RegionAlterFunction implements InternalFunction {
       }
     }
 
-    final ClassName<CacheWriter> cacheWriter = regionAlterArgs.getCacheWriter();
+    final DeclarableType cacheWriter = regionAttributes.getCacheWriter();
     if (cacheWriter != null) {
-      if (cacheWriter.equals(ClassName.EMPTY)) {
+      if (cacheWriter.equals(DeclarableType.EMPTY)) {
         mutator.setCacheWriter(null);
       } else {
         mutator.setCacheWriter(cacheWriter.newInstance(cache));
@@ -302,10 +221,49 @@ public class RegionAlterFunction implements InternalFunction {
         logger.debug("Region successfully altered - cache writer");
       }
     }
+  }
+
+  private void updateExpirationAttributes(Cache cache,
+      RegionAttributesType.ExpirationAttributesType newAttributes,
+      ExpirationAttributes existingAttributes,
+      Consumer<ExpirationAttributes> mutator1,
+      Consumer<CustomExpiry> mutator2) {
+    if (newAttributes == null) {
+      return;
+    }
+
+    if (newAttributes.hasTimoutOrAction() && existingAttributes != null) {
+      int existingTimeout = existingAttributes.getTimeout();
+      ExpirationAction existingAction = existingAttributes.getAction();
+      if (newAttributes.getTimeout() != null) {
+        existingTimeout = Integer.parseInt(newAttributes.getTimeout());
+      }
+
+      if (newAttributes.getAction() != null) {
+        existingAction = ExpirationAction.fromXmlString(newAttributes.getAction());
+      }
+      mutator1.accept(new ExpirationAttributes(existingTimeout, existingAction));
+    }
 
-    return region;
+    if (mutator2 == null) {
+      return;
+    }
+
+    if (newAttributes.hasCustomExpiry()) {
+      DeclarableType newCustomExpiry = newAttributes.getCustomExpiry();
+      if (newCustomExpiry.equals(DeclarableType.EMPTY)) {
+        mutator2.accept(null);
+      } else {
+        mutator2.accept(newCustomExpiry.newInstance(cache));
+      }
+    }
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("Region successfully altered - entry idle timeout");
+    }
   }
 
+
   private void validateParallelGatewaySenderIDs(PartitionedRegion region,
       Set<String> newGatewaySenderIds) {
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
index 1401d50..154db6b 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
@@ -249,8 +249,7 @@ public class RegionCreateFunction implements InternalFunction {
       RegionAttributesType.ExpirationAttributesType eitl = regionAttributes.getEntryIdleTime();
       factory.setEntryIdleTimeout(
           new ExpirationAttributes(Integer.valueOf(eitl.getTimeout()),
-              ExpirationAction.fromString(eitl.getAction().toUpperCase()
-                  .replace("-", "_"))));
+              ExpirationAction.fromXmlString(eitl.getAction())));
 
       try {
         if (eitl.getCustomExpiry() != null) {
@@ -266,8 +265,7 @@ public class RegionCreateFunction implements InternalFunction {
       RegionAttributesType.ExpirationAttributesType ettl = regionAttributes.getEntryTimeToLive();
       factory.setEntryTimeToLive(
           new ExpirationAttributes(Integer.valueOf(ettl.getTimeout()),
-              ExpirationAction.fromString(ettl.getAction().toUpperCase()
-                  .replace("-", "_"))));
+              ExpirationAction.fromXmlString(ettl.getAction())));
 
       try {
         if (ettl.getCustomExpiry() != null) {
@@ -283,16 +281,14 @@ public class RegionCreateFunction implements InternalFunction {
       RegionAttributesType.ExpirationAttributesType ritl = regionAttributes.getRegionIdleTime();
       factory.setRegionIdleTimeout(
           new ExpirationAttributes(Integer.valueOf(ritl.getTimeout()),
-              ExpirationAction.fromString(ritl.getAction().toUpperCase()
-                  .replace("-", "_"))));
+              ExpirationAction.fromXmlString(ritl.getAction())));
     }
 
     if (regionAttributes.getRegionTimeToLive() != null) {
       RegionAttributesType.ExpirationAttributesType rttl = regionAttributes.getRegionTimeToLive();
       factory.setRegionTimeToLive(
           new ExpirationAttributes(Integer.valueOf(rttl.getTimeout()),
-              ExpirationAction.fromString(rttl.getAction().toUpperCase()
-                  .replace("-", "_"))));
+              ExpirationAction.fromXmlString(rttl.getAction())));
     }
   }
 
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 d695a26..a1350b2 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
@@ -133,7 +133,7 @@ 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);
     InternalConfigurationPersistenceService ccService =
-        (InternalConfigurationPersistenceService) gfshCommand.getConfigurationPersistenceService();
+        gfshCommand.getConfigurationPersistenceService();
     if (ccService == null) {
       infoResultModel.addLine(SERVICE_NOT_RUNNING_CHANGE_NOT_PERSISTED);
       return resultModel;
@@ -179,7 +179,7 @@ public class CommandExecutor {
             }
           }
         } catch (Exception e) {
-          String message = "failed to update cluster config for " + group;
+          String message = "Failed to update cluster config for " + group;
           logger.error(message, e);
           // for now, if one cc update failed, we will set this flag. Will change this when we can
           // add lines to the result returned by the command
diff --git a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
index ab0c4d1..a7ca52a 100644
--- a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
+++ b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
@@ -136,6 +136,7 @@ org/apache/geode/cache/configuration/RegionAttributesIndexUpdateType,false,value
 org/apache/geode/cache/configuration/RegionAttributesMirrorType,false,value:java/lang/String
 org/apache/geode/cache/configuration/RegionAttributesScope,false,value:java/lang/String
 org/apache/geode/cache/configuration/RegionAttributesType,false,asyncEventQueueIds:java/lang/String,cacheListeners:java/util/List,cacheLoader:org/apache/geode/cache/configuration/DeclarableType,cacheWriter:org/apache/geode/cache/configuration/DeclarableType,cloningEnabled:java/lang/Boolean,compressor:org/apache/geode/cache/configuration/ClassNameType,concurrencyChecksEnabled:java/lang/Boolean,concurrencyLevel:java/lang/String,dataPolicy:org/apache/geode/cache/configuration/RegionAttribut [...]
+org/apache/geode/cache/configuration/RegionAttributesType$DiskWriteAttributes,false,asynchronousWrites:org/apache/geode/cache/configuration/RegionAttributesType$DiskWriteAttributes$AsynchronousWrites,maxOplogSize:java/lang/String,rollOplogs:java/lang/String,synchronousWrites:java/lang/Object
 org/apache/geode/cache/configuration/RegionAttributesType$EvictionAttributes,false,lruEntryCount:org/apache/geode/cache/configuration/RegionAttributesType$EvictionAttributes$LruEntryCount,lruHeapPercentage:org/apache/geode/cache/configuration/RegionAttributesType$EvictionAttributes$LruHeapPercentage,lruMemorySize:org/apache/geode/cache/configuration/RegionAttributesType$EvictionAttributes$LruMemorySize
 org/apache/geode/cache/configuration/RegionAttributesType$EvictionAttributes$LruEntryCount,false,action:org/apache/geode/cache/configuration/EnumActionDestroyOverflow,maximum:java/lang/String
 org/apache/geode/cache/configuration/RegionAttributesType$EvictionAttributes$LruHeapPercentage,false,action:org/apache/geode/cache/configuration/EnumActionDestroyOverflow
diff --git a/geode-core/src/test/java/org/apache/geode/cache/configuration/DeclarableTypeTest.java b/geode-core/src/test/java/org/apache/geode/cache/configuration/DeclarableTypeTest.java
index 42c3608..3f51557 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/configuration/DeclarableTypeTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/configuration/DeclarableTypeTest.java
@@ -19,6 +19,8 @@ package org.apache.geode.cache.configuration;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
+import java.util.Properties;
+
 import org.junit.Test;
 
 
@@ -72,4 +74,15 @@ public class DeclarableTypeTest {
     d3.getParameters().add(p3);
     assertThat(declarable).isNotEqualTo(d3);
   }
+
+  @Test
+  public void emptyDeclarable() {
+    assertThat(new DeclarableType(""))
+        .isEqualTo(new DeclarableType("", (Properties) null))
+        .isEqualTo(new DeclarableType("", ""))
+        .isEqualTo(new DeclarableType("", "{}"))
+        .isEqualTo(new DeclarableType(" ", "{}"))
+        .isEqualTo(new DeclarableType(null, "{}"))
+        .isEqualTo(DeclarableType.EMPTY);
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionAttributesTypeTest.java b/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionAttributesTypeTest.java
new file mode 100644
index 0000000..40b9584
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionAttributesTypeTest.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.cache.configuration;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.geode.cache.ExpirationAction;
+
+public class RegionAttributesTypeTest {
+
+  private RegionAttributesType.ExpirationAttributesType expirationAttributes;
+  private RegionAttributesType regionAttributes;
+
+  @Before
+  public void before() throws Exception {
+    regionAttributes = new RegionAttributesType();
+    expirationAttributes = new RegionAttributesType.ExpirationAttributesType();
+  }
+
+  @Test
+  public void emptyConstructor() {
+    expirationAttributes = new RegionAttributesType.ExpirationAttributesType();
+    assertThat(expirationAttributes.getAction()).isNull();
+    assertThat(expirationAttributes.getTimeout()).isNull();
+    assertThat(expirationAttributes.getCustomExpiry()).isNull();
+  }
+
+  @Test
+  public void constructorWithParameter() {
+    expirationAttributes =
+        new RegionAttributesType.ExpirationAttributesType(null, ExpirationAction.DESTROY, null,
+            null);
+    assertThat(expirationAttributes.getAction()).isEqualTo("destroy");
+    assertThat(expirationAttributes.getTimeout()).isNull();
+    assertThat(expirationAttributes.getCustomExpiry()).isNull();
+    assertThat(expirationAttributes.hasTimoutOrAction()).isTrue();
+    assertThat(expirationAttributes.hasCustomExpiry()).isFalse();
+
+    expirationAttributes = new RegionAttributesType.ExpirationAttributesType(10, null, null, null);
+    assertThat(expirationAttributes.getAction()).isNull();
+    assertThat(expirationAttributes.getTimeout()).isEqualTo("10");
+    assertThat(expirationAttributes.getCustomExpiry()).isNull();
+    assertThat(expirationAttributes.hasTimoutOrAction()).isTrue();
+    assertThat(expirationAttributes.hasCustomExpiry()).isFalse();
+
+    expirationAttributes =
+        new RegionAttributesType.ExpirationAttributesType(null, null, "abc", null);
+    assertThat(expirationAttributes.getAction()).isNull();
+    assertThat(expirationAttributes.getTimeout()).isNull();
+    assertThat(expirationAttributes.getCustomExpiry()).isNotNull();
+    assertThat(expirationAttributes.hasTimoutOrAction()).isFalse();
+    assertThat(expirationAttributes.hasCustomExpiry()).isTrue();
+  }
+
+  @Test
+  public void gatewaySender() {
+    regionAttributes.setGatewaySenderIds(null);
+    assertThat(regionAttributes.getGatewaySenderIdsAsSet()).isNull();
+
+    regionAttributes.setGatewaySenderIds("");
+    assertThat(regionAttributes.getGatewaySenderIdsAsSet()).isNotNull().isEmpty();
+
+    regionAttributes.setGatewaySenderIds("abc,def");
+    assertThat(regionAttributes.getGatewaySenderIdsAsSet()).isNotNull().hasSize(2);
+  }
+
+  @Test
+  public void asyncEventQueue() {
+    regionAttributes.setAsyncEventQueueIds(null);
+    assertThat(regionAttributes.getAsyncEventQueueIdsAsSet()).isNull();
+
+    regionAttributes.setAsyncEventQueueIds("");
+    assertThat(regionAttributes.getAsyncEventQueueIdsAsSet()).isNotNull().isEmpty();
+
+    regionAttributes.setAsyncEventQueueIds("abc,def");
+    assertThat(regionAttributes.getAsyncEventQueueIdsAsSet()).isNotNull().hasSize(2);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java b/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java
new file mode 100644
index 0000000..85cd175
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.configuration;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Test;
+
+public class RegionConfigTest {
+
+  @Test
+  public void regionNameShouldNotHaveSlash() {
+    RegionConfig config = new RegionConfig();
+    config.setName("/regionA");
+    assertThat(config.getName()).isEqualTo("regionA");
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandTest.java
index d8d6725..beada6c 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandTest.java
@@ -16,17 +16,36 @@
 package org.apache.geode.management.internal.cli.commands;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.cache.ExpirationAction;
+import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.cache.configuration.DeclarableType;
+import org.apache.geode.cache.configuration.EnumActionDestroyOverflow;
+import org.apache.geode.cache.configuration.RegionAttributesType;
+import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.domain.ClassName;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.test.junit.rules.GfshParserRule;
 
 
@@ -37,16 +56,35 @@ public class AlterRegionCommandTest {
 
   private AlterRegionCommand command;
   private InternalCache cache;
+  private InternalConfigurationPersistenceService ccService;
+  private CacheConfig cacheConfig;
+  private RegionConfig existingRegionConfig;
 
   @Before
-  public void before() throws Exception {
+  public void before() {
     command = spy(AlterRegionCommand.class);
     cache = mock(InternalCache.class);
-    doReturn(cache).when(command).getCache();
+    command.setCache(cache);
+    when(cache.getSecurityService()).thenReturn(mock(SecurityService.class));
+    ccService = mock(InternalConfigurationPersistenceService.class);
+    doReturn(ccService).when(command).getConfigurationPersistenceService();
+    Set<DistributedMember> members =
+        Stream.of(mock(DistributedMember.class)).collect(Collectors.toSet());
+    doReturn(members).when(command).findMembers(any(), any());
+    CliFunctionResult result =
+        new CliFunctionResult("member", CliFunctionResult.StatusState.OK, "regionA altered");
+    doReturn(Arrays.asList(result)).when(command).executeAndGetFunctionResult(any(), any(), any());
+
+    cacheConfig = new CacheConfig();
+    existingRegionConfig = new RegionConfig();
+    existingRegionConfig.setName("/regionA");
+    existingRegionConfig.setRefid("REPLICATE");
+    cacheConfig.getRegions().add(existingRegionConfig);
+    when(ccService.getCacheConfig("cluster")).thenReturn(cacheConfig);
   }
 
   @Test
-  public void cacheWriterEmpty() throws Exception {
+  public void cacheWriterEmpty() {
     String command = "alter region --name=/Person --cache-writer='' --cache-loader=' '";
     GfshParseResult result = parser.parse(command);
     assertThat(result.getParamValue("cache-writer")).isEqualTo(ClassName.EMPTY);
@@ -56,17 +94,23 @@ public class AlterRegionCommandTest {
   }
 
   @Test
-  public void cacheWriterInvalid() throws Exception {
+  public void cacheWriterInvalid() {
     String command = "alter region --name=/Person --cache-writer='1abc'";
     GfshParseResult result = parser.parse(command);
     assertThat(result).isNull();
   }
 
   @Test
-  public void emptyCustomExpiry() {
+  public void emptyCustomExpiryAndNoCloning() {
     String command = "alter region --name=/Person --entry-idle-time-custom-expiry=''";
     GfshParseResult result = parser.parse(command);
-    assertThat(result.getParamValue("entry-idle-time-custom-expiry")).isEqualTo(ClassName.EMPTY);
+    ClassName paramValue = (ClassName) result.getParamValue("entry-idle-time-custom-expiry");
+    assertThat(paramValue).isEqualTo(ClassName.EMPTY);
+    assertThat(paramValue.getClassName()).isEqualTo("");
+
+    // when enable-cloning is not specified, the value should be null
+    Object enableCloning = result.getParamValue("enable-cloning");
+    assertThat(enableCloning).isNull();
   }
 
   @Test
@@ -75,4 +119,383 @@ public class AlterRegionCommandTest {
     GfshParseResult result = parser.parse(command);
     assertThat(result.getParamValue("name")).isEqualTo("/Person");
   }
+
+  @Test
+  public void groupNotExist() {
+    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
+    parser.executeAndAssertThat(command,
+        "alter region --group=group0 --name=regionA --cache-loader=abc")
+        .statusIsError()
+        .hasInfoSection().hasLines().contains("No Members Found");
+  }
+
+  @Test
+  public void regionNotExistOnGroup() {
+    parser.executeAndAssertThat(command,
+        "alter region --name=regionB --cache-loader=abc")
+        .statusIsError()
+        .hasInfoSection().hasLines().contains("/regionB does not exist in group cluster");
+
+    parser.executeAndAssertThat(command,
+        "alter region --group=group1 --name=regionA --cache-loader=abc")
+        .statusIsError()
+        .hasInfoSection().hasLines().contains("/regionA does not exist in group group1");
+  }
+
+  @Test
+  public void ccServiceNotAvailable() {
+    doReturn(null).when(command).getConfigurationPersistenceService();
+    parser.executeAndAssertThat(command,
+        "alter region --name=regionB --cache-loader=abc")
+        .statusIsSuccess()
+        .hasInfoSection().hasOutput().contains(
+            "Cluster configuration service is not running. Configuration change is not persisted");
+  }
+
+  @Test
+  public void alterWithCloningEnabled() {
+    RegionAttributesType regionAttributes =
+        getDeltaRegionConfig("alter region --name=regionA --enable-cloning=false")
+            .getRegionAttributes();
+    assertThat(regionAttributes.isCloningEnabled()).isFalse();
+    assertThat(regionAttributes.getAsyncEventQueueIds()).isNull();
+    assertThat(regionAttributes.getDataPolicy()).isNull();
+    assertThat(regionAttributes.getGatewaySenderIds()).isNull();
+    assertThat(regionAttributes.getCacheLoader()).isNull();
+    assertThat(regionAttributes.getCacheWriter()).isNull();
+    assertThat(regionAttributes.getCacheListeners()).isNotNull().isEmpty();
+    assertThat(regionAttributes.getEvictionAttributes()).isNull();
+    assertThat(regionAttributes.getEntryIdleTime()).isNull();
+    assertThat(regionAttributes.getEntryTimeToLive()).isNull();
+    assertThat(regionAttributes.getRegionIdleTime()).isNull();
+    assertThat(regionAttributes.getRegionTimeToLive()).isNull();
+  }
+
+  @Test
+  public void alterWithEntryIdleTimeOut() {
+    // check that the deltaConfig is created as expected
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --entry-idle-time-expiration=7");
+    RegionAttributesType.ExpirationAttributesType entryIdleTime =
+        deltaConfig.getRegionAttributes().getEntryIdleTime();
+    assertThat(entryIdleTime).isNotNull();
+    assertThat(entryIdleTime.getTimeout()).isEqualTo("7");
+    assertThat(entryIdleTime.getCustomExpiry()).isNull();
+    assertThat(entryIdleTime.getAction()).isNull();
+
+    // check that the combined the configuration is created as expected
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    RegionAttributesType.ExpirationAttributesType expirationAttributesType =
+        new RegionAttributesType.ExpirationAttributesType(10, ExpirationAction.DESTROY, null, null);
+    existingAttributes.setEntryIdleTime(expirationAttributesType);
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    RegionAttributesType.ExpirationAttributesType combinedExpirationAttributes =
+        existingRegionConfig.getRegionAttributes().getEntryIdleTime();
+    assertThat(combinedExpirationAttributes.getTimeout()).isEqualTo("7");
+    assertThat(combinedExpirationAttributes.getAction()).isEqualTo("destroy");
+    assertThat(combinedExpirationAttributes.getCustomExpiry()).isNull();
+  }
+
+  @Test
+  public void alterWithEntryIdleTimeOutAction() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig(
+            "alter region --name=regionA --entry-idle-time-expiration-action=destroy");
+    RegionAttributesType.ExpirationAttributesType entryIdleTime =
+        deltaConfig.getRegionAttributes().getEntryIdleTime();
+    assertThat(entryIdleTime).isNotNull();
+    assertThat(entryIdleTime.getTimeout()).isNull();
+    assertThat(entryIdleTime.getCustomExpiry()).isNull();
+    assertThat(entryIdleTime.getAction()).isEqualTo("destroy");
+
+    // check that the combined the configuration is created as expected
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    RegionAttributesType.ExpirationAttributesType expirationAttributesType =
+        new RegionAttributesType.ExpirationAttributesType(10, ExpirationAction.INVALIDATE, null,
+            null);
+    existingAttributes.setEntryIdleTime(expirationAttributesType);
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    RegionAttributesType.ExpirationAttributesType combinedExpirationAttributes =
+        existingRegionConfig.getRegionAttributes().getEntryIdleTime();
+    assertThat(combinedExpirationAttributes.getTimeout()).isEqualTo("10");
+    assertThat(combinedExpirationAttributes.getAction()).isEqualTo("destroy");
+    assertThat(combinedExpirationAttributes.getCustomExpiry()).isNull();
+  }
+
+  @Test
+  public void alterWithEntryIdleTimeOutCustomExpiry() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --entry-idle-time-custom-expiry=abc");
+    RegionAttributesType.ExpirationAttributesType entryIdleTime =
+        deltaConfig.getRegionAttributes().getEntryIdleTime();
+    assertThat(entryIdleTime).isNotNull();
+    assertThat(entryIdleTime.getTimeout()).isNull();
+    assertThat(entryIdleTime.getCustomExpiry().getClassName()).isEqualTo("abc");
+    assertThat(entryIdleTime.getAction()).isNull();
+
+    // check that the combined the configuration is created as expected
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    RegionAttributesType.ExpirationAttributesType expirationAttributesType =
+        new RegionAttributesType.ExpirationAttributesType(10, ExpirationAction.INVALIDATE, null,
+            null);
+    existingAttributes.setEntryIdleTime(expirationAttributesType);
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    RegionAttributesType.ExpirationAttributesType combinedExpirationAttributes =
+        existingRegionConfig.getRegionAttributes().getEntryIdleTime();
+    assertThat(combinedExpirationAttributes.getTimeout()).isEqualTo("10");
+    assertThat(combinedExpirationAttributes.getAction()).isEqualTo("invalidate");
+    assertThat(combinedExpirationAttributes.getCustomExpiry().getClassName()).isEqualTo("abc");
+  }
+
+  @Test
+  public void alterWithEmptyEntryIdleTimeOutCustomExpiry() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --entry-idle-time-custom-expiry=''");
+    RegionAttributesType.ExpirationAttributesType entryIdleTime =
+        deltaConfig.getRegionAttributes().getEntryIdleTime();
+    assertThat(entryIdleTime).isNotNull();
+    assertThat(entryIdleTime.getTimeout()).isNull();
+    assertThat(entryIdleTime.getCustomExpiry()).isEqualTo(DeclarableType.EMPTY);
+    assertThat(entryIdleTime.getAction()).isNull();
+
+    // check that the combined the configuration is created as expected
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    RegionAttributesType.ExpirationAttributesType expirationAttributesType =
+        new RegionAttributesType.ExpirationAttributesType(10, ExpirationAction.INVALIDATE, null,
+            null);
+    existingAttributes.setEntryIdleTime(expirationAttributesType);
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    RegionAttributesType.ExpirationAttributesType combinedExpirationAttributes =
+        existingRegionConfig.getRegionAttributes().getEntryIdleTime();
+    assertThat(combinedExpirationAttributes.getTimeout()).isEqualTo("10");
+    assertThat(combinedExpirationAttributes.getAction()).isEqualTo("invalidate");
+    assertThat(combinedExpirationAttributes.getCustomExpiry()).isNull();
+  }
+
+  @Test
+  public void alterWithCacheListener() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --cache-listener=abc,def");
+    List<DeclarableType> cacheListeners = deltaConfig.getRegionAttributes().getCacheListeners();
+    assertThat(cacheListeners).hasSize(2);
+    assertThat(cacheListeners.get(0).getClassName()).isEqualTo("abc");
+    assertThat(cacheListeners.get(1).getClassName()).isEqualTo("def");
+
+    // check that the combined the configuration is created as expected
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingAttributes.getCacheListeners().add(new DeclarableType("ghi"));
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+
+    List<DeclarableType> updatedCacheListeners =
+        existingRegionConfig.getRegionAttributes().getCacheListeners();
+    assertThat(updatedCacheListeners).hasSize(2);
+    assertThat(updatedCacheListeners.get(0).getClassName()).isEqualTo("abc");
+    assertThat(updatedCacheListeners.get(1).getClassName()).isEqualTo("def");
+
+    assertThat(existingRegionConfig.getRegionAttributes().getEntryIdleTime()).isNull();
+  }
+
+  @Test
+  public void alterWithNoCacheListener() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --cache-listener=''");
+    List<DeclarableType> cacheListeners = deltaConfig.getRegionAttributes().getCacheListeners();
+    assertThat(cacheListeners).hasSize(1);
+    assertThat(cacheListeners.get(0)).isEqualTo(DeclarableType.EMPTY);
+
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingAttributes.getCacheListeners().add(new DeclarableType("ghi"));
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+
+    List<DeclarableType> updatedCacheListeners =
+        existingRegionConfig.getRegionAttributes().getCacheListeners();
+    assertThat(updatedCacheListeners).hasSize(0);
+  }
+
+  @Test
+  public void alterWithCacheLoader() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --cache-loader=abc");
+    RegionAttributesType deltaAttributes = deltaConfig.getRegionAttributes();
+    assertThat(deltaAttributes.getCacheWriter()).isNull();
+    assertThat(deltaAttributes.getCacheLoader().getClassName()).isEqualTo("abc");
+    assertThat(deltaAttributes.getCacheListeners()).isNotNull().isEmpty();
+
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingAttributes.getCacheListeners().add(new DeclarableType("def"));
+    existingAttributes.setCacheLoader(new DeclarableType("def"));
+    existingAttributes.setCacheWriter(new DeclarableType("def"));
+
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+
+    // after update, the cache listeners remains the same
+    assertThat(existingAttributes.getCacheListeners()).hasSize(1);
+    assertThat(existingAttributes.getCacheListeners().get(0).getClassName()).isEqualTo("def");
+
+    // after update the cache writer remains the same
+    assertThat(existingAttributes.getCacheWriter().getClassName()).isEqualTo("def");
+
+    // after update the cache loader is changed
+    assertThat(existingAttributes.getCacheLoader().getClassName()).isEqualTo("abc");
+  }
+
+  @Test
+  public void alterWithNoCacheLoader() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --cache-loader=''");
+    RegionAttributesType deltaAttributes = deltaConfig.getRegionAttributes();
+    assertThat(deltaAttributes.getCacheLoader()).isEqualTo(DeclarableType.EMPTY);
+
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingAttributes.setCacheLoader(new DeclarableType("def"));
+
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+
+    // after the update the cache loader is null
+    assertThat(existingAttributes.getCacheLoader()).isNull();
+  }
+
+  @Test
+  public void alterWithAsyncEventQueueIds() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --async-event-queue-id=abc,def");
+    assertThat(deltaConfig.getRegionAttributes().getAsyncEventQueueIds()).isEqualTo("abc,def");
+    assertThat(deltaConfig.getRegionAttributes().getGatewaySenderIds()).isNull();
+
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    existingAttributes.setAsyncEventQueueIds("xyz");
+    existingAttributes.setGatewaySenderIds("xyz");
+
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    assertThat(existingAttributes.getGatewaySenderIds()).isEqualTo("xyz");
+    assertThat(existingAttributes.getAsyncEventQueueIds()).isEqualTo("abc,def");
+  }
+
+  @Test
+  public void alterWithNoAsyncEventQueueIds() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --async-event-queue-id=''");
+    assertThat(deltaConfig.getRegionAttributes().getAsyncEventQueueIds()).isEqualTo("");
+    assertThat(deltaConfig.getRegionAttributes().getGatewaySenderIds()).isNull();
+
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    existingAttributes.setAsyncEventQueueIds("xyz");
+    existingAttributes.setGatewaySenderIds("xyz");
+
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    assertThat(existingAttributes.getGatewaySenderIds()).isEqualTo("xyz");
+    assertThat(existingAttributes.getAsyncEventQueueIds()).isEqualTo("");
+    assertThat(existingAttributes.getAsyncEventQueueIdsAsSet()).isNotNull().isEmpty();
+  }
+
+  @Test
+  public void alterWithEvictionMaxWithExistingLruHeapPercentage() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --eviction-max=20");
+
+    // we are saving the eviction-max as a lruEntryCount's maximum value
+    RegionAttributesType.EvictionAttributes.LruEntryCount lruEntryCount =
+        deltaConfig.getRegionAttributes().getEvictionAttributes().getLruEntryCount();
+    assertThat(lruEntryCount.getMaximum()).isEqualTo("20");
+
+    // when there is no eviction attributes at all
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    assertThat(existingAttributes.getEvictionAttributes()).isNull();
+
+    // when there is lruHeapPercentage eviction
+    RegionAttributesType.EvictionAttributes evictionAttributes =
+        new RegionAttributesType.EvictionAttributes();
+    evictionAttributes
+        .setLruHeapPercentage(new RegionAttributesType.EvictionAttributes.LruHeapPercentage());
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    assertThat(evictionAttributes.getLruEntryCount()).isNull();
+    assertThat(evictionAttributes.getLruMemorySize()).isNull();
+  }
+
+  @Test
+  public void alterWithEvictionMaxWithExistingLruEntryCount() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --eviction-max=20");
+
+    // we are saving the eviction-max as a lruEntryCount's maximum value
+    RegionAttributesType.EvictionAttributes.LruEntryCount lruEntryCount =
+        deltaConfig.getRegionAttributes().getEvictionAttributes().getLruEntryCount();
+    assertThat(lruEntryCount.getMaximum()).isEqualTo("20");
+
+    // when there is no eviction attributes at all
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    assertThat(existingAttributes.getEvictionAttributes()).isNull();
+
+    // when there is lruHeapPercentage eviction
+    RegionAttributesType.EvictionAttributes evictionAttributes =
+        new RegionAttributesType.EvictionAttributes();
+    existingAttributes.setEvictionAttributes(evictionAttributes);
+    RegionAttributesType.EvictionAttributes.LruEntryCount existingEntryCount =
+        new RegionAttributesType.EvictionAttributes.LruEntryCount();
+    existingEntryCount.setMaximum("100");
+    existingEntryCount.setAction(EnumActionDestroyOverflow.LOCAL_DESTROY);
+    evictionAttributes.setLruEntryCount(existingEntryCount);
+
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    assertThat(evictionAttributes.getLruEntryCount().getMaximum()).isEqualTo("20");
+    assertThat(evictionAttributes.getLruEntryCount().getAction())
+        .isEqualTo(EnumActionDestroyOverflow.LOCAL_DESTROY);
+  }
+
+  @Test
+  public void alterWithEvictionMaxWithExistingLruMemory() {
+    RegionConfig deltaConfig =
+        getDeltaRegionConfig("alter region --name=regionA --eviction-max=20");
+
+    // we are saving the eviction-max as a lruEntryCount's maximum value
+    RegionAttributesType.EvictionAttributes.LruEntryCount lruEntryCount =
+        deltaConfig.getRegionAttributes().getEvictionAttributes().getLruEntryCount();
+    assertThat(lruEntryCount.getMaximum()).isEqualTo("20");
+
+    // when there is no eviction attributes at all
+    RegionAttributesType existingAttributes = new RegionAttributesType();
+    existingRegionConfig.setRegionAttributes(existingAttributes);
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    assertThat(existingAttributes.getEvictionAttributes()).isNull();
+
+    // when there is lruHeapPercentage eviction
+    RegionAttributesType.EvictionAttributes evictionAttributes =
+        new RegionAttributesType.EvictionAttributes();
+    existingAttributes.setEvictionAttributes(evictionAttributes);
+    RegionAttributesType.EvictionAttributes.LruMemorySize existingMemorySize =
+        new RegionAttributesType.EvictionAttributes.LruMemorySize();
+    existingMemorySize.setMaximum("100");
+    existingMemorySize.setAction(EnumActionDestroyOverflow.LOCAL_DESTROY);
+    evictionAttributes.setLruMemorySize(existingMemorySize);
+
+    command.updateConfigForGroup("cluster", cacheConfig, deltaConfig);
+    assertThat(evictionAttributes.getLruMemorySize().getMaximum()).isEqualTo("20");
+    assertThat(evictionAttributes.getLruMemorySize().getAction())
+        .isEqualTo(EnumActionDestroyOverflow.LOCAL_DESTROY);
+  }
+
+  private RegionConfig getDeltaRegionConfig(String commandString) {
+    RegionConfig regionConfig =
+        (RegionConfig) parser.executeAndAssertThat(command, commandString)
+            .getResultModel().getConfigObject();
+    return regionConfig;
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/domain/ClassNameTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/domain/ClassNameTest.java
index 4d4d9eb..9b985fc 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/domain/ClassNameTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/domain/ClassNameTest.java
@@ -36,7 +36,9 @@ public class ClassNameTest {
 
   @Test
   public void empty() {
-    assertThat(new ClassName("", "{}")).isEqualTo(new ClassName(" ", "{\"k\":\"v\"}"))
+    assertThat(new ClassName("", "{}"))
+        .isEqualTo(new ClassName(null, "{}"))
+        .isEqualTo(new ClassName(" ", "{\"k\":\"v\"}"))
         .isEqualTo(ClassName.EMPTY);
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/RegionAlterFunctionTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/RegionAlterFunctionTest.java
new file mode 100644
index 0000000..8f41f60
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/RegionAlterFunctionTest.java
@@ -0,0 +1,278 @@
+/*
+ * 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.functions;
+
+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.anyString;
+import static org.mockito.ArgumentMatchers.notNull;
+import static org.mockito.Mockito.doNothing;
+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.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.HashSet;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import org.apache.geode.cache.AttributesMutator;
+import org.apache.geode.cache.CacheListener;
+import org.apache.geode.cache.CacheWriter;
+import org.apache.geode.cache.CustomExpiry;
+import org.apache.geode.cache.EvictionAttributesMutator;
+import org.apache.geode.cache.ExpirationAction;
+import org.apache.geode.cache.ExpirationAttributes;
+import org.apache.geode.cache.configuration.DeclarableType;
+import org.apache.geode.cache.configuration.RegionAttributesType;
+import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.internal.cache.AbstractRegion;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.InternalCacheForClientAccess;
+
+public class RegionAlterFunctionTest {
+
+  private RegionAlterFunction function;
+  private RegionConfig config;
+  private RegionAttributesType regionAttributes;
+  private InternalCache internalCache;
+  private InternalCacheForClientAccess cache;
+  private FunctionContext<RegionConfig> context;
+  private AttributesMutator mutator;
+  private EvictionAttributesMutator evictionMutator;
+  private AbstractRegion region;
+
+  @Before
+  public void setUp() throws Exception {
+    function = spy(RegionAlterFunction.class);
+    config = new RegionConfig();
+    regionAttributes = new RegionAttributesType();
+    config.setRegionAttributes(regionAttributes);
+
+    internalCache = mock(InternalCache.class);
+    cache = mock(InternalCacheForClientAccess.class);
+    mutator = mock(AttributesMutator.class);
+    evictionMutator = mock(EvictionAttributesMutator.class);
+    when(mutator.getEvictionAttributesMutator()).thenReturn(evictionMutator);
+    region = mock(AbstractRegion.class);
+
+    context = mock(FunctionContext.class);
+    when(context.getCache()).thenReturn(internalCache);
+    when(internalCache.getCacheForProcessingClientRequests()).thenReturn(cache);
+    when(context.getArguments()).thenReturn(config);
+    when(context.getMemberName()).thenReturn("member");
+    when(cache.getRegion(any())).thenReturn(region);
+    when(region.getAttributesMutator()).thenReturn(mutator);
+  }
+
+  @Test
+  public void executeFuntcionHappyPathRetunsStatusOK() {
+    doNothing().when(function).alterRegion(any(), any());
+    config.setName("regionA");
+    CliFunctionResult result = function.executeFunction(context);
+    assertThat(result.getMemberIdOrName()).isEqualTo("member");
+    assertThat(result.getStatus()).isEqualTo("OK");
+    assertThat(result.getStatusMessage()).isEqualTo("Region regionA altered");
+  }
+
+  @Test
+  public void alterRegionWithNullRegionThrowsIllegalArgumentException() {
+    when(cache.getRegion(anyString())).thenReturn(null);
+    config.setName("regionA");
+    assertThatThrownBy(() -> function.alterRegion(cache, config))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Region does not exist: regionA");
+  }
+
+  @Test
+  public void updateWithEmptyRegionAttributes() {
+    // the regionAttributes starts with no values inside
+    function.alterRegion(cache, config);
+    verifyZeroInteractions(mutator);
+  }
+
+  @Test
+  public void updateWithCloningEnabled() {
+    regionAttributes.setCloningEnabled(false);
+    function.alterRegion(cache, config);
+    verify(mutator).setCloningEnabled(false);
+  }
+
+  @Test
+  public void updateWithEvictionAttributes() {
+    RegionAttributesType.EvictionAttributes evictionAttributes =
+        new RegionAttributesType.EvictionAttributes();
+    RegionAttributesType.EvictionAttributes.LruEntryCount lruEntryCount =
+        new RegionAttributesType.EvictionAttributes.LruEntryCount();
+    lruEntryCount.setMaximum("10");
+    evictionAttributes.setLruEntryCount(lruEntryCount);
+    regionAttributes.setEvictionAttributes(evictionAttributes);
+
+    function.alterRegion(cache, config);
+    verify(mutator).getEvictionAttributesMutator();
+    verify(evictionMutator).setMaximum(10);
+  }
+
+  @Test
+  public void updateWithEntryIdleTime_timeoutAndAction() {
+    RegionAttributesType.ExpirationAttributesType expiration =
+        new RegionAttributesType.ExpirationAttributesType();
+    regionAttributes.setEntryIdleTime(expiration);
+    expiration.setTimeout("10");
+    expiration.setAction("invalidate");
+
+    ExpirationAttributes existing = new ExpirationAttributes();
+    when(region.getEntryIdleTimeout()).thenReturn(existing);
+
+    function.alterRegion(cache, config);
+
+    ArgumentCaptor<ExpirationAttributes> updatedCaptor =
+        ArgumentCaptor.forClass(ExpirationAttributes.class);
+    verify(mutator).setEntryIdleTimeout(updatedCaptor.capture());
+    assertThat(updatedCaptor.getValue().getTimeout()).isEqualTo(10);
+    assertThat(updatedCaptor.getValue().getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+    verify(mutator, times(0)).setCustomEntryIdleTimeout(any());
+  }
+
+  @Test
+  public void updateWithEntryIdleTime_TimeoutOnly() {
+    RegionAttributesType.ExpirationAttributesType expiration =
+        new RegionAttributesType.ExpirationAttributesType();
+    regionAttributes.setEntryIdleTime(expiration);
+    expiration.setTimeout("10");
+
+    ExpirationAttributes existing = new ExpirationAttributes(20, ExpirationAction.DESTROY);
+    when(region.getEntryIdleTimeout()).thenReturn(existing);
+
+    function.alterRegion(cache, config);
+
+    ArgumentCaptor<ExpirationAttributes> updatedCaptor =
+        ArgumentCaptor.forClass(ExpirationAttributes.class);
+    verify(mutator).setEntryIdleTimeout(updatedCaptor.capture());
+    assertThat(updatedCaptor.getValue().getTimeout()).isEqualTo(10);
+    assertThat(updatedCaptor.getValue().getAction()).isEqualTo(ExpirationAction.DESTROY);
+    verify(mutator, times(0)).setCustomEntryIdleTimeout(any());
+  }
+
+  @Test
+  public void updateWithCustomExpiry() {
+    RegionAttributesType.ExpirationAttributesType expiration =
+        new RegionAttributesType.ExpirationAttributesType();
+    regionAttributes.setEntryIdleTime(expiration);
+    DeclarableType mockExpiry = mock(DeclarableType.class);
+    when(mockExpiry.newInstance(any())).thenReturn(mock(CustomExpiry.class));
+    expiration.setCustomExpiry(mockExpiry);
+
+    function.alterRegion(cache, config);
+
+    verify(mutator, times(0)).setEntryIdleTimeout(any());
+    verify(mutator).setCustomEntryIdleTimeout(notNull());
+  }
+
+  @Test
+  public void deleteCustomExpiry() {
+    RegionAttributesType.ExpirationAttributesType expiration =
+        new RegionAttributesType.ExpirationAttributesType();
+    regionAttributes.setEntryIdleTime(expiration);
+    expiration.setCustomExpiry(DeclarableType.EMPTY);
+
+    function.alterRegion(cache, config);
+
+    verify(mutator, times(0)).setEntryIdleTimeout(any());
+    verify(mutator).setCustomEntryIdleTimeout(null);
+  }
+
+  @Test
+  public void updateWithGatewaySenders() {
+    regionAttributes.setGatewaySenderIds("2,3");
+    when(region.getGatewaySenderIds()).thenReturn(new HashSet<>(Arrays.asList("1", "2")));
+
+    function.alterRegion(cache, config);
+
+    verify(mutator).removeGatewaySenderId("1");
+    verify(mutator, times(0)).removeGatewaySenderId("2");
+    verify(mutator).addGatewaySenderId("3");
+
+    // asyncEventQueue is left intact
+    verify(mutator, times(0)).addAsyncEventQueueId(any());
+    verify(mutator, times(0)).removeAsyncEventQueueId(any());
+  }
+
+  @Test
+  public void updateWithEmptyGatewaySenders() {
+    regionAttributes.setGatewaySenderIds("");
+    when(region.getGatewaySenderIds()).thenReturn(new HashSet<>(Arrays.asList("1", "2")));
+
+    function.alterRegion(cache, config);
+
+    verify(mutator).removeGatewaySenderId("1");
+    verify(mutator).removeGatewaySenderId("2");
+  }
+
+  @Test
+  public void updateWithCacheListeners() {
+    // suppose region has one cacheListener, and we want to replace the oldOne one with the new one
+    CacheListener oldOne = mock(CacheListener.class);
+    CacheListener newOne = mock(CacheListener.class);
+    when(region.getCacheListeners()).thenReturn(new CacheListener[] {oldOne});
+
+    DeclarableType newCacheListenerType = mock(DeclarableType.class);
+    when(newCacheListenerType.newInstance(any())).thenReturn(newOne);
+    regionAttributes.getCacheListeners().add(newCacheListenerType);
+
+    function.alterRegion(cache, config);
+    verify(mutator).removeCacheListener(oldOne);
+    verify(mutator).addCacheListener(newOne);
+  }
+
+  @Test
+  public void updateWithEmptyCacheListeners() {
+    // suppose region has on listener, and we want to delete that one
+    CacheListener oldOne = mock(CacheListener.class);
+    when(region.getCacheListeners()).thenReturn(new CacheListener[] {oldOne});
+    regionAttributes.getCacheListeners().add(DeclarableType.EMPTY);
+
+    function.alterRegion(cache, config);
+    verify(mutator).removeCacheListener(oldOne);
+    verify(mutator, times(0)).addCacheListener(any());
+  }
+
+  @Test
+  public void updateWithCacheWriter() {
+    DeclarableType newCacheWriterDeclarable = mock(DeclarableType.class);
+    when(newCacheWriterDeclarable.newInstance(any())).thenReturn(mock(CacheWriter.class));
+    regionAttributes.setCacheWriter(newCacheWriterDeclarable);
+
+    function.alterRegion(cache, config);
+    verify(mutator).setCacheWriter(notNull());
+    verify(mutator, times(0)).setCacheLoader(any());
+  }
+
+  @Test
+  public void updateWithNoCacheWriter() {
+    regionAttributes.setCacheWriter(DeclarableType.EMPTY);
+
+    function.alterRegion(cache, config);
+    verify(mutator).setCacheWriter(null);
+    verify(mutator, times(0)).setCacheLoader(any());
+  }
+}
diff --git a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
index e05b15a..b18d975 100644
--- a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
+++ b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
@@ -206,7 +206,7 @@ public class LuceneClusterConfigurationDUnitTest {
   }
 
   @Test
-  public void verifyMemberWithGroupStartsAfterAlterRegion() throws Exception {
+  public void verifyMemberWithGroupStarts() throws Exception {
     // Start a member with no group
     createServer(1);
 
@@ -214,7 +214,7 @@ public class LuceneClusterConfigurationDUnitTest {
     String group = "group1";
     Properties properties = new Properties();
     properties.setProperty(GROUPS, group);
-    MemberVM vm2 = createServer(properties, 2);
+    createServer(properties, 2);
 
     // Connect Gfsh to locator
     gfshConnector.connectAndVerify(locator);
@@ -222,22 +222,11 @@ public class LuceneClusterConfigurationDUnitTest {
     // Create index and region in no group
     createLuceneIndexAndDataRegion();
 
-    // Alter region in group
-    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.ALTER_REGION);
-    csb.addOption(CliStrings.ALTER_REGION__REGION, REGION_NAME);
-    csb.addOption(CliStrings.GROUP, group);
-    csb.addOption(CliStrings.ALTER_REGION__EVICTIONMAX, "5764");
-
-    String expectedStatusOutput =
-        "Region \"/" + REGION_NAME + "\" altered on \"" + vm2.getName() + "\"";
-    gfshConnector.executeAndAssertThat(csb.toString()).statusIsSuccess()
-        .tableHasColumnWithExactValuesInExactOrder("Status", expectedStatusOutput);
-
     // Start another member with group
     createServer(properties, 3);
 
     // Verify all members have indexes
-    csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
     gfshConnector.executeAndAssertThat(csb.toString()).statusIsSuccess()
         .tableHasColumnWithExactValuesInExactOrder("Status", "INITIALIZED", "INITIALIZED",
             "INITIALIZED");