You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2017/10/24 13:47:39 UTC

[geode] branch develop updated: GEODE-1897: Initial refactor of CreateRegionCommand (#956)

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

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


The following commit(s) were added to refs/heads/develop by this push:
     new 7a7c2ef  GEODE-1897: Initial refactor of CreateRegionCommand (#956)
7a7c2ef is described below

commit 7a7c2ef7d546f278fc99aa700d7e456b8d86cf24
Author: Jens Deppe <jd...@pivotal.io>
AuthorDate: Tue Oct 24 06:47:36 2017 -0700

    GEODE-1897: Initial refactor of CreateRegionCommand (#956)
    
    * GEODE-1897: Initial refactor of CreateRegionCommand
    
    * GEODE-1897: Add test categories
    
    * GEODE-1897: Add license headers
    
    * GEODE-1897: Add missing test category
    
    * GEODE-1897: Update miscellaneous test failures
    
    - Also updated sanctionedSerializables
    
    * GEODE-1897: Review updates
    
    * GEODE-1897: More review changes
    
    * GEODE-1897: Integrate better with new develop functionality
    
    * GEODE-1897: Impromptu review update
    
    * GEODE-1897: Spotless how do I love thee... let me count the ways
    
    * GEODE-1897: fixing precheckin failures
---
 .../org/apache/geode/cache/AttributesFactory.java  |   7 +-
 .../apache/geode/cache/ExpirationAttributes.java   |  18 +-
 .../internal/cli/commands/AlterRegionCommand.java  | 192 ++----
 .../internal/cli/commands/CreateRegionCommand.java | 699 ++++++++++-----------
 .../internal/cli/commands/GfshCommand.java         |   7 +
 .../functions/FetchRegionAttributesFunction.java   |  69 +-
 .../cli/functions/RegionCreateFunction.java        | 117 +---
 .../internal/cli/functions/RegionFunctionArgs.java | 585 +++++++----------
 .../internal/cli/remote/CommandExecutor.java       |   4 +-
 .../geode/redis/internal/RegionProvider.java       |   7 +-
 .../geode/cache/ExpirationAttributesTest.java      |  49 ++
 .../cli/commands/AlterRegionCommandDUnitTest.java  |  14 +-
 .../AlterRegionCommandIntegrationTest.java         |  71 +++
 .../cli/commands/CreateRegionCommandDUnitTest.java |   2 +-
 .../CreateRegionCommandIntegrationTest.java        | 493 +++++++++++++++
 .../cli/commands/CreateRegionCommandTest.java      | 200 +++++-
 .../commands/DestroyRegionCommandDUnitTest.java    |   2 +-
 .../internal/cli/commands/TestCacheLoader.java     |  30 +
 .../internal/cli/commands/TestCacheWriter.java     |  41 ++
 .../internal/cli/commands/TestCompressor.java      |  30 +
 .../cli/commands/TestPartitionResolver.java        |  36 ++
 .../cli/functions/ExpirationAttrsTest.java         |  51 ++
 .../cli/functions/RegionFunctionArgsTest.java      |  59 ++
 .../test/junit/rules/GfshShellConnectionRule.java  |   4 +-
 .../geode/codeAnalysis/sanctionedSerializables.txt |  12 +-
 25 files changed, 1774 insertions(+), 1025 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.java b/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.java
index 0d59279..09add5e 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.java
@@ -331,6 +331,11 @@ public class AttributesFactory<K, V> {
   public static final boolean DEFAULT_DISK_SYNCHRONOUS = true;
 
   /**
+   * The default concurrency level for updates to region values
+   */
+  public static final int DEFAULT_CONCURRENCY_LEVEL = 16;
+
+  /**
    * Creates a new instance of AttributesFactory ready to create a {@code RegionAttributes} with
    * default settings.
    */
@@ -1559,7 +1564,7 @@ public class AttributesFactory<K, V> {
     Class<V> valueConstraint = null;
     int initialCapacity = 16;
     float loadFactor = 0.75f;
-    int concurrencyLevel = 16;
+    int concurrencyLevel = DEFAULT_CONCURRENCY_LEVEL;
     boolean concurrencyChecksEnabled = true;
     boolean earlyAck = false;
     boolean publisher = false;
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 5f15210..cdd1447 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
@@ -73,14 +73,22 @@ public class ExpirationAttributes implements DataSerializable {
   /**
    * Constructs an <code>ExpirationAttributes</code> with the specified expiration time and
    * expiration action.
-   * 
-   * @param expirationTime The number of seconds for a value to live before it expires
+   *
+   * @param expirationTime The number of seconds for a value to live before it expires. If this
+   *        parameter is negative, the expiration time will be set to 0, indicating no expiration.
    * @param expirationAction the action to take when the value expires
-   * @throws IllegalArgumentException if expirationTime is nonpositive
    */
   public ExpirationAttributes(int expirationTime, ExpirationAction expirationAction) {
-    this.timeout = expirationTime;
-    this.action = expirationAction;
+    if (expirationTime < 0) {
+      this.timeout = 0;
+    } else {
+      this.timeout = expirationTime;
+    }
+    if (expirationAction == null) {
+      this.action = ExpirationAction.INVALIDATE;
+    } else {
+      this.action = expirationAction;
+    }
   }
 
 
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 3ddc476..070fdd8 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
@@ -18,11 +18,9 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.commons.lang.StringUtils;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import org.apache.geode.cache.ExpirationAttributes;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.InternalCache;
@@ -30,7 +28,6 @@ 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.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.LogWrapper;
 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;
@@ -51,24 +48,21 @@ public class AlterRegionCommand implements GfshCommand {
           optionContext = ConverterHint.MEMBERGROUP,
           help = CliStrings.ALTER_REGION__GROUP__HELP) String[] groups,
       @CliOption(key = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIME,
-          specifiedDefaultValue = "-1",
           help = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIME__HELP) Integer entryExpirationIdleTime,
       @CliOption(key = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIMEACTION,
           specifiedDefaultValue = "INVALIDATE",
           help = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIMEACTION__HELP) String entryExpirationIdleTimeAction,
       @CliOption(key = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTIMETOLIVE,
-          specifiedDefaultValue = "-1",
           help = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTIMETOLIVE__HELP) Integer entryExpirationTTL,
       @CliOption(key = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION,
           specifiedDefaultValue = "INVALIDATE",
           help = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION__HELP) String entryExpirationTTLAction,
       @CliOption(key = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIME,
-          specifiedDefaultValue = "-1",
           help = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIME__HELP) Integer regionExpirationIdleTime,
       @CliOption(key = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIMEACTION,
           specifiedDefaultValue = "INVALIDATE",
           help = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIMEACTION__HELP) String regionExpirationIdleTimeAction,
-      @CliOption(key = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTL, specifiedDefaultValue = "-1",
+      @CliOption(key = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTL,
           help = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTL__HELP) Integer regionExpirationTTL,
       @CliOption(key = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTLACTION,
           specifiedDefaultValue = "INVALIDATE",
@@ -83,8 +77,9 @@ public class AlterRegionCommand implements GfshCommand {
           help = CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID__HELP) String[] asyncEventQueueIds,
       @CliOption(key = CliStrings.ALTER_REGION__GATEWAYSENDERID, specifiedDefaultValue = "",
           help = CliStrings.ALTER_REGION__GATEWAYSENDERID__HELP) String[] gatewaySenderIds,
-      @CliOption(key = CliStrings.ALTER_REGION__CLONINGENABLED, specifiedDefaultValue = "false",
-          help = CliStrings.ALTER_REGION__CLONINGENABLED__HELP) Boolean cloningEnabled,
+      @CliOption(key = CliStrings.ALTER_REGION__CLONINGENABLED, specifiedDefaultValue = "true",
+          unspecifiedDefaultValue = "false",
+          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;
@@ -92,132 +87,83 @@ public class AlterRegionCommand implements GfshCommand {
 
     getSecurityService().authorize(Resource.DATA, Operation.MANAGE, regionPath);
 
-    try {
-      InternalCache cache = getCache();
+    InternalCache cache = getCache();
 
-      if (groups != null) {
-        RegionCommandsUtils.validateGroups(cache, groups);
-      }
-
-      RegionFunctionArgs.ExpirationAttrs entryIdle = null;
-      if (entryExpirationIdleTime != null || entryExpirationIdleTimeAction != null) {
-        if (entryExpirationIdleTime != null && entryExpirationIdleTime == -1) {
-          entryExpirationIdleTime = ExpirationAttributes.DEFAULT.getTimeout();
-        }
-        if (CliMetaData.ANNOTATION_DEFAULT_VALUE.equals(entryExpirationIdleTimeAction)) {
-          entryExpirationIdleTimeAction = ExpirationAttributes.DEFAULT.getAction().toString();
-        }
-        entryIdle = new RegionFunctionArgs.ExpirationAttrs(
-            RegionFunctionArgs.ExpirationAttrs.ExpirationFor.ENTRY_IDLE, entryExpirationIdleTime,
-            entryExpirationIdleTimeAction);
-      }
-      RegionFunctionArgs.ExpirationAttrs entryTTL = null;
-      if (entryExpirationTTL != null || entryExpirationTTLAction != null) {
-        if (entryExpirationTTL != null && entryExpirationTTL == -1) {
-          entryExpirationTTL = ExpirationAttributes.DEFAULT.getTimeout();
-        }
-        if (CliMetaData.ANNOTATION_DEFAULT_VALUE.equals(entryExpirationTTLAction)) {
-          entryExpirationTTLAction = ExpirationAttributes.DEFAULT.getAction().toString();
-        }
-        entryTTL = new RegionFunctionArgs.ExpirationAttrs(
-            RegionFunctionArgs.ExpirationAttrs.ExpirationFor.ENTRY_TTL, entryExpirationTTL,
-            entryExpirationTTLAction);
-      }
-      RegionFunctionArgs.ExpirationAttrs regionIdle = null;
-      if (regionExpirationIdleTime != null || regionExpirationIdleTimeAction != null) {
-        if (regionExpirationIdleTime != null && regionExpirationIdleTime == -1) {
-          regionExpirationIdleTime = ExpirationAttributes.DEFAULT.getTimeout();
-        }
-        if (CliMetaData.ANNOTATION_DEFAULT_VALUE.equals(regionExpirationIdleTimeAction)) {
-          regionExpirationIdleTimeAction = ExpirationAttributes.DEFAULT.getAction().toString();
-        }
-        regionIdle = new RegionFunctionArgs.ExpirationAttrs(
-            RegionFunctionArgs.ExpirationAttrs.ExpirationFor.REGION_IDLE, regionExpirationIdleTime,
-            regionExpirationIdleTimeAction);
-      }
-      RegionFunctionArgs.ExpirationAttrs regionTTL = null;
-      if (regionExpirationTTL != null || regionExpirationTTLAction != null) {
-        if (regionExpirationTTL != null && regionExpirationTTL == -1) {
-          regionExpirationTTL = ExpirationAttributes.DEFAULT.getTimeout();
-        }
-        if (CliMetaData.ANNOTATION_DEFAULT_VALUE.equals(regionExpirationTTLAction)) {
-          regionExpirationTTLAction = ExpirationAttributes.DEFAULT.getAction().toString();
-        }
-        regionTTL = new RegionFunctionArgs.ExpirationAttrs(
-            RegionFunctionArgs.ExpirationAttrs.ExpirationFor.REGION_TTL, regionExpirationTTL,
-            regionExpirationTTLAction);
-      }
+    if (groups != null) {
+      RegionCommandsUtils.validateGroups(cache, groups);
+    }
 
-      cacheLoader = convertDefaultValue(cacheLoader, StringUtils.EMPTY);
-      cacheWriter = convertDefaultValue(cacheWriter, StringUtils.EMPTY);
-
-      RegionFunctionArgs regionFunctionArgs;
-      regionFunctionArgs = new RegionFunctionArgs(regionPath, null, null, false, null, null, null,
-          entryIdle, entryTTL, regionIdle, regionTTL, null, null, null, null, cacheListeners,
-          cacheLoader, cacheWriter, asyncEventQueueIds, gatewaySenderIds, null, cloningEnabled,
-          null, null, null, null, null, null, null, null, evictionMax, null, null, null, null);
-
-      Set<String> cacheListenersSet = regionFunctionArgs.getCacheListeners();
-      if (cacheListenersSet != null && !cacheListenersSet.isEmpty()) {
-        for (String cacheListener : cacheListenersSet) {
-          if (!RegionCommandsUtils.isClassNameValid(cacheListener)) {
-            throw new IllegalArgumentException(CliStrings.format(
-                CliStrings.ALTER_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHELISTENER_0_IS_INVALID,
-                new Object[] {cacheListener}));
-          }
+    RegionFunctionArgs regionFunctionArgs = new RegionFunctionArgs();
+    regionFunctionArgs.setRegionPath(regionPath);
+    regionFunctionArgs.setEntryExpirationIdleTime(entryExpirationIdleTime,
+        entryExpirationIdleTimeAction);
+    regionFunctionArgs.setEntryExpirationTTL(entryExpirationTTL, entryExpirationTTLAction);
+    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<String> cacheListenersSet = regionFunctionArgs.getCacheListeners();
+    if (cacheListenersSet != null && !cacheListenersSet.isEmpty()) {
+      for (String cacheListener : cacheListenersSet) {
+        if (!RegionCommandsUtils.isClassNameValid(cacheListener)) {
+          throw new IllegalArgumentException(CliStrings.format(
+              CliStrings.ALTER_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHELISTENER_0_IS_INVALID,
+              new Object[] {cacheListener}));
         }
       }
+    }
 
-      if (cacheLoader != null && !RegionCommandsUtils.isClassNameValid(cacheLoader)) {
-        throw new IllegalArgumentException(CliStrings.format(
-            CliStrings.ALTER_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHELOADER_0_IS_INVALID,
-            new Object[] {cacheLoader}));
-      }
+    if (cacheLoader != null && !RegionCommandsUtils.isClassNameValid(cacheLoader)) {
+      throw new IllegalArgumentException(CliStrings.format(
+          CliStrings.ALTER_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHELOADER_0_IS_INVALID,
+          new Object[] {cacheLoader}));
+    }
 
-      if (cacheWriter != null && !RegionCommandsUtils.isClassNameValid(cacheWriter)) {
-        throw new IllegalArgumentException(CliStrings.format(
-            CliStrings.ALTER_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHEWRITER_0_IS_INVALID,
-            new Object[] {cacheWriter}));
-      }
+    if (cacheWriter != null && !RegionCommandsUtils.isClassNameValid(cacheWriter)) {
+      throw new IllegalArgumentException(CliStrings.format(
+          CliStrings.ALTER_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHEWRITER_0_IS_INVALID,
+          new Object[] {cacheWriter}));
+    }
 
-      if (evictionMax != null && evictionMax < 0) {
-        throw new IllegalArgumentException(CliStrings.format(
-            CliStrings.ALTER_REGION__MSG__SPECIFY_POSITIVE_INT_FOR_EVICTIONMAX_0_IS_NOT_VALID,
-            new Object[] {evictionMax}));
-      }
+    if (evictionMax != null && evictionMax < 0) {
+      throw new IllegalArgumentException(CliStrings.format(
+          CliStrings.ALTER_REGION__MSG__SPECIFY_POSITIVE_INT_FOR_EVICTIONMAX_0_IS_NOT_VALID,
+          new Object[] {evictionMax}));
+    }
 
-      Set<DistributedMember> targetMembers = CliUtil.findMembers(groups, null);
+    Set<DistributedMember> targetMembers = CliUtil.findMembers(groups, null);
 
-      if (targetMembers.isEmpty()) {
-        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
-      }
+    if (targetMembers.isEmpty()) {
+      return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+    }
 
-      ResultCollector<?, ?> resultCollector =
-          CliUtil.executeFunction(new RegionAlterFunction(), regionFunctionArgs, targetMembers);
-      List<CliFunctionResult> regionAlterResults =
-          (List<CliFunctionResult>) resultCollector.getResult();
-
-      TabularResultData tabularResultData = ResultBuilder.createTabularResultData();
-      final String errorPrefix = "ERROR: ";
-      for (CliFunctionResult regionAlterResult : regionAlterResults) {
-        boolean success = regionAlterResult.isSuccessful();
-        tabularResultData.accumulate("Member", regionAlterResult.getMemberIdOrName());
-        if (success) {
-          tabularResultData.accumulate("Status", regionAlterResult.getMessage());
-          xmlEntity.set(regionAlterResult.getXmlEntity());
-        } else {
-          tabularResultData.accumulate("Status", errorPrefix + regionAlterResult.getMessage());
-          tabularResultData.setStatus(Result.Status.ERROR);
-        }
+    ResultCollector<?, ?> resultCollector =
+        CliUtil.executeFunction(new RegionAlterFunction(), regionFunctionArgs, targetMembers);
+    List<CliFunctionResult> regionAlterResults =
+        (List<CliFunctionResult>) resultCollector.getResult();
+
+    TabularResultData tabularResultData = ResultBuilder.createTabularResultData();
+    final String errorPrefix = "ERROR: ";
+    for (CliFunctionResult regionAlterResult : regionAlterResults) {
+      boolean success = regionAlterResult.isSuccessful();
+      tabularResultData.accumulate("Member", regionAlterResult.getMemberIdOrName());
+      if (success) {
+        tabularResultData.accumulate("Status", regionAlterResult.getMessage());
+        xmlEntity.set(regionAlterResult.getXmlEntity());
+      } else {
+        tabularResultData.accumulate("Status", errorPrefix + regionAlterResult.getMessage());
+        tabularResultData.setStatus(Result.Status.ERROR);
       }
-      result = ResultBuilder.buildResult(tabularResultData);
-    } catch (IllegalArgumentException | IllegalStateException e) {
-      LogWrapper.getInstance().info(e.getMessage());
-      result = ResultBuilder.createUserErrorResult(e.getMessage());
-    } catch (RuntimeException e) {
-      LogWrapper.getInstance().info(e.getMessage(), e);
-      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
     }
+    result = ResultBuilder.buildResult(tabularResultData);
 
     if (xmlEntity.get() != null) {
       persistClusterConfiguration(result,
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommand.java
index 99c5d0b..58a6f97 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommand.java
@@ -14,17 +14,15 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
 
 import javax.management.ObjectName;
 
-import org.apache.commons.lang.StringUtils;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
@@ -33,11 +31,7 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.compression.Compressor;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.DistributionManager;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.management.DistributedRegionMXBean;
@@ -48,7 +42,9 @@ import org.apache.geode.management.RegionMXBean;
 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.internal.cli.AbstractCliAroundInterceptor;
 import org.apache.geode.management.internal.cli.CliUtil;
+import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.LogWrapper;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.cli.functions.FetchRegionAttributesFunction;
@@ -65,22 +61,24 @@ import org.apache.geode.security.ResourcePermission;
 public class CreateRegionCommand implements GfshCommand {
 
   @CliCommand(value = CliStrings.CREATE_REGION, help = CliStrings.CREATE_REGION__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION,
+      interceptor = "org.apache.geode.management.internal.cli.commands.CreateRegionCommand$Interceptor")
   @ResourceOperation(resource = ResourcePermission.Resource.DATA,
       operation = ResourcePermission.Operation.MANAGE)
   public Result createRegion(
       @CliOption(key = CliStrings.CREATE_REGION__REGION, mandatory = true,
+          optionContext = ConverterHint.REGION_PATH,
           help = CliStrings.CREATE_REGION__REGION__HELP) String regionPath,
       @CliOption(key = CliStrings.CREATE_REGION__REGIONSHORTCUT,
           help = CliStrings.CREATE_REGION__REGIONSHORTCUT__HELP) RegionShortcut regionShortcut,
       @CliOption(key = CliStrings.CREATE_REGION__USEATTRIBUTESFROM,
           optionContext = ConverterHint.REGION_PATH,
-          help = CliStrings.CREATE_REGION__USEATTRIBUTESFROM__HELP) String useAttributesFrom,
+          help = CliStrings.CREATE_REGION__USEATTRIBUTESFROM__HELP) String templateRegion,
       @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
           optionContext = ConverterHint.MEMBERGROUP,
           help = CliStrings.CREATE_REGION__GROUP__HELP) String[] groups,
-      @CliOption(key = CliStrings.CREATE_REGION__SKIPIFEXISTS, unspecifiedDefaultValue = "true",
-          specifiedDefaultValue = "true",
+      @CliOption(key = CliStrings.CREATE_REGION__SKIPIFEXISTS, specifiedDefaultValue = "true",
+          unspecifiedDefaultValue = "false",
           help = CliStrings.CREATE_REGION__SKIPIFEXISTS__HELP) boolean skipIfExists,
 
       // the following should all be in alphabetical order according to
@@ -103,18 +101,21 @@ public class CreateRegionCommand implements GfshCommand {
       @CliOption(key = CliStrings.CREATE_REGION__DISKSTORE,
           help = CliStrings.CREATE_REGION__DISKSTORE__HELP) String diskStore,
       @CliOption(key = CliStrings.CREATE_REGION__ENABLEASYNCCONFLATION,
+          specifiedDefaultValue = "true",
           help = CliStrings.CREATE_REGION__ENABLEASYNCCONFLATION__HELP) Boolean enableAsyncConflation,
-      @CliOption(key = CliStrings.CREATE_REGION__CLONINGENABLED,
+      @CliOption(key = CliStrings.CREATE_REGION__CLONINGENABLED, specifiedDefaultValue = "true",
           help = CliStrings.CREATE_REGION__CLONINGENABLED__HELP) Boolean cloningEnabled,
       @CliOption(key = CliStrings.CREATE_REGION__CONCURRENCYCHECKSENABLED,
+          specifiedDefaultValue = "true",
           help = CliStrings.CREATE_REGION__CONCURRENCYCHECKSENABLED__HELP) Boolean concurrencyChecksEnabled,
-      @CliOption(key = CliStrings.CREATE_REGION__MULTICASTENABLED,
+      @CliOption(key = CliStrings.CREATE_REGION__MULTICASTENABLED, specifiedDefaultValue = "true",
           help = CliStrings.CREATE_REGION__MULTICASTENABLED__HELP) Boolean mcastEnabled,
-      @CliOption(key = CliStrings.CREATE_REGION__STATISTICSENABLED,
+      @CliOption(key = CliStrings.CREATE_REGION__STATISTICSENABLED, specifiedDefaultValue = "true",
           help = CliStrings.CREATE_REGION__STATISTICSENABLED__HELP) Boolean statisticsEnabled,
       @CliOption(key = CliStrings.CREATE_REGION__ENABLESUBSCRIPTIONCONFLATION,
+          specifiedDefaultValue = "true",
           help = CliStrings.CREATE_REGION__ENABLESUBSCRIPTIONCONFLATION__HELP) Boolean enableSubscriptionConflation,
-      @CliOption(key = CliStrings.CREATE_REGION__DISKSYNCHRONOUS,
+      @CliOption(key = CliStrings.CREATE_REGION__DISKSYNCHRONOUS, specifiedDefaultValue = "true",
           help = CliStrings.CREATE_REGION__DISKSYNCHRONOUS__HELP) Boolean diskSynchronous,
       @CliOption(key = CliStrings.CREATE_REGION__ENTRYEXPIRATIONIDLETIME,
           help = CliStrings.CREATE_REGION__ENTRYEXPIRATIONIDLETIME__HELP) Integer entryExpirationIdleTime,
@@ -159,411 +160,243 @@ public class CreateRegionCommand implements GfshCommand {
     Result result;
     AtomicReference<XmlEntity> xmlEntity = new AtomicReference<>();
 
-    try {
-      InternalCache cache = getCache();
-
-      if (regionShortcut != null && useAttributesFrom != null) {
-        throw new IllegalArgumentException(
-            CliStrings.CREATE_REGION__MSG__ONLY_ONE_OF_REGIONSHORTCUT_AND_USEATTRIBUESFROM_CAN_BE_SPECIFIED);
-      } else if (regionShortcut == null && useAttributesFrom == null) {
-        throw new IllegalArgumentException(
-            CliStrings.CREATE_REGION__MSG__ONE_OF_REGIONSHORTCUT_AND_USEATTRIBUTESFROM_IS_REQUIRED);
-      }
-
-      validateRegionPathAndParent(cache, regionPath);
-      RegionCommandsUtils.validateGroups(cache, groups);
+    if (regionShortcut != null && templateRegion != null) {
+      return ResultBuilder.createUserErrorResult(
+          CliStrings.CREATE_REGION__MSG__ONLY_ONE_OF_REGIONSHORTCUT_AND_USEATTRIBUESFROM_CAN_BE_SPECIFIED);
+    }
 
-      RegionFunctionArgs.ExpirationAttrs entryIdle = null;
-      if (entryExpirationIdleTime != null) {
-        entryIdle = new RegionFunctionArgs.ExpirationAttrs(
-            RegionFunctionArgs.ExpirationAttrs.ExpirationFor.ENTRY_IDLE, entryExpirationIdleTime,
-            entryExpirationIdleTimeAction);
-      }
-      RegionFunctionArgs.ExpirationAttrs entryTTL = null;
-      if (entryExpirationTTL != null) {
-        entryTTL = new RegionFunctionArgs.ExpirationAttrs(
-            RegionFunctionArgs.ExpirationAttrs.ExpirationFor.ENTRY_TTL, entryExpirationTTL,
-            entryExpirationTTLAction);
-      }
-      RegionFunctionArgs.ExpirationAttrs regionIdle = null;
-      if (regionExpirationIdleTime != null) {
-        regionIdle = new RegionFunctionArgs.ExpirationAttrs(
-            RegionFunctionArgs.ExpirationAttrs.ExpirationFor.REGION_IDLE, regionExpirationIdleTime,
-            regionExpirationIdleTimeAction);
-      }
-      RegionFunctionArgs.ExpirationAttrs regionTTL = null;
-      if (regionExpirationTTL != null) {
-        regionTTL = new RegionFunctionArgs.ExpirationAttrs(
-            RegionFunctionArgs.ExpirationAttrs.ExpirationFor.REGION_TTL, regionExpirationTTL,
-            regionExpirationTTLAction);
-      }
+    if (regionShortcut == null && templateRegion == null) {
+      return ResultBuilder.createUserErrorResult(
+          CliStrings.CREATE_REGION__MSG__ONE_OF_REGIONSHORTCUT_AND_USEATTRIBUTESFROM_IS_REQUIRED);
+    }
 
-      RegionFunctionArgs regionFunctionArgs;
-      if (useAttributesFrom != null) {
-        if (!regionExists(cache, useAttributesFrom)) {
-          throw new IllegalArgumentException(CliStrings.format(
-              CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH_FOR_0_REGIONPATH_1_NOT_FOUND,
-              CliStrings.CREATE_REGION__USEATTRIBUTESFROM, useAttributesFrom));
-        }
+    InternalCache cache = getCache();
 
-        FetchRegionAttributesFunction.FetchRegionAttributesFunctionResult<Object, Object> regionAttributesResult =
-            getRegionAttributes(cache, useAttributesFrom);
-        RegionAttributes<?, ?> regionAttributes = regionAttributesResult.getRegionAttributes();
-
-        // give preference to user specified plugins than the ones retrieved from other region
-        String[] cacheListenerClasses = cacheListener != null && cacheListener.length != 0
-            ? cacheListener : regionAttributesResult.getCacheListenerClasses();
-        String cacheLoaderClass =
-            cacheLoader != null ? cacheLoader : regionAttributesResult.getCacheLoaderClass();
-        String cacheWriterClass =
-            cacheWriter != null ? cacheWriter : regionAttributesResult.getCacheWriterClass();
-
-        regionFunctionArgs = new RegionFunctionArgs(regionPath, useAttributesFrom, skipIfExists,
-            keyConstraint, valueConstraint, statisticsEnabled, entryIdle, entryTTL, regionIdle,
-            regionTTL, diskStore, diskSynchronous, enableAsyncConflation,
-            enableSubscriptionConflation, cacheListenerClasses, cacheLoaderClass, cacheWriterClass,
-            asyncEventQueueIds, gatewaySenderIds, concurrencyChecksEnabled, cloningEnabled,
-            concurrencyLevel, prColocatedWith, prLocalMaxMemory, prRecoveryDelay, prRedundantCopies,
-            prStartupRecoveryDelay, prTotalMaxMemory, prTotalNumBuckets, offHeap, mcastEnabled,
-            regionAttributes, partitionResolver);
-
-        if (regionAttributes.getPartitionAttributes() == null
-            && regionFunctionArgs.hasPartitionAttributes()) {
-          throw new IllegalArgumentException(CliStrings.format(
-              CliStrings.CREATE_REGION__MSG__OPTION_0_CAN_BE_USED_ONLY_FOR_PARTITIONEDREGION,
-              regionFunctionArgs.getPartitionArgs().getUserSpecifiedPartitionAttributes()) + " "
-              + CliStrings.format(CliStrings.CREATE_REGION__MSG__0_IS_NOT_A_PARITIONEDREGION,
-                  useAttributesFrom));
-        }
-      } else {
-        regionFunctionArgs = new RegionFunctionArgs(regionPath, regionShortcut, useAttributesFrom,
-            skipIfExists, keyConstraint, valueConstraint, statisticsEnabled, entryIdle, entryTTL,
-            regionIdle, regionTTL, diskStore, diskSynchronous, enableAsyncConflation,
-            enableSubscriptionConflation, cacheListener, cacheLoader, cacheWriter,
-            asyncEventQueueIds, gatewaySenderIds, concurrencyChecksEnabled, cloningEnabled,
-            concurrencyLevel, prColocatedWith, prLocalMaxMemory, prRecoveryDelay, prRedundantCopies,
-            prStartupRecoveryDelay, prTotalMaxMemory, prTotalNumBuckets, null, compressor, offHeap,
-            mcastEnabled, partitionResolver);
-
-        if (!regionShortcut.name().startsWith("PARTITION")
-            && regionFunctionArgs.hasPartitionAttributes()) {
-          throw new IllegalArgumentException(CliStrings.format(
-              CliStrings.CREATE_REGION__MSG__OPTION_0_CAN_BE_USED_ONLY_FOR_PARTITIONEDREGION,
-              regionFunctionArgs.getPartitionArgs().getUserSpecifiedPartitionAttributes()) + " "
-              + CliStrings.format(CliStrings.CREATE_REGION__MSG__0_IS_NOT_A_PARITIONEDREGION,
-                  regionPath));
-        }
+    // validating the region path
+    RegionPath regionPathData = new RegionPath(regionPath);
+    String parentRegionPath = regionPathData.getParent();
+    if (parentRegionPath != null && !Region.SEPARATOR.equals(parentRegionPath)) {
+      if (!regionExists(cache, parentRegionPath)) {
+        return ResultBuilder.createUserErrorResult(
+            CliStrings.format(CliStrings.CREATE_REGION__MSG__PARENT_REGION_FOR_0_DOES_NOT_EXIST,
+                new Object[] {regionPath}));
       }
+    }
 
-      // Do we prefer to validate or authorize first?
-      validateRegionFunctionArgs(cache, regionFunctionArgs);
-      if (isPersistentShortcut(regionFunctionArgs.getRegionShortcut())
-          || isAttributePersistent(regionFunctionArgs.getRegionAttributes())) {
-        getSecurityService().authorize(ResourcePermission.Resource.CLUSTER,
-            ResourcePermission.Operation.WRITE, ResourcePermission.Target.DISK);
+    // creating the RegionFunctionArgs
+    RegionFunctionArgs functionArgs = new RegionFunctionArgs();
+    functionArgs.setRegionPath(regionPath);
+    functionArgs.setSkipIfExists(skipIfExists);
+    functionArgs.setKeyConstraint(keyConstraint);
+    functionArgs.setValueConstraint(valueConstraint);
+    functionArgs.setStatisticsEnabled(statisticsEnabled);
+    functionArgs.setEntryExpirationIdleTime(entryExpirationIdleTime, entryExpirationIdleTimeAction);
+    functionArgs.setEntryExpirationTTL(entryExpirationTTL, entryExpirationTTLAction);
+    functionArgs.setRegionExpirationIdleTime(regionExpirationIdleTime,
+        regionExpirationIdleTimeAction);
+    functionArgs.setRegionExpirationTTL(regionExpirationTTL, regionExpirationTTLAction);
+    functionArgs.setDiskStore(diskStore);
+    functionArgs.setDiskSynchronous(diskSynchronous);
+    functionArgs.setEnableAsyncConflation(enableAsyncConflation);
+    functionArgs.setEnableSubscriptionConflation(enableSubscriptionConflation);
+    functionArgs.setAsyncEventQueueIds(asyncEventQueueIds);
+    functionArgs.setGatewaySenderIds(gatewaySenderIds);
+    functionArgs.setConcurrencyChecksEnabled(concurrencyChecksEnabled);
+    functionArgs.setCloningEnabled(cloningEnabled);
+    functionArgs.setConcurrencyLevel(concurrencyLevel);
+    functionArgs.setPartitionArgs(prColocatedWith, prLocalMaxMemory, prRecoveryDelay,
+        prRedundantCopies, prStartupRecoveryDelay, prTotalMaxMemory, prTotalNumBuckets,
+        partitionResolver);
+    functionArgs.setOffHeap(offHeap);
+    functionArgs.setMcastEnabled(mcastEnabled);
+
+    RegionAttributes<?, ?> regionAttributes = null;
+    if (regionShortcut != null) {
+      if (!regionShortcut.name().startsWith("PARTITION") && functionArgs.hasPartitionAttributes()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.format(
+            CliStrings.CREATE_REGION__MSG__OPTION_0_CAN_BE_USED_ONLY_FOR_PARTITIONEDREGION,
+            functionArgs.getPartitionArgs().getUserSpecifiedPartitionAttributes()) + " "
+            + CliStrings.format(CliStrings.CREATE_REGION__MSG__0_IS_NOT_A_PARITIONEDREGION,
+                regionPath));
       }
-
-      Set<DistributedMember> membersToCreateRegionOn;
-      if (groups != null && groups.length != 0) {
-        membersToCreateRegionOn = CliUtil.getDistributedMembersByGroup(cache, groups);
-        // have only normal members from the group
-        membersToCreateRegionOn
-            .removeIf(distributedMember -> ((InternalDistributedMember) distributedMember)
-                .getVmKind() == DistributionManager.LOCATOR_DM_TYPE);
-      } else {
-        membersToCreateRegionOn = CliUtil.getAllNormalMembers(cache);
+      functionArgs.setRegionShortcut(regionShortcut);
+    } else if (templateRegion != null) {
+      if (!regionExists(cache, templateRegion)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.format(
+            CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH_FOR_0_REGIONPATH_1_NOT_FOUND,
+            CliStrings.CREATE_REGION__USEATTRIBUTESFROM, templateRegion));
       }
 
-      if (membersToCreateRegionOn.isEmpty()) {
-        return ResultBuilder.createUserErrorResult(CliStrings.NO_CACHING_MEMBERS_FOUND_MESSAGE);
-      }
+      regionAttributes = getRegionAttributes(cache, templateRegion);
 
-      ResultCollector<?, ?> resultCollector = CliUtil.executeFunction(RegionCreateFunction.INSTANCE,
-          regionFunctionArgs, membersToCreateRegionOn);
-      @SuppressWarnings("unchecked")
-      List<CliFunctionResult> regionCreateResults =
-          (List<CliFunctionResult>) resultCollector.getResult();
-
-      TabularResultData tabularResultData = ResultBuilder.createTabularResultData();
-      final String errorPrefix = "ERROR: ";
-      for (CliFunctionResult regionCreateResult : regionCreateResults) {
-        boolean success = regionCreateResult.isSuccessful();
-        tabularResultData.accumulate("Member", regionCreateResult.getMemberIdOrName());
-        tabularResultData.accumulate("Status",
-            (success ? "" : errorPrefix) + regionCreateResult.getMessage());
-
-        if (success) {
-          xmlEntity.set(regionCreateResult.getXmlEntity());
-        } else {
-          tabularResultData.setStatus(Result.Status.ERROR);
-        }
+      if (regionAttributes == null) {
+        return ResultBuilder.createGemFireErrorResult(CliStrings.format(
+            CliStrings.CREATE_REGION__MSG__COULD_NOT_RETRIEVE_REGION_ATTRS_FOR_PATH_0_VERIFY_REGION_EXISTS,
+            templateRegion));
       }
-      result = ResultBuilder.buildResult(tabularResultData);
-      verifyDistributedRegionMbean(cache, regionPath);
-
-    } catch (IllegalArgumentException | IllegalStateException e) {
-      LogWrapper.getInstance().info(e.getMessage());
-      result = ResultBuilder.createUserErrorResult(e.getMessage());
-    }
-
-    if (xmlEntity.get() != null) {
-      persistClusterConfiguration(result,
-          () -> getSharedConfiguration().addXmlEntity(xmlEntity.get(), groups));
-    }
-
-    return result;
-  }
-
-  private boolean verifyDistributedRegionMbean(InternalCache cache, String regionName) {
-    int federationInterval =
-        cache.getInternalDistributedSystem().getConfig().getJmxManagerUpdateRate();
-    long timeEnd = System.currentTimeMillis() + federationInterval + 50;
 
-    for (; System.currentTimeMillis() <= timeEnd;) {
-      try {
-        DistributedRegionMXBean bean =
-            ManagementService.getManagementService(cache).getDistributedRegionMXBean(regionName);
-        if (bean == null) {
-          bean = ManagementService.getManagementService(cache)
-              .getDistributedRegionMXBean(Region.SEPARATOR + regionName);
-        }
-        if (bean != null) {
-          return true;
-        } else {
-          Thread.sleep(2);
-        }
-      } catch (Exception ignored) {
+      if (regionAttributes.getPartitionAttributes() == null
+          && functionArgs.hasPartitionAttributes()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.format(
+            CliStrings.CREATE_REGION__MSG__OPTION_0_CAN_BE_USED_ONLY_FOR_PARTITIONEDREGION,
+            functionArgs.getPartitionArgs().getUserSpecifiedPartitionAttributes()) + " "
+            + CliStrings.format(CliStrings.CREATE_REGION__MSG__0_IS_NOT_A_PARITIONEDREGION,
+                templateRegion));
       }
+      functionArgs.setTemplateRegion(templateRegion);
+      functionArgs.setRegionAttributes(regionAttributes);
     }
-    return false;
-  }
 
-  void validateRegionFunctionArgs(InternalCache cache, RegionFunctionArgs regionFunctionArgs) {
-    if (regionFunctionArgs.getRegionPath() == null) {
-      throw new IllegalArgumentException(CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH);
-    }
+    functionArgs.setCacheListeners(cacheListener);
+    functionArgs.setCacheLoader(cacheLoader);
+    functionArgs.setCacheWriter(cacheWriter);
+    functionArgs.setCompressor(compressor);
 
     DistributedSystemMXBean dsMBean = getDSMBean(cache);
-
-    String useAttributesFrom = regionFunctionArgs.getUseAttributesFrom();
-    if (useAttributesFrom != null && !useAttributesFrom.isEmpty()
-        && regionExists(cache, useAttributesFrom)) {
-      if (!regionExists(cache, useAttributesFrom)) { // check already done in createRegion !!!
-        throw new IllegalArgumentException(CliStrings.format(
-            CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH_FOR_0_REGIONPATH_1_NOT_FOUND,
-            CliStrings.CREATE_REGION__USEATTRIBUTESFROM, useAttributesFrom));
-      }
-      if (!regionFunctionArgs.isSetUseAttributesFrom()
-          || regionFunctionArgs.getRegionAttributes() == null) {
-        throw new IllegalArgumentException(CliStrings.format(
-            CliStrings.CREATE_REGION__MSG__COULD_NOT_RETRIEVE_REGION_ATTRS_FOR_PATH_0_VERIFY_REGION_EXISTS,
-            useAttributesFrom));
-      }
-    }
-
-    if (regionFunctionArgs.hasPartitionAttributes()) {
-      RegionFunctionArgs.PartitionArgs partitionArgs = regionFunctionArgs.getPartitionArgs();
-      String colocatedWith = partitionArgs.getPrColocatedWith();
-      if (colocatedWith != null && !colocatedWith.isEmpty()) {
-        String[] listAllRegionPaths = dsMBean.listAllRegionPaths();
-        String foundRegionPath = null;
-        for (String regionPath : listAllRegionPaths) {
-          if (regionPath.equals(colocatedWith)) {
-            foundRegionPath = regionPath;
-            break;
-          }
-        }
-        if (foundRegionPath == null) {
-          throw new IllegalArgumentException(CliStrings.format(
-              CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH_FOR_0_REGIONPATH_1_NOT_FOUND,
-              CliStrings.CREATE_REGION__COLOCATEDWITH, colocatedWith));
-        }
+    // validating colocation
+    if (functionArgs.hasPartitionAttributes()) {
+      if (prColocatedWith != null) {
         ManagementService mgmtService = ManagementService.getExistingManagementService(cache);
         DistributedRegionMXBean distributedRegionMXBean =
-            mgmtService.getDistributedRegionMXBean(foundRegionPath);
+            mgmtService.getDistributedRegionMXBean(prColocatedWith);
+        if (distributedRegionMXBean == null) {
+          return ResultBuilder.createUserErrorResult(CliStrings.format(
+              CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH_FOR_0_REGIONPATH_1_NOT_FOUND,
+              CliStrings.CREATE_REGION__COLOCATEDWITH, prColocatedWith));
+        }
         String regionType = distributedRegionMXBean.getRegionType();
         if (!(DataPolicy.PARTITION.toString().equals(regionType)
             || DataPolicy.PERSISTENT_PARTITION.toString().equals(regionType))) {
-          throw new IllegalArgumentException(CliStrings.format(
+          return ResultBuilder.createUserErrorResult(CliStrings.format(
               CliStrings.CREATE_REGION__MSG__COLOCATEDWITH_REGION_0_IS_NOT_PARTITIONEDREGION,
-              new Object[] {colocatedWith}));
-        }
-      }
-      if (partitionArgs.isSetPRLocalMaxMemory()) {
-        int prLocalMaxMemory = partitionArgs.getPrLocalMaxMemory();
-        if (prLocalMaxMemory < 0) {
-          throw new IllegalArgumentException(
-              LocalizedStrings.AttributesFactory_PARTITIONATTRIBUTES_LOCALMAXMEMORY_MUST_NOT_BE_NEGATIVE
-                  .toLocalizedString());
-        }
-      }
-      if (partitionArgs.isSetPRTotalMaxMemory()) {
-        long prTotalMaxMemory = partitionArgs.getPrTotalMaxMemory();
-        if (prTotalMaxMemory <= 0) {
-          throw new IllegalArgumentException(
-              LocalizedStrings.AttributesFactory_TOTAL_SIZE_OF_PARTITION_REGION_MUST_BE_0
-                  .toLocalizedString());
-        }
-      }
-      if (partitionArgs.isSetPRRedundantCopies()) {
-        int prRedundantCopies = partitionArgs.getPrRedundantCopies();
-        switch (prRedundantCopies) {
-          case 0:
-          case 1:
-          case 2:
-          case 3:
-            break;
-          default:
-            throw new IllegalArgumentException(CliStrings.format(
-                CliStrings.CREATE_REGION__MSG__REDUNDANT_COPIES_SHOULD_BE_ONE_OF_0123,
-                new Object[] {prRedundantCopies}));
-        }
-      }
-    }
-
-    String keyConstraint = regionFunctionArgs.getKeyConstraint();
-    if (keyConstraint != null && !RegionCommandsUtils.isClassNameValid(keyConstraint)) {
-      throw new IllegalArgumentException(CliStrings.format(
-          CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_KEYCONSTRAINT_0_IS_INVALID,
-          new Object[] {keyConstraint}));
-    }
-
-    String valueConstraint = regionFunctionArgs.getValueConstraint();
-    if (valueConstraint != null && !RegionCommandsUtils.isClassNameValid(valueConstraint)) {
-      throw new IllegalArgumentException(CliStrings.format(
-          CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_VALUECONSTRAINT_0_IS_INVALID,
-          new Object[] {valueConstraint}));
-    }
-
-    Set<String> cacheListeners = regionFunctionArgs.getCacheListeners();
-    if (cacheListeners != null && !cacheListeners.isEmpty()) {
-      for (String cacheListener : cacheListeners) {
-        if (!RegionCommandsUtils.isClassNameValid(cacheListener)) {
-          throw new IllegalArgumentException(CliStrings.format(
-              CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHELISTENER_0_IS_INVALID,
-              new Object[] {cacheListener}));
+              new Object[] {prColocatedWith}));
         }
       }
     }
 
-    String cacheLoader = regionFunctionArgs.getCacheLoader();
-    if (cacheLoader != null && !RegionCommandsUtils.isClassNameValid(cacheLoader)) {
-      throw new IllegalArgumentException(CliStrings.format(
-          CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHELOADER_0_IS_INVALID,
-          new Object[] {cacheLoader}));
-    }
-
-    String cacheWriter = regionFunctionArgs.getCacheWriter();
-    if (cacheWriter != null && !RegionCommandsUtils.isClassNameValid(cacheWriter)) {
-      throw new IllegalArgumentException(CliStrings.format(
-          CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHEWRITER_0_IS_INVALID,
-          new Object[] {cacheWriter}));
-    }
-
-    Set<String> gatewaySenderIds = regionFunctionArgs.getGatewaySenderIds();
-    if (gatewaySenderIds != null && !gatewaySenderIds.isEmpty()) {
-      String[] gatewaySenders = dsMBean.listGatewaySenders();
-      if (gatewaySenders.length == 0) {
-        throw new IllegalArgumentException(
-            CliStrings.CREATE_REGION__MSG__NO_GATEWAYSENDERS_IN_THE_SYSTEM);
+    // validating gateway senders
+    if (gatewaySenderIds != null) {
+      Set<String> existingGatewaySenders =
+          Arrays.stream(dsMBean.listGatewaySenders()).collect(Collectors.toSet());
+      if (existingGatewaySenders.size() == 0) {
+        return ResultBuilder
+            .createUserErrorResult(CliStrings.CREATE_REGION__MSG__NO_GATEWAYSENDERS_IN_THE_SYSTEM);
       } else {
-        List<String> gatewaySendersList = new ArrayList<>(Arrays.asList(gatewaySenders));
-        gatewaySenderIds = new HashSet<>(gatewaySenderIds);
-        gatewaySenderIds.removeAll(gatewaySendersList);
-        if (!gatewaySenderIds.isEmpty()) {
-          throw new IllegalArgumentException(CliStrings.format(
+        Set<String> specifiedGatewaySenders =
+            Arrays.stream(gatewaySenderIds).collect(Collectors.toSet());
+        specifiedGatewaySenders.removeAll(existingGatewaySenders);
+        if (!specifiedGatewaySenders.isEmpty()) {
+          return ResultBuilder.createUserErrorResult(CliStrings.format(
               CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_GATEWAYSENDER_ID_UNKNOWN_0,
               new Object[] {gatewaySenderIds}));
         }
       }
     }
 
-    if (regionFunctionArgs.isSetConcurrencyLevel()) {
-      int concurrencyLevel = regionFunctionArgs.getConcurrencyLevel();
-      if (concurrencyLevel < 0) {
-        throw new IllegalArgumentException(CliStrings.format(
-            CliStrings.CREATE_REGION__MSG__SPECIFY_POSITIVE_INT_FOR_CONCURRENCYLEVEL_0_IS_NOT_VALID,
-            new Object[] {concurrencyLevel}));
-      }
-    }
-
-    String diskStore = regionFunctionArgs.getDiskStore();
+    // validating diskstore with other attributes
     if (diskStore != null) {
-      RegionShortcut regionShortcut = regionFunctionArgs.getRegionShortcut();
-      if (regionShortcut != null
-          && !RegionCommandsUtils.PERSISTENT_OVERFLOW_SHORTCUTS.contains(regionShortcut)) {
-        String subMessage =
-            LocalizedStrings.DiskStore_IS_USED_IN_NONPERSISTENT_REGION.toLocalizedString();
-        String message = subMessage + ". "
-            + CliStrings.format(CliStrings.CREATE_REGION__MSG__USE_ONE_OF_THESE_SHORTCUTS_0,
-                new Object[] {String.valueOf(RegionCommandsUtils.PERSISTENT_OVERFLOW_SHORTCUTS)});
-
-        throw new IllegalArgumentException(message);
-      }
-
-      RegionAttributes<?, ?> regionAttributes = regionFunctionArgs.getRegionAttributes();
+      regionAttributes = functionArgs.getRegionAttributes();
       if (regionAttributes != null && !regionAttributes.getDataPolicy().withPersistence()) {
         String subMessage =
             LocalizedStrings.DiskStore_IS_USED_IN_NONPERSISTENT_REGION.toLocalizedString();
         String message = subMessage + ". "
             + CliStrings.format(
                 CliStrings.CREATE_REGION__MSG__USE_ATTRIBUTES_FROM_REGION_0_IS_NOT_WITH_PERSISTENCE,
-                new Object[] {String.valueOf(regionFunctionArgs.getUseAttributesFrom())});
+                new Object[] {String.valueOf(functionArgs.getTemplateRegion())});
 
-        throw new IllegalArgumentException(message);
+        return ResultBuilder.createUserErrorResult(message);
       }
 
       if (!diskStoreExists(cache, diskStore)) {
-        throw new IllegalArgumentException(CliStrings.format(
+        return ResultBuilder.createUserErrorResult(CliStrings.format(
             CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_DISKSTORE_UNKNOWN_DISKSTORE_0,
             new Object[] {diskStore}));
       }
     }
 
-    RegionFunctionArgs.ExpirationAttrs entryExpirationIdleTime =
-        regionFunctionArgs.getEntryExpirationIdleTime();
-    RegionFunctionArgs.ExpirationAttrs entryExpirationTTL =
-        regionFunctionArgs.getEntryExpirationTTL();
-    RegionFunctionArgs.ExpirationAttrs regionExpirationIdleTime =
-        regionFunctionArgs.getRegionExpirationIdleTime();
-    RegionFunctionArgs.ExpirationAttrs regionExpirationTTL =
-        regionFunctionArgs.getRegionExpirationTTL();
-
-    if ((!regionFunctionArgs.isSetStatisticsEnabled() || !regionFunctionArgs.isStatisticsEnabled())
-        && (entryExpirationIdleTime != null || entryExpirationTTL != null
-            || regionExpirationIdleTime != null || regionExpirationTTL != null)) {
-      String message = LocalizedStrings.AttributesFactory_STATISTICS_MUST_BE_ENABLED_FOR_EXPIRATION
-          .toLocalizedString();
-      throw new IllegalArgumentException(message + ".");
+    // additional authorization
+    if (isPersistentShortcut(functionArgs.getRegionShortcut())
+        || isAttributePersistent(functionArgs.getRegionAttributes())) {
+      getSecurityService().authorize(ResourcePermission.Resource.CLUSTER,
+          ResourcePermission.Operation.WRITE, ResourcePermission.Target.DISK);
     }
 
-    boolean compressorFailure = false;
-    if (regionFunctionArgs.isSetCompressor()) {
-      String compressorClassName = regionFunctionArgs.getCompressor();
-      Object compressor = null;
-      try {
-        Class<?> compressorClass = ClassPathLoader.getLatest().forName(compressorClassName);
-        compressor = compressorClass.newInstance();
-      } catch (InstantiationException | ClassNotFoundException | IllegalAccessException e) {
-        compressorFailure = true;
+    // validating the groups
+    Set<DistributedMember> membersToCreateRegionOn = findMembers(groups, null);
+    // just in case we found no members with this group name
+    if (membersToCreateRegionOn.isEmpty()) {
+      if (groups == null || groups.length == 0) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      } else {
+        return ResultBuilder.createUserErrorResult(
+            CliStrings.format(CliStrings.CREATE_REGION__MSG__GROUPS_0_ARE_INVALID,
+                new Object[] {String.valueOf(groups)}));
+      }
+    }
+
+    ResultCollector<?, ?> resultCollector =
+        executeFunction(RegionCreateFunction.INSTANCE, functionArgs, membersToCreateRegionOn);
+    @SuppressWarnings("unchecked")
+    List<CliFunctionResult> regionCreateResults =
+        (List<CliFunctionResult>) resultCollector.getResult();
+
+    TabularResultData tabularResultData = ResultBuilder.createTabularResultData();
+    final String errorPrefix = "ERROR: ";
+    for (CliFunctionResult regionCreateResult : regionCreateResults) {
+      boolean success = regionCreateResult.isSuccessful();
+      tabularResultData.accumulate("Member", regionCreateResult.getMemberIdOrName());
+      tabularResultData.accumulate("Status",
+          (success ? "" : errorPrefix) + regionCreateResult.getMessage());
+
+      if (success) {
+        xmlEntity.set(regionCreateResult.getXmlEntity());
+      } else {
+        tabularResultData.setStatus(Result.Status.ERROR);
       }
+    }
+
+    result = ResultBuilder.buildResult(tabularResultData);
+    verifyDistributedRegionMbean(cache, regionPath);
+
+    if (xmlEntity.get() != null) {
+      persistClusterConfiguration(result,
+          () -> getSharedConfiguration().addXmlEntity(xmlEntity.get(), groups));
+    }
+
+    return result;
+  }
+
+  boolean verifyDistributedRegionMbean(InternalCache cache, String regionName) {
+    int federationInterval =
+        cache.getInternalDistributedSystem().getConfig().getJmxManagerUpdateRate();
+    long timeEnd = System.currentTimeMillis() + federationInterval + 50;
 
-      if (compressorFailure || !(compressor instanceof Compressor)) {
-        throw new IllegalArgumentException(
-            CliStrings.format(CliStrings.CREATE_REGION__MSG__INVALID_COMPRESSOR,
-                new Object[] {regionFunctionArgs.getCompressor()}));
+    for (; System.currentTimeMillis() <= timeEnd;) {
+      try {
+        DistributedRegionMXBean bean =
+            ManagementService.getManagementService(cache).getDistributedRegionMXBean(regionName);
+        if (bean == null) {
+          bean = ManagementService.getManagementService(cache)
+              .getDistributedRegionMXBean(Region.SEPARATOR + regionName);
+        }
+        if (bean != null) {
+          return true;
+        } else {
+          Thread.sleep(2);
+        }
+      } catch (Exception ignored) {
       }
     }
+    return false;
   }
 
-  private static <K, V> FetchRegionAttributesFunction.FetchRegionAttributesFunctionResult<K, V> getRegionAttributes(
-      InternalCache cache, String regionPath) {
+  RegionAttributes getRegionAttributes(InternalCache cache, String regionPath) {
     if (!isClusterWideSameConfig(cache, regionPath)) {
       throw new IllegalStateException(CliStrings.format(
           CliStrings.CREATE_REGION__MSG__USE_ATTRIBUTES_FORM_REGIONS_EXISTS_BUT_DIFFERENT_SCOPE_OR_DATAPOLICY_USE_DESCRIBE_REGION_FOR_0,
           regionPath));
     }
-    FetchRegionAttributesFunction.FetchRegionAttributesFunctionResult<K, V> attributes = null;
+    RegionAttributes attributes = null;
 
     // First check whether the region exists on a this manager, if yes then no
     // need to use FetchRegionAttributesFunction to fetch RegionAttributes
@@ -595,8 +428,7 @@ public class CreateRegionCommand implements GfshCommand {
                   regionPath, th.getMessage()));
             } else { // has to be RegionAttributes
               @SuppressWarnings("unchecked") // to avoid warning :(
-              FetchRegionAttributesFunction.FetchRegionAttributesFunctionResult<K, V> regAttr =
-                  ((FetchRegionAttributesFunction.FetchRegionAttributesFunctionResult<K, V>) object);
+              RegionAttributes regAttr = ((RegionAttributes) object);
               if (attributes == null) {
                 attributes = regAttr;
                 break;
@@ -643,7 +475,7 @@ public class CreateRegionCommand implements GfshCommand {
     return true;
   }
 
-  static boolean regionExists(InternalCache cache, String regionPath) {
+  boolean regionExists(InternalCache cache, String regionPath) {
     if (regionPath == null || Region.SEPARATOR.equals(regionPath)) {
       return false;
     }
@@ -672,22 +504,6 @@ public class CreateRegionCommand implements GfshCommand {
     return false;
   }
 
-  private void validateRegionPathAndParent(InternalCache cache, String regionPath) {
-    if (StringUtils.isEmpty(regionPath)) {
-      throw new IllegalArgumentException(CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH);
-    }
-    // If a region path indicates a sub-region, check whether the parent region exists
-    RegionPath regionPathData = new RegionPath(regionPath);
-    String parentRegionPath = regionPathData.getParent();
-    if (parentRegionPath != null && !Region.SEPARATOR.equals(parentRegionPath)) {
-      if (!regionExists(cache, parentRegionPath)) {
-        throw new IllegalArgumentException(
-            CliStrings.format(CliStrings.CREATE_REGION__MSG__PARENT_REGION_FOR_0_DOES_NOT_EXIST,
-                new Object[] {regionPath}));
-      }
-    }
-  }
-
   private boolean isPersistentShortcut(RegionShortcut shortcut) {
     return shortcut == RegionShortcut.LOCAL_PERSISTENT
         || shortcut == RegionShortcut.LOCAL_PERSISTENT_OVERFLOW
@@ -708,4 +524,133 @@ public class CreateRegionCommand implements GfshCommand {
     ManagementService managementService = ManagementService.getExistingManagementService(cache);
     return managementService.getDistributedSystemMXBean();
   }
+
+
+  public static class Interceptor extends AbstractCliAroundInterceptor {
+    @Override
+    public Result preExecution(GfshParseResult parseResult) {
+      Integer localMaxMemory =
+          (Integer) parseResult.getParamValue(CliStrings.CREATE_REGION__LOCALMAXMEMORY);
+      if (localMaxMemory != null) {
+        if (localMaxMemory < 0) {
+          return ResultBuilder.createUserErrorResult(
+              LocalizedStrings.AttributesFactory_PARTITIONATTRIBUTES_LOCALMAXMEMORY_MUST_NOT_BE_NEGATIVE
+                  .toLocalizedString());
+        }
+      }
+      Long totalMaxMemory =
+          (Long) parseResult.getParamValue(CliStrings.CREATE_REGION__TOTALMAXMEMORY);
+      if (totalMaxMemory != null) {
+        if (totalMaxMemory <= 0) {
+          return ResultBuilder.createUserErrorResult(
+              LocalizedStrings.AttributesFactory_TOTAL_SIZE_OF_PARTITION_REGION_MUST_BE_0
+                  .toLocalizedString());
+        }
+      }
+      Integer redundantCopies =
+          (Integer) parseResult.getParamValue(CliStrings.CREATE_REGION__REDUNDANTCOPIES);
+      if (redundantCopies != null) {
+        if (redundantCopies < 0 || redundantCopies > 3) {
+          return ResultBuilder.createUserErrorResult(CliStrings.format(
+              CliStrings.CREATE_REGION__MSG__REDUNDANT_COPIES_SHOULD_BE_ONE_OF_0123,
+              new Object[] {redundantCopies}));
+        }
+      }
+
+      Integer concurrencyLevel =
+          (Integer) parseResult.getParamValue(CliStrings.CREATE_REGION__CONCURRENCYLEVEL);
+      if (concurrencyLevel != null) {
+        if (concurrencyLevel < 0) {
+          return ResultBuilder.createUserErrorResult(CliStrings.format(
+              CliStrings.CREATE_REGION__MSG__SPECIFY_POSITIVE_INT_FOR_CONCURRENCYLEVEL_0_IS_NOT_VALID,
+              new Object[] {concurrencyLevel}));
+        }
+      }
+
+      String keyConstraint =
+          parseResult.getParamValueAsString(CliStrings.CREATE_REGION__KEYCONSTRAINT);
+      if (keyConstraint != null && !RegionCommandsUtils.isClassNameValid(keyConstraint)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.format(
+            CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_KEYCONSTRAINT_0_IS_INVALID,
+            new Object[] {keyConstraint}));
+      }
+
+      String valueConstraint =
+          parseResult.getParamValueAsString(CliStrings.CREATE_REGION__VALUECONSTRAINT);
+      if (valueConstraint != null && !RegionCommandsUtils.isClassNameValid(valueConstraint)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.format(
+            CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_VALUECONSTRAINT_0_IS_INVALID,
+            new Object[] {valueConstraint}));
+      }
+
+      String cacheListenerList =
+          parseResult.getParamValueAsString(CliStrings.CREATE_REGION__CACHELISTENER);
+      if (cacheListenerList != null) {
+        String[] cacheListeners = cacheListenerList.split(",");
+        for (String cacheListener : cacheListeners) {
+          if (!RegionCommandsUtils.isClassNameValid(cacheListener)) {
+            return ResultBuilder.createUserErrorResult(CliStrings.format(
+                CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHELISTENER_0_IS_INVALID,
+                new Object[] {cacheListener}));
+          }
+        }
+      }
+
+      String cacheLoader = parseResult.getParamValueAsString(CliStrings.CREATE_REGION__CACHELOADER);
+      if (cacheLoader != null && !RegionCommandsUtils.isClassNameValid(cacheLoader)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.format(
+            CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHELOADER_0_IS_INVALID,
+            new Object[] {cacheLoader}));
+      }
+
+      String cacheWriter = parseResult.getParamValueAsString(CliStrings.CREATE_REGION__CACHEWRITER);
+      if (cacheWriter != null && !RegionCommandsUtils.isClassNameValid(cacheWriter)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.format(
+            CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_CLASSNAME_FOR_CACHEWRITER_0_IS_INVALID,
+            new Object[] {cacheWriter}));
+      }
+
+      String compressor = parseResult.getParamValueAsString(CliStrings.CREATE_REGION__COMPRESSOR);
+      if (compressor != null && !RegionCommandsUtils.isClassNameValid(compressor)) {
+        return ResultBuilder.createUserErrorResult(CliStrings
+            .format(CliStrings.CREATE_REGION__MSG__INVALID_COMPRESSOR, new Object[] {compressor}));
+      }
+
+      String diskStore = parseResult.getParamValueAsString(CliStrings.CREATE_REGION__DISKSTORE);
+      if (diskStore != null) {
+        String regionShortcut =
+            parseResult.getParamValueAsString(CliStrings.CREATE_REGION__REGIONSHORTCUT);
+        if (regionShortcut != null && !RegionCommandsUtils.PERSISTENT_OVERFLOW_SHORTCUTS
+            .contains(RegionShortcut.valueOf(regionShortcut))) {
+          String subMessage =
+              LocalizedStrings.DiskStore_IS_USED_IN_NONPERSISTENT_REGION.toLocalizedString();
+          String message = subMessage + ". "
+              + CliStrings.format(CliStrings.CREATE_REGION__MSG__USE_ONE_OF_THESE_SHORTCUTS_0,
+                  new Object[] {String.valueOf(RegionCommandsUtils.PERSISTENT_OVERFLOW_SHORTCUTS)});
+
+          return ResultBuilder.createUserErrorResult(message);
+        }
+      }
+
+      String statisticsEnabled =
+          parseResult.getParamValueAsString(CliStrings.CREATE_REGION__STATISTICSENABLED);
+      if (!Boolean.parseBoolean(statisticsEnabled)) {
+        String entryIdle =
+            parseResult.getParamValueAsString(CliStrings.CREATE_REGION__ENTRYEXPIRATIONIDLETIME);
+        String entryTtl =
+            parseResult.getParamValueAsString(CliStrings.CREATE_REGION__ENTRYEXPIRATIONTIMETOLIVE);
+        String regionIdle =
+            parseResult.getParamValueAsString(CliStrings.CREATE_REGION__REGIONEXPIRATIONIDLETIME);
+        String regionTtl =
+            parseResult.getParamValueAsString(CliStrings.CREATE_REGION__REGIONEXPIRATIONTTL);
+        if (entryIdle != null || entryTtl != null || regionIdle != null || regionTtl != null) {
+          String message =
+              LocalizedStrings.AttributesFactory_STATISTICS_MUST_BE_ENABLED_FOR_EXPIRATION
+                  .toLocalizedString();
+          return ResultBuilder.createUserErrorResult(message + ".");
+        }
+      }
+      return ResultBuilder.createInfoResult("");
+    }
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
index c6699d4..a785068 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
@@ -24,7 +24,9 @@ import org.springframework.shell.core.CommandMarker;
 
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.execute.Execution;
+import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
@@ -154,4 +156,9 @@ public interface GfshCommand extends CommandMarker {
     return CliUtil.getRegionAssociatedMembers(regionPath, cache);
   }
 
+  default ResultCollector<?, ?> executeFunction(final Function function, Object args,
+      final Set<DistributedMember> targetMembers) {
+    return CliUtil.executeFunction(function, args, targetMembers);
+  }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/FetchRegionAttributesFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/FetchRegionAttributesFunction.java
index 5ff5a81..21c395d 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/FetchRegionAttributesFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/FetchRegionAttributesFunction.java
@@ -14,21 +14,15 @@
  */
 package org.apache.geode.management.internal.cli.functions;
 
-import java.io.Serializable;
-
-import org.apache.logging.log4j.Logger;
-
 import org.apache.geode.cache.AttributesFactory;
 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.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.execute.FunctionAdapter;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.logging.log4j.Logger;
 
 /**
  * 
@@ -57,7 +51,7 @@ public class FetchRegionAttributesFunction extends FunctionAdapter {
         throw new IllegalArgumentException(
             CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH);
       }
-      FetchRegionAttributesFunctionResult<?, ?> result = getRegionAttributes(cache, regionPath);
+      RegionAttributes<?, ?> result = getRegionAttributes(cache, regionPath);
       context.getResultSender().lastResult(result);
     } catch (IllegalArgumentException e) {
       if (logger.isDebugEnabled()) {
@@ -68,8 +62,7 @@ public class FetchRegionAttributesFunction extends FunctionAdapter {
   }
 
   @SuppressWarnings("deprecation")
-  public static <K, V> FetchRegionAttributesFunctionResult<K, V> getRegionAttributes(Cache cache,
-      String regionPath) {
+  public static <K, V> RegionAttributes<K, V> getRegionAttributes(Cache cache, String regionPath) {
     Region<K, V> foundRegion = cache.getRegion(regionPath);
 
     if (foundRegion == null) {
@@ -81,65 +74,11 @@ public class FetchRegionAttributesFunction extends FunctionAdapter {
     // Using AttributesFactory to get the serializable RegionAttributes
     // Is there a better way?
     AttributesFactory<K, V> afactory = new AttributesFactory<K, V>(foundRegion.getAttributes());
-    FetchRegionAttributesFunctionResult<K, V> result =
-        new FetchRegionAttributesFunctionResult<K, V>(afactory);
-    return result;
+    return afactory.create();
   }
 
   @Override
   public String getId() {
     return ID;
   }
-
-  public static class FetchRegionAttributesFunctionResult<K, V> implements Serializable {
-    private static final long serialVersionUID = -3970828263897978845L;
-
-    private RegionAttributes<K, V> regionAttributes;
-    private String[] cacheListenerClasses;
-    private String cacheLoaderClass;
-    private String cacheWriterClass;
-
-    @SuppressWarnings("deprecation")
-    public FetchRegionAttributesFunctionResult(AttributesFactory<K, V> afactory) {
-      this.regionAttributes = afactory.create();
-
-      CacheListener<K, V>[] cacheListeners = this.regionAttributes.getCacheListeners();
-      if (cacheListeners != null && cacheListeners.length != 0) {
-        cacheListenerClasses = new String[cacheListeners.length];
-        for (int i = 0; i < cacheListeners.length; i++) {
-          cacheListenerClasses[i] = cacheListeners[i].getClass().getName();
-        }
-        afactory.initCacheListeners(null);
-      }
-      CacheLoader<K, V> cacheLoader = this.regionAttributes.getCacheLoader();
-      if (cacheLoader != null) {
-        cacheLoaderClass = cacheLoader.getClass().getName();
-        afactory.setCacheLoader(null);
-      }
-      CacheWriter<K, V> cacheWriter = this.regionAttributes.getCacheWriter();
-      if (cacheWriter != null) {
-        cacheWriterClass = cacheWriter.getClass().getName();
-        afactory.setCacheWriter(null);
-      }
-
-      // recreate attributes
-      this.regionAttributes = afactory.create();
-    }
-
-    public RegionAttributes<K, V> getRegionAttributes() {
-      return regionAttributes;
-    }
-
-    public String[] getCacheListenerClasses() {
-      return cacheListenerClasses;
-    }
-
-    public String getCacheLoaderClass() {
-      return cacheLoaderClass;
-    }
-
-    public String getCacheWriterClass() {
-      return cacheWriterClass;
-    }
-  }
 }
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 a58705e..d3eb5e9 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
@@ -135,74 +135,22 @@ public class RegionCreateFunction implements Function, InternalEntity {
   public static <K, V> Region<?, ?> createRegion(Cache cache, RegionFunctionArgs regionCreateArgs) {
     Region<K, V> createdRegion = null;
 
-    final String regionPath = regionCreateArgs.getRegionPath();
     final RegionShortcut regionShortcut = regionCreateArgs.getRegionShortcut();
-    final String useAttributesFrom = regionCreateArgs.getUseAttributesFrom();
-
-    // If a region path indicates a sub-region, check whether the parent region exists
-    RegionPath regionPathData = new RegionPath(regionPath);
-    String parentRegionPath = regionPathData.getParent();
-    Region<?, ?> parentRegion = null;
-    if (parentRegionPath != null && !Region.SEPARATOR.equals(parentRegionPath)) {
-      parentRegion = cache.getRegion(parentRegionPath);
-      if (parentRegion == null) {
-        throw new IllegalArgumentException(
-            CliStrings.format(CliStrings.CREATE_REGION__MSG__PARENT_REGION_FOR_0_DOES_NOT_EXIST,
-                new Object[] {regionPath}));
-      }
-
-      if (parentRegion.getAttributes().getPartitionAttributes() != null) {
-        // For a PR, sub-regions are not supported.
-        throw new CreateSubregionException(
-            CliStrings.format(CliStrings.CREATE_REGION__MSG__0_IS_A_PR_CANNOT_HAVE_SUBREGIONS,
-                parentRegion.getFullPath()));
-      }
-    }
-
-    // One of Region Shortcut OR Use Attributes From has to be given
-    if (regionShortcut == null && useAttributesFrom == null) {
-      throw new IllegalArgumentException(
-          CliStrings.CREATE_REGION__MSG__ONE_OF_REGIONSHORTCUT_AND_USEATTRIBUTESFROM_IS_REQUIRED);
-    }
 
+    // create the region factory using the arguments
     boolean isPartitioned = false;
     RegionFactory<K, V> factory = null;
     RegionAttributes<K, V> regionAttributes = null;
     if (regionShortcut != null) {
       regionAttributes = cache.getRegionAttributes(regionShortcut.toString());
-      if (logger.isDebugEnabled()) {
-        logger.debug("Using shortcut {} for {} region attributes : {}", regionShortcut, regionPath,
-            regionAttributes);
-      }
-
-      if (regionAttributes == null) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("Shortcut {} doesn't have attributes in {}", regionShortcut,
-              cache.listRegionAttributes());
-        }
-        throw new IllegalStateException(CliStrings.format(
-            CliStrings.CREATE_REGION__MSG__COULD_NOT_LOAD_REGION_ATTRIBUTES_FOR_SHORTCUT_0,
-            regionShortcut));
-      }
+      regionCreateArgs.setRegionAttributes(regionAttributes);
     } else {
-      if (logger.isDebugEnabled()) {
-        logger.debug("Using Manager's region attributes for {}", regionPath);
-      }
       regionAttributes = regionCreateArgs.getRegionAttributes();
-      if (logger.isDebugEnabled()) {
-        logger.debug("Using Attributes : {}", regionAttributes);
-      }
     }
-    isPartitioned = regionAttributes.getPartitionAttributes() != null;
 
+    isPartitioned = regionAttributes.getPartitionAttributes() != null;
     factory = cache.createRegionFactory(regionAttributes);
 
-    if (!isPartitioned && regionCreateArgs.hasPartitionAttributes()) {
-      throw new IllegalArgumentException(CliStrings.format(
-          CliStrings.CREATE_REGION__MSG__OPTION_0_CAN_BE_USED_ONLY_FOR_PARTITIONEDREGION,
-          regionCreateArgs.getPartitionArgs().getUserSpecifiedPartitionAttributes()));
-    }
-
     if (isPartitioned) {
       PartitionAttributes<K, V> partitionAttributes =
           extractPartitionAttributes(cache, regionAttributes, regionCreateArgs);
@@ -244,12 +192,12 @@ public class RegionCreateFunction implements Function, InternalEntity {
     final RegionFunctionArgs.ExpirationAttrs regionExpirationIdleTime =
         regionCreateArgs.getRegionExpirationIdleTime();
     if (regionExpirationIdleTime != null) {
-      factory.setEntryIdleTimeout(regionExpirationIdleTime.convertToExpirationAttributes());
+      factory.setRegionIdleTimeout(regionExpirationIdleTime.convertToExpirationAttributes());
     }
     final RegionFunctionArgs.ExpirationAttrs regionExpirationTTL =
         regionCreateArgs.getRegionExpirationTTL();
     if (regionExpirationTTL != null) {
-      factory.setEntryTimeToLive(regionExpirationTTL.convertToExpirationAttributes());
+      factory.setRegionTimeToLive(regionExpirationTTL.convertToExpirationAttributes());
     }
 
     // Associate a Disk Store
@@ -257,24 +205,24 @@ public class RegionCreateFunction implements Function, InternalEntity {
     if (diskStore != null && !diskStore.isEmpty()) {
       factory.setDiskStoreName(diskStore);
     }
-    if (regionCreateArgs.isSetDiskSynchronous()) {
+
+    if (regionCreateArgs.isDiskSynchronous() != null) {
       factory.setDiskSynchronous(regionCreateArgs.isDiskSynchronous());
     }
 
-    if (regionCreateArgs.isSetOffHeap()) {
+    if (regionCreateArgs.isOffHeap() != null) {
       factory.setOffHeap(regionCreateArgs.isOffHeap());
     }
 
-    // Set stats enabled
-    if (regionCreateArgs.isSetStatisticsEnabled()) {
+    if (regionCreateArgs.isStatisticsEnabled() != null) {
       factory.setStatisticsEnabled(regionCreateArgs.isStatisticsEnabled());
     }
 
-    // Set conflation
-    if (regionCreateArgs.isSetEnableAsyncConflation()) {
+    if (regionCreateArgs.isEnableAsyncConflation() != null) {
       factory.setEnableAsyncConflation(regionCreateArgs.isEnableAsyncConflation());
     }
-    if (regionCreateArgs.isSetEnableSubscriptionConflation()) {
+
+    if (regionCreateArgs.isEnableSubscriptionConflation() != null) {
       factory.setEnableSubscriptionConflation(regionCreateArgs.isEnableSubscriptionConflation());
     }
 
@@ -294,21 +242,19 @@ public class RegionCreateFunction implements Function, InternalEntity {
       }
     }
 
-    // concurrency check enabled & concurrency level
-    if (regionCreateArgs.isSetConcurrencyChecksEnabled()) {
+    if (regionCreateArgs.isConcurrencyChecksEnabled() != null) {
       factory.setConcurrencyChecksEnabled(regionCreateArgs.isConcurrencyChecksEnabled());
     }
-    if (regionCreateArgs.isSetConcurrencyLevel()) {
+
+    if (regionCreateArgs.getConcurrencyLevel() != null) {
       factory.setConcurrencyLevel(regionCreateArgs.getConcurrencyLevel());
     }
 
-    // cloning enabled for delta
-    if (regionCreateArgs.isSetCloningEnabled()) {
+    if (regionCreateArgs.isCloningEnabled() != null) {
       factory.setCloningEnabled(regionCreateArgs.isCloningEnabled());
     }
 
-    // multicast enabled for replication
-    if (regionCreateArgs.isSetMcastEnabled()) {
+    if (regionCreateArgs.isMcastEnabled() != null) {
       factory.setMulticastEnabled(regionCreateArgs.isMcastEnabled());
     }
 
@@ -324,7 +270,7 @@ public class RegionCreateFunction implements Function, InternalEntity {
     }
 
     // Compression provider
-    if (regionCreateArgs.isSetCompressor()) {
+    if (regionCreateArgs.getCompressor() != null) {
       Class<Compressor> compressorKlass =
           CliUtil.forName(regionCreateArgs.getCompressor(), CliStrings.CREATE_REGION__COMPRESSOR);
       factory.setCompressor(
@@ -347,9 +293,13 @@ public class RegionCreateFunction implements Function, InternalEntity {
           CliUtil.newInstance(cacheWriterKlass, CliStrings.CREATE_REGION__CACHEWRITER));
     }
 
+    // If a region path indicates a sub-region,
+    final String regionPath = regionCreateArgs.getRegionPath();
+    RegionPath regionPathData = new RegionPath(regionPath);
     String regionName = regionPathData.getName();
-
-    if (parentRegion != null) {
+    String parentRegionPath = regionPathData.getParent();
+    if (parentRegionPath != null && !Region.SEPARATOR.equals(parentRegionPath)) {
+      Region<?, ?> parentRegion = cache.getRegion(parentRegionPath);
       createdRegion = factory.createSubregion(parentRegion, regionName);
     } else {
       createdRegion = factory.create(regionName);
@@ -386,28 +336,29 @@ public class RegionCreateFunction implements Function, InternalEntity {
       }
       prAttrFactory.setColocatedWith(colocatedWith);
     }
-    if (partitionArgs.isSetPRLocalMaxMemory()) {
+    if (partitionArgs.getPrLocalMaxMemory() != null) {
       prAttrFactory.setLocalMaxMemory(partitionArgs.getPrLocalMaxMemory());
     }
-    if (partitionArgs.isSetPRTotalMaxMemory()) {
+    if (partitionArgs.getPrTotalMaxMemory() != null) {
       prAttrFactory.setTotalMaxMemory(partitionArgs.getPrTotalMaxMemory());
     }
-    if (partitionArgs.isSetPRTotalNumBuckets()) {
+    if (partitionArgs.getPrTotalNumBuckets() != null) {
       prAttrFactory.setTotalNumBuckets(partitionArgs.getPrTotalNumBuckets());
     }
-    if (partitionArgs.isSetPRRedundantCopies()) {
+    if (partitionArgs.getPrRedundantCopies() != null) {
       prAttrFactory.setRedundantCopies(partitionArgs.getPrRedundantCopies());
     }
-    if (partitionArgs.isSetPRRecoveryDelay()) {
+    if (partitionArgs.getPrRecoveryDelay() != null) {
       prAttrFactory.setRecoveryDelay(partitionArgs.getPrRecoveryDelay());
     }
-    if (partitionArgs.isSetPRStartupRecoveryDelay()) {
+    if (partitionArgs.getPrStartupRecoveryDelay() != null) {
       prAttrFactory.setStartupRecoveryDelay(partitionArgs.getPrStartupRecoveryDelay());
     }
 
-    if (regionCreateArgs.isPartitionResolverSet()) {
-      Class<PartitionResolver> partitionResolverClass = forName(
-          regionCreateArgs.getPartitionResolver(), CliStrings.CREATE_REGION__PARTITION_RESOLVER);
+    if (regionCreateArgs.getPartitionArgs().getPartitionResolver() != null) {
+      Class<PartitionResolver> partitionResolverClass =
+          forName(regionCreateArgs.getPartitionArgs().getPartitionResolver(),
+              CliStrings.CREATE_REGION__PARTITION_RESOLVER);
       prAttrFactory
           .setPartitionResolver((PartitionResolver<K, V>) newInstance(partitionResolverClass,
               CliStrings.CREATE_REGION__PARTITION_RESOLVER));
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionFunctionArgs.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionFunctionArgs.java
index 3af79eb..5b278e7 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionFunctionArgs.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionFunctionArgs.java
@@ -19,14 +19,13 @@ import java.io.Serializable;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.LinkedHashSet;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.geode.cache.ExpirationAction;
 import org.apache.geode.cache.ExpirationAttributes;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 
 /**
@@ -36,167 +35,183 @@ import org.apache.geode.management.internal.cli.i18n.CliStrings;
  * @since GemFire 7.0
  */
 public class RegionFunctionArgs implements Serializable {
-  private static final long serialVersionUID = -5158224572470173267L;
-
-  private final String regionPath;
-  private final RegionShortcut regionShortcut;
-  private final String useAttributesFrom;
-  private final Boolean skipIfExists;
-  private final String keyConstraint;
-  private final String valueConstraint;
+  private static final long serialVersionUID = 2204943186081037301L;
+
+  private String regionPath;
+  private RegionShortcut regionShortcut;
+  private String templateRegion;
+  private boolean skipIfExists;
+  private String keyConstraint;
+  private String valueConstraint;
   private Boolean statisticsEnabled;
-  private final boolean isSetStatisticsEnabled;
-  private final RegionFunctionArgs.ExpirationAttrs entryExpirationIdleTime;
-  private final RegionFunctionArgs.ExpirationAttrs entryExpirationTTL;
-  private final RegionFunctionArgs.ExpirationAttrs regionExpirationIdleTime;
-  private final RegionFunctionArgs.ExpirationAttrs regionExpirationTTL;
-  private final String diskStore;
+  private RegionFunctionArgs.ExpirationAttrs entryExpirationIdleTime;
+  private RegionFunctionArgs.ExpirationAttrs entryExpirationTTL;
+  private RegionFunctionArgs.ExpirationAttrs regionExpirationIdleTime;
+  private RegionFunctionArgs.ExpirationAttrs regionExpirationTTL;
+  private String diskStore;
   private Boolean diskSynchronous;
-  private final boolean isSetDiskSynchronous;
   private Boolean enableAsyncConflation;
-  private final boolean isSetEnableAsyncConflation;
   private Boolean enableSubscriptionConflation;
-  private final boolean isSetEnableSubscriptionConflation;
-  private final Set<String> cacheListeners;
-  private final String cacheLoader;
-  private final String cacheWriter;
-  private final Set<String> asyncEventQueueIds;
-  private final Set<String> gatewaySenderIds;
+  private Set<String> cacheListeners = Collections.emptySet();
+  private String cacheLoader;
+  private String cacheWriter;
+  private Set<String> asyncEventQueueIds = Collections.emptySet();
+  private Set<String> gatewaySenderIds = Collections.emptySet();
   private Boolean concurrencyChecksEnabled;
-  private final boolean isSetConcurrencyChecksEnabled;
   private Boolean cloningEnabled;
-  private final boolean isSetCloningEnabled;
   private Boolean mcastEnabled;
-  private final boolean isSetMcastEnabled;
   private Integer concurrencyLevel;
-  private final boolean isSetConcurrencyLevel;
-  private final PartitionArgs partitionArgs;
-  private final Integer evictionMax;
+  private PartitionArgs partitionArgs;
+  private Integer evictionMax;
   private String compressor;
-  private final boolean isSetCompressor;
   private Boolean offHeap;
-  private final boolean isSetOffHeap;
   private RegionAttributes<?, ?> regionAttributes;
-  private final boolean isPartitionResolver;
-  private String partitionResolver;
-
-  public RegionFunctionArgs(String regionPath, RegionShortcut regionShortcut,
-      String useAttributesFrom, boolean skipIfExists, String keyConstraint, String valueConstraint,
-      Boolean statisticsEnabled, RegionFunctionArgs.ExpirationAttrs entryExpirationIdleTime,
-      RegionFunctionArgs.ExpirationAttrs entryExpirationTTL,
-      RegionFunctionArgs.ExpirationAttrs regionExpirationIdleTime,
-      RegionFunctionArgs.ExpirationAttrs regionExpirationTTL, String diskStore,
-      Boolean diskSynchronous, Boolean enableAsyncConflation, Boolean enableSubscriptionConflation,
-      String[] cacheListeners, String cacheLoader, String cacheWriter, String[] asyncEventQueueIds,
-      String[] gatewaySenderIds, Boolean concurrencyChecksEnabled, Boolean cloningEnabled,
-      Integer concurrencyLevel, String prColocatedWith, Integer prLocalMaxMemory,
-      Long prRecoveryDelay, Integer prRedundantCopies, Long prStartupRecoveryDelay,
-      Long prTotalMaxMemory, Integer prTotalNumBuckets, Integer evictionMax, String compressor,
-      Boolean offHeap, Boolean mcastEnabled, final String partitionResolver) {
+
+  public RegionFunctionArgs() {
+    this.partitionArgs = new PartitionArgs();
+  }
+
+  public void setRegionPath(String regionPath) {
     this.regionPath = regionPath;
+  }
+
+  public void setTemplateRegion(String templateRegion) {
+    this.templateRegion = templateRegion;
+  }
+
+  public void setRegionShortcut(RegionShortcut regionShortcut) {
     this.regionShortcut = regionShortcut;
-    this.useAttributesFrom = useAttributesFrom;
+  }
+
+
+  public void setSkipIfExists(boolean skipIfExists) {
     this.skipIfExists = skipIfExists;
+  }
+
+  public void setKeyConstraint(String keyConstraint) {
     this.keyConstraint = keyConstraint;
+  }
+
+  public void setValueConstraint(String valueConstraint) {
     this.valueConstraint = valueConstraint;
-    this.evictionMax = evictionMax;
-    this.isSetStatisticsEnabled = statisticsEnabled != null;
-    if (this.isSetStatisticsEnabled) {
-      this.statisticsEnabled = statisticsEnabled;
-    }
-    this.entryExpirationIdleTime = entryExpirationIdleTime;
-    this.entryExpirationTTL = entryExpirationTTL;
-    this.regionExpirationIdleTime = regionExpirationIdleTime;
-    this.regionExpirationTTL = regionExpirationTTL;
-    this.diskStore = diskStore;
-    this.isSetDiskSynchronous = diskSynchronous != null;
-    if (this.isSetDiskSynchronous) {
-      this.diskSynchronous = diskSynchronous;
+  }
+
+  public void setStatisticsEnabled(Boolean statisticsEnabled) {
+    this.statisticsEnabled = statisticsEnabled;
+  }
+
+  public void setEntryExpirationIdleTime(Integer timeout, String action) {
+    if (timeout != null) {
+      this.entryExpirationIdleTime = new ExpirationAttrs(
+          RegionFunctionArgs.ExpirationAttrs.ExpirationFor.ENTRY_IDLE, timeout, action);
+    }
+  }
+
+  public void setEntryExpirationTTL(Integer timeout, String action) {
+    if (timeout != null) {
+      this.entryExpirationTTL = new ExpirationAttrs(
+          RegionFunctionArgs.ExpirationAttrs.ExpirationFor.ENTRY_TTL, timeout, action);
     }
-    this.isSetEnableAsyncConflation = enableAsyncConflation != null;
-    if (this.isSetEnableAsyncConflation) {
-      this.enableAsyncConflation = enableAsyncConflation;
+  }
+
+  public void setRegionExpirationIdleTime(Integer timeout, String action) {
+    if (timeout != null) {
+      this.regionExpirationIdleTime = new ExpirationAttrs(
+          RegionFunctionArgs.ExpirationAttrs.ExpirationFor.REGION_IDLE, timeout, action);
     }
-    this.isSetEnableSubscriptionConflation = enableSubscriptionConflation != null;
-    if (this.isSetEnableSubscriptionConflation) {
-      this.enableSubscriptionConflation = enableSubscriptionConflation;
+  }
+
+  public void setRegionExpirationTTL(Integer timeout, String action) {
+    if (timeout != null) {
+      this.regionExpirationTTL = new ExpirationAttrs(
+          RegionFunctionArgs.ExpirationAttrs.ExpirationFor.REGION_TTL, timeout, action);
     }
+  }
+
+  public void setDiskStore(String diskStore) {
+    this.diskStore = diskStore;
+  }
+
+  public void setDiskSynchronous(Boolean diskSynchronous) {
+    this.diskSynchronous = diskSynchronous;
+  }
+
+  public void setEnableAsyncConflation(Boolean enableAsyncConflation) {
+    this.enableAsyncConflation = enableAsyncConflation;
+  }
+
+  public void setEnableSubscriptionConflation(Boolean enableSubscriptionConflation) {
+    this.enableSubscriptionConflation = enableSubscriptionConflation;
+  }
+
+  public void setCacheListeners(String[] cacheListeners) {
     if (cacheListeners != null) {
-      this.cacheListeners = new LinkedHashSet<>();
-      this.cacheListeners.addAll(Arrays.asList(cacheListeners));
-    } else {
-      this.cacheListeners = null;
+      this.cacheListeners = Arrays.stream(cacheListeners).collect(Collectors.toSet());
     }
+  }
+
+  public void setCacheLoader(String cacheLoader) {
     this.cacheLoader = cacheLoader;
+  }
+
+  public void setCacheWriter(String cacheWriter) {
     this.cacheWriter = cacheWriter;
+  }
+
+  public void setAsyncEventQueueIds(String[] asyncEventQueueIds) {
     if (asyncEventQueueIds != null) {
-      this.asyncEventQueueIds = new LinkedHashSet<>();
-      this.asyncEventQueueIds.addAll(Arrays.asList(asyncEventQueueIds));
-    } else {
-      this.asyncEventQueueIds = null;
+      this.asyncEventQueueIds = Arrays.stream(asyncEventQueueIds).collect(Collectors.toSet());
     }
+  }
+
+  public void setGatewaySenderIds(String[] gatewaySenderIds) {
     if (gatewaySenderIds != null) {
-      this.gatewaySenderIds = new LinkedHashSet<>();
-      this.gatewaySenderIds.addAll(Arrays.asList(gatewaySenderIds));
-    } else {
-      this.gatewaySenderIds = null;
-    }
-    this.isSetConcurrencyChecksEnabled = concurrencyChecksEnabled != null;
-    if (this.isSetConcurrencyChecksEnabled) {
-      this.concurrencyChecksEnabled = concurrencyChecksEnabled;
-    }
-    this.isSetCloningEnabled = cloningEnabled != null;
-    if (this.isSetCloningEnabled) {
-      this.cloningEnabled = cloningEnabled;
-    }
-    this.isSetMcastEnabled = mcastEnabled != null;
-    if (isSetMcastEnabled) {
-      this.mcastEnabled = mcastEnabled;
-    }
-    this.isSetConcurrencyLevel = concurrencyLevel != null;
-    if (this.isSetConcurrencyLevel) {
-      this.concurrencyLevel = concurrencyLevel;
-    }
-    this.partitionArgs =
-        new PartitionArgs(prColocatedWith, prLocalMaxMemory, prRecoveryDelay, prRedundantCopies,
-            prStartupRecoveryDelay, prTotalMaxMemory, prTotalNumBuckets, partitionResolver);
-
-    this.isSetCompressor = (compressor != null);
-    if (this.isSetCompressor) {
-      this.compressor = compressor;
-    }
-    this.isSetOffHeap = (offHeap != null);
-    if (this.isSetOffHeap) {
-      this.offHeap = offHeap;
-    }
-    this.isPartitionResolver = (partitionResolver != null);
-    if (this.isPartitionResolver) {
-      this.partitionResolver = partitionResolver;
-    }
-  }
-
-  // Constructor to be used for supplied region attributes
-  public RegionFunctionArgs(String regionPath, String useAttributesFrom, boolean skipIfExists,
-      String keyConstraint, String valueConstraint, Boolean statisticsEnabled,
-      RegionFunctionArgs.ExpirationAttrs entryExpirationIdleTime,
-      RegionFunctionArgs.ExpirationAttrs entryExpirationTTL,
-      RegionFunctionArgs.ExpirationAttrs regionExpirationIdleTime,
-      RegionFunctionArgs.ExpirationAttrs regionExpirationTTL, String diskStore,
-      Boolean diskSynchronous, Boolean enableAsyncConflation, Boolean enableSubscriptionConflation,
-      String[] cacheListeners, String cacheLoader, String cacheWriter, String[] asyncEventQueueIds,
-      String[] gatewaySenderIds, Boolean concurrencyChecksEnabled, Boolean cloningEnabled,
-      Integer concurrencyLevel, String prColocatedWith, Integer prLocalMaxMemory,
+      this.gatewaySenderIds = Arrays.stream(gatewaySenderIds).collect(Collectors.toSet());
+    }
+  }
+
+  public void setConcurrencyChecksEnabled(Boolean concurrencyChecksEnabled) {
+    this.concurrencyChecksEnabled = concurrencyChecksEnabled;
+  }
+
+  public void setCloningEnabled(Boolean cloningEnabled) {
+    this.cloningEnabled = cloningEnabled;
+  }
+
+  public void setMcastEnabled(Boolean mcastEnabled) {
+    this.mcastEnabled = mcastEnabled;
+  }
+
+  public void setConcurrencyLevel(Integer concurrencyLevel) {
+    this.concurrencyLevel = concurrencyLevel;
+  }
+
+  public void setPartitionArgs(String prColocatedWith, Integer prLocalMaxMemory,
       Long prRecoveryDelay, Integer prRedundantCopies, Long prStartupRecoveryDelay,
-      Long prTotalMaxMemory, Integer prTotalNumBuckets, Boolean offHeap, Boolean mcastEnabled,
-      RegionAttributes<?, ?> regionAttributes, final String partitionResolver) {
-    this(regionPath, null, useAttributesFrom, skipIfExists, keyConstraint, valueConstraint,
-        statisticsEnabled, entryExpirationIdleTime, entryExpirationTTL, regionExpirationIdleTime,
-        regionExpirationTTL, diskStore, diskSynchronous, enableAsyncConflation,
-        enableSubscriptionConflation, cacheListeners, cacheLoader, cacheWriter, asyncEventQueueIds,
-        gatewaySenderIds, concurrencyChecksEnabled, cloningEnabled, concurrencyLevel,
-        prColocatedWith, prLocalMaxMemory, prRecoveryDelay, prRedundantCopies,
-        prStartupRecoveryDelay, prTotalMaxMemory, prTotalNumBuckets, null, null, offHeap,
-        mcastEnabled, partitionResolver);
+      Long prTotalMaxMemory, Integer prTotalNumBuckets, String partitionResolver) {
+    partitionArgs.setPrColocatedWith(prColocatedWith);
+    partitionArgs.setPrLocalMaxMemory(prLocalMaxMemory);
+    partitionArgs.setPrRecoveryDelay(prRecoveryDelay);
+    partitionArgs.setPrRedundantCopies(prRedundantCopies);
+    partitionArgs.setPrStartupRecoveryDelay(prStartupRecoveryDelay);
+    partitionArgs.setPrTotalMaxMemory(prTotalMaxMemory);
+    partitionArgs.setPrTotalNumBuckets(prTotalNumBuckets);
+    partitionArgs.setPartitionResolver(partitionResolver);
+  }
+
+  public void setEvictionMax(Integer evictionMax) {
+    this.evictionMax = evictionMax;
+  }
+
+  public void setCompressor(String compressor) {
+    this.compressor = compressor;
+  }
+
+  public void setOffHeap(Boolean offHeap) {
+    this.offHeap = offHeap;
+  }
+
+  public void setRegionAttributes(RegionAttributes<?, ?> regionAttributes) {
     this.regionAttributes = regionAttributes;
   }
 
@@ -215,18 +230,10 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the useAttributesFrom
-   */
-  public String getUseAttributesFrom() {
-    return this.useAttributesFrom;
-  }
-
-  /**
-   * @return true if need to use specified region attributes
+   * @return the templateRegion
    */
-  public Boolean isSetUseAttributesFrom() {
-    return this.regionShortcut == null && this.useAttributesFrom != null
-        && this.regionAttributes != null;
+  public String getTemplateRegion() {
+    return this.templateRegion;
   }
 
   /**
@@ -258,13 +265,6 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the isSetStatisticsEnabled
-   */
-  public Boolean isSetStatisticsEnabled() {
-    return this.isSetStatisticsEnabled;
-  }
-
-  /**
    * @return the entryExpirationIdleTime
    */
   public RegionFunctionArgs.ExpirationAttrs getEntryExpirationIdleTime() {
@@ -306,21 +306,10 @@ public class RegionFunctionArgs implements Serializable {
     return this.diskSynchronous;
   }
 
-  /**
-   * @return the isSetDiskSynchronous
-   */
-  public Boolean isSetDiskSynchronous() {
-    return this.isSetDiskSynchronous;
-  }
-
   public Boolean isOffHeap() {
     return this.offHeap;
   }
 
-  public Boolean isSetOffHeap() {
-    return this.isSetOffHeap;
-  }
-
   /**
    * @return the enableAsyncConflation
    */
@@ -329,13 +318,6 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the isSetEnableAsyncConflation
-   */
-  public Boolean isSetEnableAsyncConflation() {
-    return this.isSetEnableAsyncConflation;
-  }
-
-  /**
    * @return the enableSubscriptionConflation
    */
   public Boolean isEnableSubscriptionConflation() {
@@ -343,13 +325,6 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the isSetEnableSubscriptionConflation
-   */
-  public Boolean isSetEnableSubscriptionConflation() {
-    return this.isSetEnableSubscriptionConflation;
-  }
-
-  /**
    * @return the cacheListeners
    */
   public Set<String> getCacheListeners() {
@@ -394,20 +369,6 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the PartitionResolver
-   */
-  public String getPartitionResolver() {
-    return this.partitionResolver;
-  }
-
-  /**
-   * @return True if Partition Resolver is set otherwise False
-   */
-  public Boolean isPartitionResolverSet() {
-    return this.isPartitionResolver;
-  }
-
-  /**
    * @return the concurrencyChecksEnabled
    */
   public Boolean isConcurrencyChecksEnabled() {
@@ -415,13 +376,6 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the isSetConcurrencyChecksEnabled
-   */
-  public Boolean isSetConcurrencyChecksEnabled() {
-    return this.isSetConcurrencyChecksEnabled;
-  }
-
-  /**
    * @return the cloningEnabled
    */
   public Boolean isCloningEnabled() {
@@ -429,13 +383,6 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the isSetCloningEnabled
-   */
-  public Boolean isSetCloningEnabled() {
-    return this.isSetCloningEnabled;
-  }
-
-  /**
    * @return the mcastEnabled setting
    */
   public Boolean isMcastEnabled() {
@@ -443,26 +390,12 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the isSetCloningEnabled
-   */
-  public Boolean isSetMcastEnabled() {
-    return this.isSetMcastEnabled;
-  }
-
-  /**
    * @return the concurrencyLevel
    */
   public Integer getConcurrencyLevel() {
     return this.concurrencyLevel;
   }
 
-  /**
-   * @return the isSetConcurrencyLevel
-   */
-  public Boolean isSetConcurrencyLevel() {
-    return this.isSetConcurrencyLevel;
-  }
-
   public boolean withPartitioning() {
     return hasPartitionAttributes()
         || (this.regionShortcut != null && this.regionShortcut.name().startsWith("PARTITION"));
@@ -472,7 +405,7 @@ public class RegionFunctionArgs implements Serializable {
    * @return the partitionArgs
    */
   public boolean hasPartitionAttributes() {
-    return this.partitionArgs != null && this.partitionArgs.hasPartitionAttributes();
+    return this.partitionArgs.hasPartitionAttributes();
   }
 
   /**
@@ -497,13 +430,6 @@ public class RegionFunctionArgs implements Serializable {
   }
 
   /**
-   * @return the isSetCompressor.
-   */
-  public boolean isSetCompressor() {
-    return this.isSetCompressor;
-  }
-
-  /**
    * @return the regionAttributes
    */
   @SuppressWarnings("unchecked")
@@ -514,26 +440,20 @@ public class RegionFunctionArgs implements Serializable {
   public static class ExpirationAttrs implements Serializable {
     private static final long serialVersionUID = 1474255033398008062L;
 
-    private ExpirationFor type;
-    private Integer time;
-    private ExpirationAction action;
+    private final ExpirationFor type;
+    private final ExpirationAttributes timeAndAction;
 
     public ExpirationAttrs(ExpirationFor type, Integer time, String action) {
       this.type = type;
-      this.time = time;
-      if (action != null) {
-        this.action = getExpirationAction(action);
+      if (time != null) {
+        this.timeAndAction = new ExpirationAttributes(time, getExpirationAction(action));
+      } else {
+        this.timeAndAction = new ExpirationAttributes(0, getExpirationAction(action));
       }
     }
 
     public ExpirationAttributes convertToExpirationAttributes() {
-      ExpirationAttributes expirationAttr;
-      if (action != null) {
-        expirationAttr = new ExpirationAttributes(time, action);
-      } else {
-        expirationAttr = new ExpirationAttributes(time);
-      }
-      return expirationAttr;
+      return timeAndAction;
     }
 
     /**
@@ -547,14 +467,14 @@ public class RegionFunctionArgs implements Serializable {
      * @return the time
      */
     public Integer getTime() {
-      return time;
+      return timeAndAction.getTimeout();
     }
 
     /**
      * @return the action
      */
     public ExpirationAction getAction() {
-      return action;
+      return timeAndAction.getAction();
     }
 
     @Override
@@ -563,9 +483,9 @@ public class RegionFunctionArgs implements Serializable {
       builder.append(ExpirationAttrs.class.getSimpleName() + " [type=");
       builder.append(type);
       builder.append(", time=");
-      builder.append(time);
+      builder.append(timeAndAction.getTimeout());
       builder.append(", action=");
-      builder.append(action);
+      builder.append(timeAndAction.getAction());
       builder.append("]");
       return builder.toString();
     }
@@ -598,90 +518,104 @@ public class RegionFunctionArgs implements Serializable {
   public static class PartitionArgs implements Serializable {
     private static final long serialVersionUID = 5907052187323280919L;
 
-    private final String prColocatedWith;
-    private int prLocalMaxMemory;
-    private final boolean isSetPRLocalMaxMemory;
-    private long prRecoveryDelay;
-    private final boolean isSetPRRecoveryDelay;
-    private int prRedundantCopies;
-    private final boolean isSetPRRedundantCopies;
-    private long prStartupRecoveryDelay;
-    private final boolean isSetPRStartupRecoveryDelay;
-    private long prTotalMaxMemory;
-    private final boolean isSetPRTotalMaxMemory;
-    private int prTotalNumBuckets;
-    private final boolean isSetPRTotalNumBuckets;
-    private final boolean isPartitionResolver;
+    private String prColocatedWith;
+    private Integer prLocalMaxMemory;
+    private Long prRecoveryDelay;
+    private Integer prRedundantCopies;
+    private Long prStartupRecoveryDelay;
+    private Long prTotalMaxMemory;
+    private Integer prTotalNumBuckets;
     private String partitionResolver;
 
-    private boolean hasPartitionAttributes;
-    private final Set<String> userSpecifiedPartitionAttributes = new HashSet<>();
+    public PartitionArgs() {}
+
+    /**
+     * @return the hasPartitionAttributes
+     */
+    public Boolean hasPartitionAttributes() {
+      return !getUserSpecifiedPartitionAttributes().isEmpty();
+    }
+
+    /**
+     * @return the userSpecifiedPartitionAttributes
+     */
+    public Set<String> getUserSpecifiedPartitionAttributes() {
+      Set<String> userSpecifiedPartitionAttributes = new HashSet<>();
 
-    public PartitionArgs(String prColocatedWith, Integer prLocalMaxMemory, Long prRecoveryDelay,
-        Integer prRedundantCopies, Long prStartupRecoveryDelay, Long prTotalMaxMemory,
-        Integer prTotalNumBuckets, String partitionResolver) {
-      this.prColocatedWith = prColocatedWith;
       if (this.prColocatedWith != null) {
-        this.hasPartitionAttributes = true;
         userSpecifiedPartitionAttributes.add(CliStrings.CREATE_REGION__COLOCATEDWITH);
       }
-      this.isSetPRLocalMaxMemory = prLocalMaxMemory != null;
-      if (this.isSetPRLocalMaxMemory) {
-        this.prLocalMaxMemory = prLocalMaxMemory;
-        this.hasPartitionAttributes = true;
+      if (this.prLocalMaxMemory != null) {
         userSpecifiedPartitionAttributes.add(CliStrings.CREATE_REGION__LOCALMAXMEMORY);
       }
-      this.isSetPRRecoveryDelay = prRecoveryDelay != null;
-      if (this.isSetPRRecoveryDelay) {
-        this.prRecoveryDelay = prRecoveryDelay;
-        this.hasPartitionAttributes = true;
+      if (this.prRecoveryDelay != null) {
         userSpecifiedPartitionAttributes.add(CliStrings.CREATE_REGION__RECOVERYDELAY);
       }
-      this.isSetPRRedundantCopies = prRedundantCopies != null;
-      if (this.isSetPRRedundantCopies) {
-        this.prRedundantCopies = prRedundantCopies;
-        this.hasPartitionAttributes = true;
+      if (this.prRedundantCopies != null) {
         userSpecifiedPartitionAttributes.add(CliStrings.CREATE_REGION__REDUNDANTCOPIES);
       }
-      this.isSetPRStartupRecoveryDelay = prStartupRecoveryDelay != null;
-      if (this.isSetPRStartupRecoveryDelay) {
-        this.prStartupRecoveryDelay = prStartupRecoveryDelay;
-        this.hasPartitionAttributes = true;
+      if (this.prStartupRecoveryDelay != null) {
         userSpecifiedPartitionAttributes.add(CliStrings.CREATE_REGION__STARTUPRECOVERYDDELAY);
       }
-      this.isSetPRTotalMaxMemory = prTotalMaxMemory != null;
-      if (this.isSetPRTotalMaxMemory) {
-        this.prTotalMaxMemory = prTotalMaxMemory;
-        this.hasPartitionAttributes = true;
+      if (this.prTotalMaxMemory != null) {
         userSpecifiedPartitionAttributes.add(CliStrings.CREATE_REGION__TOTALMAXMEMORY);
       }
-      this.isSetPRTotalNumBuckets = prTotalNumBuckets != null;
-      if (this.isSetPRTotalNumBuckets) {
-        this.prTotalNumBuckets = prTotalNumBuckets;
-        this.hasPartitionAttributes = true;
+      if (this.prTotalNumBuckets != null) {
         userSpecifiedPartitionAttributes.add(CliStrings.CREATE_REGION__TOTALNUMBUCKETS);
       }
-      this.isPartitionResolver = partitionResolver != null;
-      if (this.isPartitionResolver) {
-        this.partitionResolver = partitionResolver;
-        this.hasPartitionAttributes = true;
+      if (this.partitionResolver != null) {
         userSpecifiedPartitionAttributes.add(CliStrings.CREATE_REGION__PARTITION_RESOLVER);
       }
 
+      return userSpecifiedPartitionAttributes;
     }
 
-    /**
-     * @return the hasPartitionAttributes
-     */
-    public Boolean hasPartitionAttributes() {
-      return hasPartitionAttributes;
+    public void setPrColocatedWith(String prColocatedWith) {
+      if (prColocatedWith != null) {
+        this.prColocatedWith = prColocatedWith;
+      }
     }
 
-    /**
-     * @return the userSpecifiedPartitionAttributes
-     */
-    public String getUserSpecifiedPartitionAttributes() {
-      return CliUtil.collectionToString(userSpecifiedPartitionAttributes, -1);
+    public void setPrLocalMaxMemory(Integer prLocalMaxMemory) {
+      if (prLocalMaxMemory != null) {
+        this.prLocalMaxMemory = prLocalMaxMemory;
+      }
+    }
+
+    public void setPrRecoveryDelay(Long prRecoveryDelay) {
+      if (prRecoveryDelay != null) {
+        this.prRecoveryDelay = prRecoveryDelay;
+      }
+    }
+
+    public void setPrRedundantCopies(Integer prRedundantCopies) {
+      if (prRedundantCopies != null) {
+        this.prRedundantCopies = prRedundantCopies;
+      }
+    }
+
+    public void setPrStartupRecoveryDelay(Long prStartupRecoveryDelay) {
+      if (prStartupRecoveryDelay != null) {
+        this.prStartupRecoveryDelay = prStartupRecoveryDelay;
+      }
+    }
+
+    public void setPrTotalMaxMemory(Long prTotalMaxMemory) {
+      if (prTotalMaxMemory != null) {
+        this.prTotalMaxMemory = prTotalMaxMemory;
+      }
+    }
+
+    public void setPrTotalNumBuckets(Integer prTotalNumBuckets) {
+      if (prTotalNumBuckets != null) {
+        this.prTotalNumBuckets = prTotalNumBuckets;
+      }
+    }
+
+    public void setPartitionResolver(String partitionResolver) {
+      if (partitionResolver != null) {
+        this.partitionResolver = partitionResolver;
+      }
     }
 
     /**
@@ -699,13 +633,6 @@ public class RegionFunctionArgs implements Serializable {
     }
 
     /**
-     * @return the isSetPRLocalMaxMemory
-     */
-    public Boolean isSetPRLocalMaxMemory() {
-      return isSetPRLocalMaxMemory;
-    }
-
-    /**
      * @return the prRecoveryDelay
      */
     public Long getPrRecoveryDelay() {
@@ -713,13 +640,6 @@ public class RegionFunctionArgs implements Serializable {
     }
 
     /**
-     * @return the isSetPRRecoveryDelay
-     */
-    public Boolean isSetPRRecoveryDelay() {
-      return isSetPRRecoveryDelay;
-    }
-
-    /**
      * @return the prRedundantCopies
      */
     public Integer getPrRedundantCopies() {
@@ -727,13 +647,6 @@ public class RegionFunctionArgs implements Serializable {
     }
 
     /**
-     * @return the isSetPRRedundantCopies
-     */
-    public Boolean isSetPRRedundantCopies() {
-      return isSetPRRedundantCopies;
-    }
-
-    /**
      * @return the prStartupRecoveryDelay
      */
     public Long getPrStartupRecoveryDelay() {
@@ -741,13 +654,6 @@ public class RegionFunctionArgs implements Serializable {
     }
 
     /**
-     * @return the isSetPRStartupRecoveryDelay
-     */
-    public Boolean isSetPRStartupRecoveryDelay() {
-      return isSetPRStartupRecoveryDelay;
-    }
-
-    /**
      * @return the prTotalMaxMemory
      */
     public Long getPrTotalMaxMemory() {
@@ -755,13 +661,6 @@ public class RegionFunctionArgs implements Serializable {
     }
 
     /**
-     * @return the isSetPRTotalMaxMemory
-     */
-    public Boolean isSetPRTotalMaxMemory() {
-      return isSetPRTotalMaxMemory;
-    }
-
-    /**
      * @return the prTotalNumBuckets
      */
     public Integer getPrTotalNumBuckets() {
@@ -769,10 +668,10 @@ public class RegionFunctionArgs implements Serializable {
     }
 
     /**
-     * @return the isSetPRTotalNumBuckets
+     * @return the partition resolver
      */
-    public Boolean isSetPRTotalNumBuckets() {
-      return isSetPRTotalNumBuckets;
+    public String getPartitionResolver() {
+      return partitionResolver;
     }
   }
 }
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 22956ee..0ea9764 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
@@ -45,8 +45,8 @@ public class CommandExecutor {
       throw e;
     } catch (Exception e) {
       logger.error("Could not execute \"" + parseResult + "\".", e);
-      return ResultBuilder.createGemFireErrorResult("Error while processing command <" + parseResult
-          + "> Reason : " + ExceptionUtils.getStackTrace(e));
+      return ResultBuilder.createGemFireErrorResult(
+          "Error while processing command <" + parseResult + "> Reason : " + e.getMessage());
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/RegionProvider.java b/geode-core/src/main/java/org/apache/geode/redis/internal/RegionProvider.java
index 86478a3..d41588e 100644
--- a/geode-core/src/main/java/org/apache/geode/redis/internal/RegionProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/RegionProvider.java
@@ -400,9 +400,10 @@ public class RegionProvider implements Closeable {
       return r;
     do {
       Result result = createRegionCmd.createRegion(key, defaultRegionType, null, null, true, null,
-          null, null, null, null, null, null, null, null, null, null, null, null, null, null, null,
-          null, null, null, null, null, null, null, null, null, null, null, null, null, null, null,
-          null, null, null);
+          null, null, null, null, null, null, null, false, false, true, false, false, false, true,
+          null, null, null, null, null, null, null, false, null, null, null, null, null, null, null,
+          null, null, null, null);
+
       r = cache.getRegion(key);
       if (result.getStatus() == Status.ERROR && r == null) {
         String err = "";
diff --git a/geode-core/src/test/java/org/apache/geode/cache/ExpirationAttributesTest.java b/geode-core/src/test/java/org/apache/geode/cache/ExpirationAttributesTest.java
new file mode 100644
index 0000000..8ebe205
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/cache/ExpirationAttributesTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ExpirationAttributesTest {
+  @Test
+  public void constructor() throws Exception {
+    ExpirationAttributes attributes = new ExpirationAttributes();
+    assertThat(attributes.getTimeout()).isEqualTo(0);
+    assertThat(attributes.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+
+    attributes = new ExpirationAttributes(-10, null);
+    assertThat(attributes.getTimeout()).isEqualTo(0);
+    assertThat(attributes.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+
+    attributes = new ExpirationAttributes(10);
+    assertThat(attributes.getTimeout()).isEqualTo(10);
+    assertThat(attributes.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+
+    attributes = new ExpirationAttributes(10, null);
+    assertThat(attributes.getTimeout()).isEqualTo(10);
+    assertThat(attributes.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+
+    attributes = new ExpirationAttributes(20, ExpirationAction.DESTROY);
+    assertThat(attributes.getTimeout()).isEqualTo(20);
+    assertThat(attributes.getAction()).isEqualTo(ExpirationAction.DESTROY);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
index 6b6469d..21444e2 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
@@ -16,6 +16,7 @@ package org.apache.geode.management.internal.cli.commands;
 
 import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
 import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -24,10 +25,12 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.stream.Collectors;
 
 import org.junit.Ignore;
 import org.junit.Test;
@@ -201,12 +204,11 @@ public class AlterRegionCommandDUnitTest extends CliCommandTestBase {
     this.alterVm1.invoke(() -> {
       RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
       assertEquals(3, attributes.getCacheListeners().length);
-      assertEquals("com.cadrdunit.RegionAlterCacheListenerA",
-          attributes.getCacheListeners()[0].getClass().getName());
-      assertEquals("com.cadrdunit.RegionAlterCacheListenerB",
-          attributes.getCacheListeners()[1].getClass().getName());
-      assertEquals("com.cadrdunit.RegionAlterCacheListenerC",
-          attributes.getCacheListeners()[2].getClass().getName());
+
+      assertThat(Arrays.stream(attributes.getCacheListeners()).map(c -> c.getClass().getName())
+          .collect(Collectors.toSet())).containsExactlyInAnyOrder(
+              "com.cadrdunit.RegionAlterCacheListenerA", "com.cadrdunit.RegionAlterCacheListenerB",
+              "com.cadrdunit.RegionAlterCacheListenerC");
     });
 
     // Add 1 back to each of the sets
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandIntegrationTest.java
new file mode 100644
index 0000000..d312c26
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandIntegrationTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.apache.geode.test.junit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.junit.rules.ServerStarterRule;
+
+@Category(IntegrationTest.class)
+public class AlterRegionCommandIntegrationTest {
+  @ClassRule
+  public static ServerStarterRule server =
+      new ServerStarterRule().withJMXManager().withRegion(RegionShortcut.REPLICATE, "REPLICATED");
+
+  @ClassRule
+  public static GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @BeforeClass
+  public static void before() throws Exception {
+    gfsh.connectAndVerify(server.getJmxPort(), GfshShellConnectionRule.PortType.jmxManager);
+  }
+
+  @Test
+  public void validateGroup() throws Exception {
+    gfsh.executeAndVerifyCommandError("alter region --name=/REPLICATED --group=unknown",
+        "Group\\(s\\) \"\\[unknown\\]\" are invalid.");
+  }
+
+  @Test
+  public void invalidCacheListener() throws Exception {
+    gfsh.executeAndVerifyCommandError("alter region --name=/REPLICATED --cache-listener=abc-def",
+        "Specify a valid class name for cache-listener");
+  }
+
+  @Test
+  public void invalidCacheLoader() throws Exception {
+    gfsh.executeAndVerifyCommandError("alter region --name=/REPLICATED --cache-loader=abc-def",
+        "Specify a valid class name for cache-loader");
+  }
+
+  @Test
+  public void invalidCacheWriter() throws Exception {
+    gfsh.executeAndVerifyCommandError("alter region --name=/REPLICATED --cache-writer=abc-def",
+        "Specify a valid class name for cache-writer");
+  }
+
+  @Test
+  public void invalidEvictionMax() throws Exception {
+    gfsh.executeAndVerifyCommandError("alter region --name=/REPLICATED --eviction-max=-1",
+        "Specify 0 or a positive integer value for eviction-max");
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandDUnitTest.java
index 41155a0..d6d8ac6 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandDUnitTest.java
@@ -149,7 +149,7 @@ public class CreateRegionCommandDUnitTest {
         + " --type=REPLICATE --partition-resolver=InvalidPartitionResolver");
 
     assertThat(((String) ((JSONArray) result.getContent().get("message")).get(0)))
-        .contains("\"" + regionName + "\" is not a Partitioned Region");
+        .contains("\"/" + regionName + "\" is not a Partitioned Region");
   }
 
   private void writeJarBytesToFile(File jarFile, byte[] jarBytes) throws IOException {
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandIntegrationTest.java
new file mode 100644
index 0000000..391fd91
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandIntegrationTest.java
@@ -0,0 +1,493 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+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.RegionShortcut;
+import org.apache.geode.cache.util.CacheListenerAdapter;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.apache.geode.test.junit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.junit.rules.ServerStarterRule;
+
+@Category(IntegrationTest.class)
+public class CreateRegionCommandIntegrationTest {
+
+  private static String CREATE_REGION = "create region --type=REPLICATE ";
+
+  public static class TestCacheListener extends CacheListenerAdapter {
+  }
+
+  public static class TestConstraint {
+  }
+
+  @ClassRule
+  public static ServerStarterRule server =
+      new ServerStarterRule().withJMXManager().withRegion(RegionShortcut.REPLICATE, "REPLICATED");
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @Before
+  public void before() throws Exception {
+    gfsh.connectAndVerify(server.getJmxPort(), GfshShellConnectionRule.PortType.jmxManager);
+  }
+
+  @Test
+  public void parentRegionDoesNotExist() throws Exception {
+    gfsh.executeAndVerifyCommandError(CREATE_REGION + "--name=/A/B",
+        "Parent region for \"/A/B\" doesnt exist");
+  }
+
+  @Test
+  public void groupDoesNotExist() throws Exception {
+    gfsh.executeAndVerifyCommandError(CREATE_REGION + "--name=/FOO --groups=unknown",
+        "Group\\(s\\) .* are invalid");
+  }
+
+  @Test
+  public void templateRegionDoesNotExist() throws Exception {
+    gfsh.executeAndVerifyCommandError("create region --name=/FOO --template-region=/BAR",
+        "Specify a valid region path for template-region");
+  }
+
+  @Test
+  public void conflictingPartitionAttributesWithTemplate() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --template-region=REPLICATED --redundant-copies=2",
+        "Parameter\\(s\\) \"\\[redundant-copies\\]\" can be used only for creating a Partitioned Region");
+  }
+
+  @Test
+  public void conflictingPartitionAttributesWithShortCut() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --redundant-copies=2",
+        "Parameter\\(s\\) \"\\[redundant-copies\\]\" can be used only for creating a Partitioned Region");
+  }
+
+  @Test
+  public void colocatedWithRegionDoesNotExist() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --type=PARTITION --name=/FOO --colocated-with=/BAR",
+        "Specify a valid region path for colocated-with");
+  }
+
+  @Test
+  public void colocatedWithRegionIsNotPartitioned() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --type=PARTITION --name=/FOO --colocated-with=/REPLICATED",
+        "colocated-with \"/REPLICATED\" is not a Partitioned Region");
+  }
+
+  @Test
+  public void negativeLocalMaxMemory() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --type=PARTITION --name=/FOO --local-max-memory=-1",
+        "PartitionAttributes localMaxMemory must not be negative");
+  }
+
+  @Test
+  public void zeroLocalMaxMemoryIsOK() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --type=PARTITION --name=/FOO --local-max-memory=0",
+        "Region \"/FOO\" created");
+    gfsh.executeAndVerifyCommand("destroy region --name=/FOO");
+  }
+
+  @Test
+  public void negativeTotalMaxMemory() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --type=PARTITION --name=/FOO --total-max-memory=-1",
+        "Total size of partition region must be > 0");
+  }
+
+  @Test
+  public void zeroTotalMaxMemory() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --type=PARTITION --name=/FOO --total-max-memory=0",
+        "Total size of partition region must be > 0");
+  }
+
+  @Test
+  public void redundantCopies() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/FOO --type=PARTITION --redundant-copies=2",
+        "Region \"/FOO\" created");
+    gfsh.executeAndVerifyCommand("destroy region --name=/FOO");
+  }
+
+  @Test
+  public void tooManyredundantCopies() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=PARTITION --redundant-copies=4",
+        "redundant-copies \"4\" is not valid");
+  }
+
+  @Test
+  public void keyConstraint() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --key-constraint=abc-def",
+        "Specify a valid class name for key-constraint");
+  }
+
+  @Test
+  public void valueConstraint() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --value-constraint=abc-def",
+        "Specify a valid class name for value-constraint");
+  }
+
+  @Test
+  public void invalidCacheListener() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --cache-listener=abc-def",
+        "Specify a valid class name for cache-listener");
+  }
+
+  @Test
+  public void invalidCacheLoader() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --cache-loader=abc-def",
+        "Specify a valid class name for cache-loader");
+  }
+
+  @Test
+  public void invalidCacheWriter() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --cache-writer=abc-def",
+        "Specify a valid class name for cache-writer");
+  }
+
+  @Test
+  public void invalidGatewaySenders() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --gateway-sender-id=unknown",
+        "There are no GatewaySenders defined currently in the system");
+  }
+
+  // TODO: Write test for invalid gateway name (gateways already need to exist).
+
+  @Test
+  public void invalidConcurrencyLevel() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --template-region=/REPLICATED --concurrency-level=-1",
+        "Specify positive integer value for concurrency-level");
+  }
+
+  @Test
+  public void nonPersistentRegionWithdiskStore() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --disk-store=unknown",
+        "Only regions with persistence or overflow to disk can specify DiskStore");
+  }
+
+  @Test
+  public void nonPersistentTemplateWithdiskStore() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --template-region=/REPLICATED --disk-store=unknown",
+        "Only regions with persistence or overflow to disk can specify DiskStore",
+        "template-region region \"/REPLICATED\" is not persistent");
+  }
+
+
+  @Test
+  public void invalidDiskStore() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE_PERSISTENT --disk-store=unknown",
+        "Specify valid disk-store. Unknown Disk Store : \"unknown\"");
+  }
+
+  @Test
+  public void entryIdleTimeWithoutStatisticsEnabled() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --entry-idle-time-expiration=1",
+        "Statistics must be enabled for expiration");
+  }
+
+  @Test
+  public void invalidCompressor() throws Exception {
+    gfsh.executeAndVerifyCommandError(
+        "create region --name=/FOO --type=REPLICATE --compressor=java.lang.String",
+        "java.lang.String cannot be cast to org.apache.geode.compression.Compressor");
+  }
+
+  @Test
+  public void validateDefaultExpirationAttributes() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/A --type=REPLICATE");
+
+    Region region = server.getCache().getRegion("/A");
+    RegionAttributes attributes = region.getAttributes();
+    ExpirationAttributes entryIdle = attributes.getEntryIdleTimeout();
+    ExpirationAttributes entryTTL = attributes.getEntryTimeToLive();
+    ExpirationAttributes regionIdle = attributes.getRegionIdleTimeout();
+    ExpirationAttributes regionTTL = attributes.getRegionTimeToLive();
+
+    assertThat(entryIdle).isNotNull();
+    assertThat(entryIdle.getTimeout()).isEqualTo(0);
+    assertThat(entryIdle.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+    assertThat(entryTTL).isNotNull();
+    assertThat(entryTTL.getTimeout()).isEqualTo(0);
+    assertThat(entryTTL.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+    assertThat(regionIdle).isNotNull();
+    assertThat(regionIdle.getTimeout()).isEqualTo(0);
+    assertThat(regionIdle.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+    assertThat(regionTTL).isNotNull();
+    assertThat(regionTTL.getTimeout()).isEqualTo(0);
+    assertThat(regionTTL.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/A");
+  }
+
+  @Test
+  public void validateNonDefaultBinaryOptions() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/FOO --type=REPLICATE"
+        + " --enable-async-conflation" + " --enable-cloning" + " --enable-concurrency-checks=false"
+        + " --enable-multicast" + " --enable-statistics" + " --enable-subscription-conflation"
+        + " --enable-synchronous-disk=false");
+
+    Region foo = server.getCache().getRegion("/FOO");
+
+    assertThat(foo.getAttributes().getEnableAsyncConflation()).isTrue();
+    assertThat(foo.getAttributes().getCloningEnabled()).isTrue();
+    assertThat(foo.getAttributes().getConcurrencyChecksEnabled()).isFalse();
+    assertThat(foo.getAttributes().getMulticastEnabled()).isTrue();
+    assertThat(foo.getAttributes().getStatisticsEnabled()).isTrue();
+    assertThat(foo.getAttributes().getEnableSubscriptionConflation()).isTrue();
+    assertThat(foo.getAttributes().isDiskSynchronous()).isFalse();
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/FOO");
+  }
+
+  @Test
+  public void validateExpirationOptions() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/FOO --type=REPLICATE"
+        + " --enable-statistics" + " --entry-idle-time-expiration=3"
+        + " --entry-idle-time-expiration-action=DESTROY" + " --entry-time-to-live-expiration=5"
+        + " --entry-time-to-live-expiration-action=DESTROY" + " --region-idle-time-expiration=7"
+        + " --region-idle-time-expiration-action=DESTROY" + " --region-time-to-live-expiration=11"
+        + " --region-time-to-live-expiration-action=DESTROY");
+
+    Region foo = server.getCache().getRegion("/FOO");
+
+    assertThat(foo.getAttributes().getStatisticsEnabled()).isTrue();
+    assertThat(foo.getAttributes().getEntryIdleTimeout().getTimeout()).isEqualTo(3);
+    assertThat(foo.getAttributes().getEntryIdleTimeout().getAction())
+        .isEqualTo(ExpirationAction.DESTROY);
+    assertThat(foo.getAttributes().getEntryTimeToLive().getTimeout()).isEqualTo(5);
+    assertThat(foo.getAttributes().getEntryTimeToLive().getAction())
+        .isEqualTo(ExpirationAction.DESTROY);
+    assertThat(foo.getAttributes().getRegionIdleTimeout().getTimeout()).isEqualTo(7);
+    assertThat(foo.getAttributes().getRegionIdleTimeout().getAction())
+        .isEqualTo(ExpirationAction.DESTROY);
+    assertThat(foo.getAttributes().getRegionTimeToLive().getTimeout()).isEqualTo(11);
+    assertThat(foo.getAttributes().getRegionTimeToLive().getAction())
+        .isEqualTo(ExpirationAction.DESTROY);
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/FOO");
+  }
+
+  @Test
+  public void validatePartitionRegionOptions() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/FOO --type=PARTITION_REDUNDANT"
+        + " --local-max-memory=1001" + " --recovery-delay=7" + " --redundant-copies=1"
+        + " --startup-recovery-delay=5" + " --total-max-memory=2001" + " --total-num-buckets=11"
+        + " --partition-resolver=" + TestPartitionResolver.class.getName());
+
+    Region foo = server.getCache().getRegion("/FOO");
+
+    assertThat(foo.getAttributes().getPartitionAttributes().getLocalMaxMemory()).isEqualTo(1001);
+    assertThat(foo.getAttributes().getPartitionAttributes().getRecoveryDelay()).isEqualTo(7);
+    assertThat(foo.getAttributes().getPartitionAttributes().getRedundantCopies()).isEqualTo(1);
+    assertThat(foo.getAttributes().getPartitionAttributes().getStartupRecoveryDelay()).isEqualTo(5);
+    assertThat(foo.getAttributes().getPartitionAttributes().getTotalMaxMemory()).isEqualTo(2001);
+    assertThat(foo.getAttributes().getPartitionAttributes().getTotalNumBuckets()).isEqualTo(11);
+    assertThat(
+        foo.getAttributes().getPartitionAttributes().getPartitionResolver().getClass().getName())
+            .isEqualTo(TestPartitionResolver.class.getName());
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/FOO");
+  }
+
+  @Test
+  public void validateCallbackOptions() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/FOO --type=PARTITION_REDUNDANT"
+        + " --cache-listener=" + TestCacheListener.class.getName() + " --cache-loader="
+        + TestCacheLoader.class.getName() + " --cache-writer=" + TestCacheWriter.class.getName()
+        + " --compressor=" + TestCompressor.class.getName());
+
+    Region foo = server.getCache().getRegion("/FOO");
+
+    assertThat(Arrays.stream(foo.getAttributes().getCacheListeners())
+        .map(c -> c.getClass().getName()).collect(Collectors.toSet()))
+            .contains(TestCacheListener.class.getName());
+    assertThat(foo.getAttributes().getCacheLoader().getClass().getName())
+        .isEqualTo(TestCacheLoader.class.getName());
+    assertThat(foo.getAttributes().getCacheWriter().getClass().getName())
+        .isEqualTo(TestCacheWriter.class.getName());
+    assertThat(foo.getAttributes().getCompressor().getClass().getName())
+        .isEqualTo(TestCompressor.class.getName());
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/FOO");
+  }
+
+  @Test
+  public void validateConstraints() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/FOO --type=REPLICATE" + " --key-constraint="
+        + TestConstraint.class.getName() + " --value-constraint=" + TestConstraint.class.getName());
+
+    Region foo = server.getCache().getRegion("/FOO");
+
+    assertThat(foo.getAttributes().getKeyConstraint().getName())
+        .isEqualTo(TestConstraint.class.getName());
+    assertThat(foo.getAttributes().getValueConstraint().getName())
+        .isEqualTo(TestConstraint.class.getName());
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/FOO");
+  }
+
+  @Test
+  public void validateEntryIdleTimeExpiration() throws Exception {
+    gfsh.executeAndVerifyCommand(
+        "create region --name=/FOO --type=REPLICATE --entry-idle-time-expiration=7 --enable-statistics");
+    Region template = server.getCache().getRegion("/FOO");
+    assertThat(template.getAttributes().getEntryIdleTimeout().getTimeout()).isEqualTo(7);
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/FOO");
+  }
+
+  @Test
+  public void validateTemplateRegionAttributesForReplicate() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/TEMPLATE --type=REPLICATE"
+        + " --enable-async-conflation" + " --enable-cloning" + " --enable-concurrency-checks=false"
+        + " --enable-multicast" + " --enable-statistics" + " --enable-subscription-conflation"
+        + " --enable-synchronous-disk=false" + " --entry-idle-time-expiration=3"
+        + " --entry-idle-time-expiration-action=DESTROY" + " --entry-time-to-live-expiration=5"
+        + " --entry-time-to-live-expiration-action=DESTROY" + " --region-idle-time-expiration=7"
+        + " --region-idle-time-expiration-action=DESTROY" + " --region-time-to-live-expiration=11"
+        + " --region-time-to-live-expiration-action=DESTROY" + " --cache-listener="
+        + TestCacheListener.class.getName() + " --cache-loader=" + TestCacheLoader.class.getName()
+        + " --cache-writer=" + TestCacheWriter.class.getName() + " --compressor="
+        + TestCompressor.class.getName() + " --key-constraint=" + TestConstraint.class.getName()
+        + " --value-constraint=" + TestConstraint.class.getName());
+
+    gfsh.executeAndVerifyCommand("create region --name=/COPY --template-region=/TEMPLATE");
+
+    Region copy = server.getCache().getRegion("/COPY");
+
+    assertThat(copy.getAttributes().getStatisticsEnabled()).isTrue();
+    assertThat(copy.getAttributes().getEnableAsyncConflation()).isTrue();
+    assertThat(copy.getAttributes().getCloningEnabled()).isTrue();
+    assertThat(copy.getAttributes().getConcurrencyChecksEnabled()).isFalse();
+    assertThat(copy.getAttributes().getMulticastEnabled()).isTrue();
+    assertThat(copy.getAttributes().getStatisticsEnabled()).isTrue();
+    assertThat(copy.getAttributes().getEnableSubscriptionConflation()).isTrue();
+    assertThat(copy.getAttributes().isDiskSynchronous()).isFalse();
+    assertThat(copy.getAttributes().getEntryIdleTimeout().getTimeout()).isEqualTo(3);
+    assertThat(copy.getAttributes().getEntryIdleTimeout().getAction())
+        .isEqualTo(ExpirationAction.DESTROY);
+    assertThat(copy.getAttributes().getEntryTimeToLive().getTimeout()).isEqualTo(5);
+    assertThat(copy.getAttributes().getEntryTimeToLive().getAction())
+        .isEqualTo(ExpirationAction.DESTROY);
+    assertThat(copy.getAttributes().getRegionIdleTimeout().getTimeout()).isEqualTo(7);
+    assertThat(copy.getAttributes().getRegionIdleTimeout().getAction())
+        .isEqualTo(ExpirationAction.DESTROY);
+    assertThat(copy.getAttributes().getRegionTimeToLive().getTimeout()).isEqualTo(11);
+    assertThat(copy.getAttributes().getRegionTimeToLive().getAction())
+        .isEqualTo(ExpirationAction.DESTROY);
+    assertThat(Arrays.stream(copy.getAttributes().getCacheListeners())
+        .map(c -> c.getClass().getName()).collect(Collectors.toSet()))
+            .contains(TestCacheListener.class.getName());
+    assertThat(copy.getAttributes().getCacheLoader().getClass().getName())
+        .isEqualTo(TestCacheLoader.class.getName());
+    assertThat(copy.getAttributes().getCacheWriter().getClass().getName())
+        .isEqualTo(TestCacheWriter.class.getName());
+    assertThat(copy.getAttributes().getCompressor().getClass().getName())
+        .isEqualTo(TestCompressor.class.getName());
+    assertThat(copy.getAttributes().getKeyConstraint().getName())
+        .isEqualTo(TestConstraint.class.getName());
+    assertThat(copy.getAttributes().getValueConstraint().getName())
+        .isEqualTo(TestConstraint.class.getName());
+
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/COPY");
+    gfsh.executeAndVerifyCommand("destroy region --name=/TEMPLATE");
+  }
+
+  @Test
+  public void validateTemplateRegionAttributesForPartitionRedundant() throws Exception {
+    gfsh.executeAndVerifyCommand("create region --name=/TEMPLATE --type=PARTITION_REDUNDANT"
+        + " --enable-async-conflation" + " --enable-cloning" + " --enable-concurrency-checks=false"
+        + " --enable-multicast" + " --enable-statistics" + " --enable-subscription-conflation"
+        + " --enable-synchronous-disk=false" + " --cache-listener="
+        + TestCacheListener.class.getName() + " --cache-loader=" + TestCacheLoader.class.getName()
+        + " --cache-writer=" + TestCacheWriter.class.getName() + " --compressor="
+        + TestCompressor.class.getName() + " --key-constraint=" + TestConstraint.class.getName()
+        + " --value-constraint=" + TestConstraint.class.getName() + " --local-max-memory=1001"
+        + " --recovery-delay=7" + " --redundant-copies=1" + " --startup-recovery-delay=5"
+        + " --total-max-memory=2001" + " --total-num-buckets=11" + " --partition-resolver="
+        + TestPartitionResolver.class.getName());
+
+    gfsh.executeAndVerifyCommand("create region --name=/COPY --template-region=/TEMPLATE");
+
+    Region copy = server.getCache().getRegion("/COPY");
+
+    assertThat(copy.getAttributes().getStatisticsEnabled()).isTrue();
+    assertThat(copy.getAttributes().getEnableAsyncConflation()).isTrue();
+    assertThat(copy.getAttributes().getCloningEnabled()).isTrue();
+    assertThat(copy.getAttributes().getConcurrencyChecksEnabled()).isFalse();
+    assertThat(copy.getAttributes().getMulticastEnabled()).isTrue();
+    assertThat(copy.getAttributes().getStatisticsEnabled()).isTrue();
+    assertThat(copy.getAttributes().getEnableSubscriptionConflation()).isTrue();
+    assertThat(copy.getAttributes().isDiskSynchronous()).isFalse();
+    assertThat(Arrays.stream(copy.getAttributes().getCacheListeners())
+        .map(c -> c.getClass().getName()).collect(Collectors.toSet()))
+            .contains(TestCacheListener.class.getName());
+    assertThat(copy.getAttributes().getCacheLoader().getClass().getName())
+        .isEqualTo(TestCacheLoader.class.getName());
+    assertThat(copy.getAttributes().getCacheWriter().getClass().getName())
+        .isEqualTo(TestCacheWriter.class.getName());
+    assertThat(copy.getAttributes().getCompressor().getClass().getName())
+        .isEqualTo(TestCompressor.class.getName());
+    assertThat(copy.getAttributes().getKeyConstraint().getName())
+        .isEqualTo(TestConstraint.class.getName());
+    assertThat(copy.getAttributes().getValueConstraint().getName())
+        .isEqualTo(TestConstraint.class.getName());
+    assertThat(copy.getAttributes().getPartitionAttributes().getLocalMaxMemory()).isEqualTo(1001);
+    assertThat(copy.getAttributes().getPartitionAttributes().getRecoveryDelay()).isEqualTo(7);
+    assertThat(copy.getAttributes().getPartitionAttributes().getRedundantCopies()).isEqualTo(1);
+    assertThat(copy.getAttributes().getPartitionAttributes().getStartupRecoveryDelay())
+        .isEqualTo(5);
+    assertThat(copy.getAttributes().getPartitionAttributes().getTotalMaxMemory()).isEqualTo(2001);
+    assertThat(copy.getAttributes().getPartitionAttributes().getTotalNumBuckets()).isEqualTo(11);
+    assertThat(
+        copy.getAttributes().getPartitionAttributes().getPartitionResolver().getClass().getName())
+            .isEqualTo(TestPartitionResolver.class.getName());
+
+    gfsh.executeAndVerifyCommand("destroy region --name=/COPY");
+    gfsh.executeAndVerifyCommand("destroy region --name=/TEMPLATE");
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandTest.java
index a4e2eb4..4bf1532 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandTest.java
@@ -15,31 +15,219 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
-import static org.apache.geode.management.internal.cli.commands.CreateRegionCommand.regionExists;
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
+import org.mockito.ArgumentCaptor;
 
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.DistributedSystemMXBean;
 import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.functions.RegionFunctionArgs;
 import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.test.junit.categories.UnitTest;
 import org.apache.geode.test.junit.rules.GfshParserRule;
-import org.apache.geode.test.junit.categories.IntegrationTest;
 
-@Category(IntegrationTest.class)
+@Category(UnitTest.class)
 public class CreateRegionCommandTest {
   @Rule
   public GfshParserRule parser = new GfshParserRule();
 
+  private CreateRegionCommand command;
+  private InternalCache cache;
+
+  @Before
+  public void before() throws Exception {
+    command = spy(CreateRegionCommand.class);
+    cache = mock(InternalCache.class);
+    doReturn(cache).when(command).getCache();
+  }
+
   @Test
   public void testRegionExistsReturnsCorrectValue() throws Exception {
-    InternalCache cache = mock(InternalCache.class);
-    assertThat(regionExists(cache, null)).isFalse();
+    assertThat(command.regionExists(cache, null)).isFalse();
+  }
+
+  @Test
+  public void missingName() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command, "create region");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString()).contains("Invalid command");
+  }
+
+  @Test
+  public void missingBothTypeAndUseAttributeFrom() throws Exception {
+    CommandResult result =
+        parser.executeCommandWithInstance(command, "create region --name=region");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString())
+        .contains("One of \\\"type\\\" or \\\"template-region\\\" is required.");
+  }
+
+  @Test
+  public void haveBothTypeAndUseAttributeFrom() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --template-region=regionB");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString())
+        .contains("Only one of type & template-region can be specified.");
+  }
+
+  @Ignore("Eviction is not configurable yet")
+  @Test
+  public void invalidEvictionAction() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --eviction-action=invalidAction");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString()).contains("Invalid command");
+  }
+
+  @Ignore("Eviction is not configurable yet")
+  @Test
+  public void invalidEvictionAttributes() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --eviction-max-memory=1000 --eviction-entry-count=200");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString())
+        .contains("eviction-max-memory and eviction-entry-count cannot both be specified.");
+  }
+
+  @Ignore("Eviction is not configurable yet")
+  @Test
+  public void missingEvictionAction() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --eviction-max-memory=1000");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString()).contains("eviction-action must be specified.");
+  }
+
+  @Test
+  public void templateRegionAttributesNotAvailable() throws Exception {
+    doReturn(null).when(command).getRegionAttributes(eq(cache), any());
+    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
+    doReturn(true).when(command).regionExists(eq(cache), any());
+
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --template-region=regionA");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString())
+        .contains("Could not retrieve region attributes for given path");
+  }
+
+  @Test
+  public void defaultValues() throws Exception {
+    ResultCollector resultCollector = mock(ResultCollector.class);
+    doReturn(resultCollector).when(command).executeFunction(any(), any(), any());
+    when(resultCollector.getResult()).thenReturn(Collections.emptyList());
+    DistributedSystemMXBean dsMBean = mock(DistributedSystemMXBean.class);
+    doReturn(dsMBean).when(command).getDSMBean(any());
+    doReturn(Collections.singleton(mock(DistributedMember.class))).when(command).findMembers(any(),
+        any());
+    doReturn(true).when(command).verifyDistributedRegionMbean(any(), any());
+
+    parser.executeCommandWithInstance(command, "create region --name=A --type=REPLICATE");
+    ArgumentCaptor<RegionFunctionArgs> argsCaptor =
+        ArgumentCaptor.forClass(RegionFunctionArgs.class);
+    verify(command).executeFunction(any(), argsCaptor.capture(), any());
+    RegionFunctionArgs args = argsCaptor.getValue();
+
+    assertThat(args.getRegionPath()).isEqualTo("/A");
+    assertThat(args.getRegionShortcut()).isEqualTo(RegionShortcut.REPLICATE);
+    assertThat(args.getTemplateRegion()).isNull();
+    assertThat(args.isSkipIfExists()).isFalse();
+    assertThat(args.getKeyConstraint()).isNull();
+    assertThat(args.getValueConstraint()).isNull();
+    assertThat(args.isStatisticsEnabled()).isNull();
+
+    assertThat(args.getEntryExpirationIdleTime()).isNull();
+    assertThat(args.getEntryExpirationTTL()).isNull();
+    assertThat(args.getRegionExpirationIdleTime()).isNull();
+    assertThat(args.getRegionExpirationTTL()).isNull();
+
+    assertThat(args.getDiskStore()).isNull();
+    assertThat(args.isDiskSynchronous()).isNull();
+    assertThat(args.isEnableAsyncConflation()).isNull();
+    assertThat(args.isEnableSubscriptionConflation()).isNull();
+    assertThat(args.getCacheListeners()).isEmpty();
+    assertThat(args.getCacheLoader()).isNull();
+    assertThat(args.getCacheWriter()).isNull();
+    assertThat(args.getAsyncEventQueueIds()).isEmpty();
+    assertThat(args.getGatewaySenderIds()).isEmpty();
+    assertThat(args.isConcurrencyChecksEnabled()).isNull();
+    assertThat(args.isCloningEnabled()).isNull();
+    assertThat(args.isMcastEnabled()).isNull();
+    assertThat(args.getConcurrencyLevel()).isNull();
+    assertThat(args.getPartitionArgs()).isNotNull();
+    assertThat(args.getEvictionMax()).isNull();
+    assertThat(args.getCompressor()).isNull();
+    assertThat(args.isOffHeap()).isNull();
+    assertThat(args.getRegionAttributes()).isNull();
+  }
+
+  @Test
+  public void invalidCacheListener() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --cache-listener=abc-def");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString())
+        .contains("Specify a valid class name for cache-listener.");
+  }
+
+  @Test
+  public void invalidCacheLoader() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --cache-loader=abc-def");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString())
+        .contains("Specify a valid class name for cache-loader.");
+  }
+
+  @Test
+  public void invalidCacheWriter() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --cache-writer=abc-def");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString())
+        .contains("Specify a valid class name for cache-writer.");
+  }
+
+  @Test
+  public void invalidCompressor() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --compressor=abc-def");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString()).contains("abc-def is an invalid Compressor.");
+  }
+
+  @Test
+  public void invalidKeyType() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --key-type=abc-def");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString()).contains("Invalid command");
+  }
+
+  @Test
+  public void invalidValueType() throws Exception {
+    CommandResult result = parser.executeCommandWithInstance(command,
+        "create region --name=region --type=REPLICATE --value-type=abc-def");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.getContent().toString()).contains("Invalid command");
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
index 89bb3bd..ee4a14d 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
@@ -89,7 +89,7 @@ public class DestroyRegionCommandDUnitTest {
 
     // Test unable to destroy with co-location
     gfsh.executeAndVerifyCommandError("destroy region --name=/Customer",
-        "The parent region [/Customer] in colocation chain cannot be destroyed");
+        "The parent region \\[/Customer\\] in colocation chain cannot be destroyed");
 
     // Test success
     gfsh.executeAndVerifyCommand("destroy region --name=/Order", "destroyed successfully");
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCacheLoader.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCacheLoader.java
new file mode 100644
index 0000000..4089a4c
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCacheLoader.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import org.apache.geode.cache.CacheLoader;
+import org.apache.geode.cache.CacheLoaderException;
+import org.apache.geode.cache.LoaderHelper;
+
+public class TestCacheLoader implements CacheLoader {
+  @Override
+  public Object load(LoaderHelper helper) throws CacheLoaderException {
+    return null;
+  }
+
+  @Override
+  public void close() {}
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCacheWriter.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCacheWriter.java
new file mode 100644
index 0000000..041d310
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCacheWriter.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import org.apache.geode.cache.CacheWriter;
+import org.apache.geode.cache.CacheWriterException;
+import org.apache.geode.cache.EntryEvent;
+import org.apache.geode.cache.RegionEvent;
+
+public class TestCacheWriter implements CacheWriter {
+  @Override
+  public void beforeUpdate(EntryEvent event) throws CacheWriterException {}
+
+  @Override
+  public void beforeCreate(EntryEvent event) throws CacheWriterException {}
+
+  @Override
+  public void beforeDestroy(EntryEvent event) throws CacheWriterException {}
+
+  @Override
+  public void beforeRegionDestroy(RegionEvent event) throws CacheWriterException {}
+
+  @Override
+  public void beforeRegionClear(RegionEvent event) throws CacheWriterException {}
+
+  @Override
+  public void close() {}
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCompressor.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCompressor.java
new file mode 100644
index 0000000..9f62502
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestCompressor.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import org.apache.geode.compression.Compressor;
+
+public class TestCompressor implements Compressor {
+  @Override
+  public byte[] compress(byte[] input) {
+    return new byte[0];
+  }
+
+  @Override
+  public byte[] decompress(byte[] input) {
+    return new byte[0];
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestPartitionResolver.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestPartitionResolver.java
new file mode 100644
index 0000000..2c1f693
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/TestPartitionResolver.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import org.apache.geode.cache.EntryOperation;
+import org.apache.geode.cache.PartitionResolver;
+
+public class TestPartitionResolver implements PartitionResolver {
+  @Override
+  public Object getRoutingObject(EntryOperation opDetails) {
+    return null;
+  }
+
+  @Override
+  public String getName() {
+    return null;
+  }
+
+  @Override
+  public void close() {
+
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/ExpirationAttrsTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/ExpirationAttrsTest.java
new file mode 100644
index 0000000..37049a0
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/ExpirationAttrsTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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 org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.ExpirationAction;
+import org.apache.geode.management.internal.cli.functions.RegionFunctionArgs.ExpirationAttrs;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ExpirationAttrsTest {
+
+  private static ExpirationAttrs.ExpirationFor expirationFor =
+      ExpirationAttrs.ExpirationFor.ENTRY_IDLE;
+
+  @Test
+  public void constructor() throws Exception {
+    ExpirationAttrs attrs = new ExpirationAttrs(expirationFor, null, null);
+    assertThat(attrs.getTime()).isEqualTo(0);
+    assertThat(attrs.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+
+    attrs = new ExpirationAttrs(expirationFor, -1, null);
+    assertThat(attrs.getTime()).isEqualTo(0);
+    assertThat(attrs.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+
+    attrs = new ExpirationAttrs(expirationFor, 0, null);
+    assertThat(attrs.getTime()).isEqualTo(0);
+    assertThat(attrs.getAction()).isEqualTo(ExpirationAction.INVALIDATE);
+
+    attrs = new ExpirationAttrs(expirationFor, 2, "destroy");
+    assertThat(attrs.getTime()).isEqualTo(2);
+    assertThat(attrs.getAction()).isEqualTo(ExpirationAction.DESTROY);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/RegionFunctionArgsTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/RegionFunctionArgsTest.java
new file mode 100644
index 0000000..fcf7020
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/RegionFunctionArgsTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class RegionFunctionArgsTest {
+
+  private RegionFunctionArgs args;
+  private RegionFunctionArgs.PartitionArgs partitionArgs;
+
+  @Before
+  public void before() {
+    args = new RegionFunctionArgs();
+    partitionArgs = new RegionFunctionArgs.PartitionArgs();
+  }
+
+  @Test
+  public void defaultRegionFunctionArgs() throws Exception {
+    assertThat(args.isDiskSynchronous()).isNull();
+    assertThat(args.isCloningEnabled()).isNull();
+    assertThat(args.isConcurrencyChecksEnabled()).isNull();
+    assertThat(args.getConcurrencyLevel()).isNull();
+    assertThat(args.getPartitionArgs()).isNotNull();
+    assertThat(args.getPartitionArgs().hasPartitionAttributes()).isFalse();
+  }
+
+  @Test
+  public void defaultPartitionArgs() throws Exception {
+    assertThat(partitionArgs.hasPartitionAttributes()).isFalse();
+
+    partitionArgs.setPartitionResolver(null);
+    assertThat(partitionArgs.hasPartitionAttributes()).isFalse();
+
+    partitionArgs.setPrTotalNumBuckets(10);
+    assertThat(partitionArgs.getPrTotalNumBuckets()).isEqualTo(10);
+    assertThat(partitionArgs.hasPartitionAttributes()).isTrue();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/test/junit/rules/GfshShellConnectionRule.java b/geode-core/src/test/java/org/apache/geode/test/junit/rules/GfshShellConnectionRule.java
index a6e24e5..9c869b2 100644
--- a/geode-core/src/test/java/org/apache/geode/test/junit/rules/GfshShellConnectionRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/junit/rules/GfshShellConnectionRule.java
@@ -254,7 +254,7 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     }
     assertThat(result.getStatus()).describedAs(getGfshOutput()).isEqualTo(Result.Status.OK);
     for (String expectedOutput : expectedOutputs) {
-      assertThat(getGfshOutput()).contains(expectedOutput);
+      assertThat(getGfshOutput()).containsPattern(expectedOutput);
     }
     return result;
   }
@@ -268,7 +268,7 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     }
     assertThat(result.getStatus()).describedAs(getGfshOutput()).isEqualTo(Result.Status.ERROR);
     for (String expectedOutput : expectedOutputs) {
-      assertThat(getGfshOutput()).contains(expectedOutput);
+      assertThat(getGfshOutput()).containsPattern(expectedOutput);
     }
     return result;
   }
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
index 1a26a17..1c8e667 100755
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
@@ -34,7 +34,6 @@ org/apache/geode/admin/RegionNotFoundException,true,1758668137691463909
 org/apache/geode/admin/RuntimeAdminException,true,-7512771113818634005
 org/apache/geode/admin/SystemMemberType,true,3284366994485749302,ordinal:int
 org/apache/geode/admin/UnmodifiableConfigurationException,true,-7653547392992060646
-org/apache/geode/admin/internal/BackupStatusImpl,true,3704162840296921840,backedUpDiskStores:java/util/Map,offlineDiskStores:java/util/Set
 org/apache/geode/admin/internal/CacheHealthConfigImpl,false,maxEventQueueSize:long,maxLoadTime:long,maxNetSearchTime:long,minHitRatio:double
 org/apache/geode/admin/internal/GemFireHealthConfigImpl,true,-6797673296902808018,hostName:java/lang/String,interval:int
 org/apache/geode/admin/internal/MemberHealthConfigImpl,true,3966032573073580490,maxMessageQueueSize:long,maxReplyTimeouts:long,maxRetransmissionRatio:double,maxVMProcessSize:long
@@ -250,6 +249,7 @@ org/apache/geode/internal/SystemAdmin$CombinedResources,false
 org/apache/geode/internal/admin/CompoundEntrySnapshot,true,5776382582897895718,allUserAttributes:java/util/Set,allValues:java/util/Set,hitRatio:float,hitRatioSum:double,hitResponders:long,lastAccessTime:long,lastModifiedTime:long,name:java/lang/Object,numHits:long,numMisses:long
 org/apache/geode/internal/admin/CompoundRegionSnapshot,true,6295026394298398004,allCacheLoaders:java/util/Set,allCacheWriters:java/util/Set,allCapControllers:java/util/Set,allConcLevels:java/util/Set,allCustomIdle:java/util/HashSet,allCustomTtl:java/util/HashSet,allDataPolicies:java/util/Set,allEntryIdleTimeout:java/util/Set,allEntryTtl:java/util/Set,allInitialCaps:java/util/Set,allKeyConstraints:java/util/Set,allListeners:java/util/Set,allLoadFactors:java/util/Set,allRegionIdleTimeout:j [...]
 org/apache/geode/internal/admin/StatAlert,true,5725457607122449170,definitionId:int,time:java/util/Date,values:java/lang/Number[]
+org/apache/geode/internal/admin/remote/DistributionLocatorId,true,6587390186971937865,bindAddress:java/lang/String,host:java/net/InetAddress,hostname:java/lang/String,hostnameForClients:java/lang/String,peerLocator:boolean,port:int,serverLocator:boolean
 org/apache/geode/internal/admin/remote/EntryValueNodeImpl,false,fields:org/apache/geode/internal/admin/remote/EntryValueNodeImpl[],name:java/lang/String,primitive:boolean,primitiveVal:java/lang/Object,type:java/lang/String
 org/apache/geode/internal/cache/BucketAdvisor$SetFromMap,true,2454657854757543876,m:java/util/Map
 org/apache/geode/internal/cache/BucketNotFoundException,true,2898657229184289911
@@ -465,6 +465,7 @@ org/apache/geode/management/internal/cli/AbstractCliAroundInterceptor$Response,f
 org/apache/geode/management/internal/cli/CliUtil$DeflaterInflaterData,true,1104813333595216795,data:byte[],dataLength:int
 org/apache/geode/management/internal/cli/domain/AsyncEventQueueDetails,true,1,batchSize:int,diskStoreName:java/lang/String,id:java/lang/String,listener:java/lang/String,listenerProperties:java/util/Properties,maxQueueMemory:int,persistent:boolean
 org/apache/geode/management/internal/cli/domain/CacheServerInfo,true,1,bindAddress:java/lang/String,isRunning:boolean,port:int
+org/apache/geode/management/internal/cli/domain/DataCommandRequest,false,command:java/lang/String,key:java/lang/String,keyClass:java/lang/String,loadOnCacheMiss:boolean,principal:java/lang/Object,putIfAbsent:boolean,query:java/lang/String,recursive:boolean,regionName:java/lang/String,removeAllKeys:java/lang/String,value:java/lang/String,valueClass:java/lang/String
 org/apache/geode/management/internal/cli/domain/DataCommandResult,true,1,command:java/lang/String,error:java/lang/Throwable,errorString:java/lang/String,getResult:java/lang/Object,hasResultForAggregation:boolean,infoString:java/lang/String,inputKey:java/lang/Object,inputQuery:java/lang/Object,inputValue:java/lang/Object,keyClass:java/lang/String,limit:int,locateEntryLocations:java/util/List,locateEntryResult:org/apache/geode/management/internal/cli/domain/DataCommandResult$KeyInfo,operat [...]
 org/apache/geode/management/internal/cli/domain/DataCommandResult$KeyInfo,false,host:java/lang/String,locations:java/util/ArrayList,memberId:java/lang/String,memberName:java/lang/String,pid:int
 org/apache/geode/management/internal/cli/domain/DataCommandResult$SelectResultRow,true,1,type:int,value:java/lang/Object
@@ -514,7 +515,6 @@ org/apache/geode/management/internal/cli/functions/ExportDataFunction,true,1
 org/apache/geode/management/internal/cli/functions/ExportLogsFunction,true,1
 org/apache/geode/management/internal/cli/functions/ExportLogsFunction$Args,false,endTime:java/time/LocalDateTime,includeLogs:boolean,includeStats:boolean,logLevel:org/apache/logging/log4j/Level,startTime:java/time/LocalDateTime,thisLogLevelOnly:boolean
 org/apache/geode/management/internal/cli/functions/FetchRegionAttributesFunction,true,4366812590788342070
-org/apache/geode/management/internal/cli/functions/FetchRegionAttributesFunction$FetchRegionAttributesFunctionResult,true,-3970828263897978845,cacheListenerClasses:java/lang/String[],cacheLoaderClass:java/lang/String,cacheWriterClass:java/lang/String,regionAttributes:org/apache/geode/cache/RegionAttributes
 org/apache/geode/management/internal/cli/functions/FetchSharedConfigurationStatusFunction,true,1
 org/apache/geode/management/internal/cli/functions/GarbageCollectionFunction,true,1
 org/apache/geode/management/internal/cli/functions/GatewayReceiverCreateFunction,true,8746830191680509335
@@ -545,10 +545,10 @@ org/apache/geode/management/internal/cli/functions/RebalanceFunction,true,1
 org/apache/geode/management/internal/cli/functions/RegionAlterFunction,true,-4846425364943216425
 org/apache/geode/management/internal/cli/functions/RegionCreateFunction,true,8746830191680509335
 org/apache/geode/management/internal/cli/functions/RegionDestroyFunction,true,9172773671865750685
-org/apache/geode/management/internal/cli/functions/RegionFunctionArgs,true,-5158224572470173267,asyncEventQueueIds:java/util/Set,cacheListeners:java/util/Set,cacheLoader:java/lang/String,cacheWriter:java/lang/String,cloningEnabled:java/lang/Boolean,compressor:java/lang/String,concurrencyChecksEnabled:java/lang/Boolean,concurrencyLevel:java/lang/Integer,diskStore:java/lang/String,diskSynchronous:java/lang/Boolean,enableAsyncConflation:java/lang/Boolean,enableSubscriptionConflation:java/la [...]
-org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$ExpirationAttrs,true,1474255033398008062,action:org/apache/geode/cache/ExpirationAction,time:java/lang/Integer,type:org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$ExpirationAttrs$ExpirationFor
+org/apache/geode/management/internal/cli/functions/RegionFunctionArgs,true,2204943186081037301,asyncEventQueueIds:java/util/Set,cacheListeners:java/util/Set,cacheLoader:java/lang/String,cacheWriter:java/lang/String,cloningEnabled:java/lang/Boolean,compressor:java/lang/String,concurrencyChecksEnabled:java/lang/Boolean,concurrencyLevel:java/lang/Integer,diskStore:java/lang/String,diskSynchronous:java/lang/Boolean,enableAsyncConflation:java/lang/Boolean,enableSubscriptionConflation:java/lan [...]
+org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$ExpirationAttrs,true,1474255033398008062,timeAndAction:org/apache/geode/cache/ExpirationAttributes,type:org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$ExpirationAttrs$ExpirationFor
 org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$ExpirationAttrs$ExpirationFor,false
-org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$PartitionArgs,true,5907052187323280919,hasPartitionAttributes:boolean,isPartitionResolver:boolean,isSetPRLocalMaxMemory:boolean,isSetPRRecoveryDelay:boolean,isSetPRRedundantCopies:boolean,isSetPRStartupRecoveryDelay:boolean,isSetPRTotalMaxMemory:boolean,isSetPRTotalNumBuckets:boolean,partitionResolver:java/lang/String,prColocatedWith:java/lang/String,prLocalMaxMemory:int,prRecoveryDelay:long,prRedundantCopies:int,prSta [...]
+org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$PartitionArgs,true,5907052187323280919,partitionResolver:java/lang/String,prColocatedWith:java/lang/String,prLocalMaxMemory:java/lang/Integer,prRecoveryDelay:java/lang/Long,prRedundantCopies:java/lang/Integer,prStartupRecoveryDelay:java/lang/Long,prTotalMaxMemory:java/lang/Long,prTotalNumBuckets:java/lang/Integer
 org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunction,false
 org/apache/geode/management/internal/cli/functions/ShutDownFunction,true,1
 org/apache/geode/management/internal/cli/functions/SizeExportLogsFunction,true,1
@@ -604,5 +604,3 @@ org/apache/geode/security/AuthenticationFailedException,true,-820286647227908887
 org/apache/geode/security/AuthenticationRequiredException,true,4675976651103154919
 org/apache/geode/security/GemFireSecurityException,true,3814254578203076926,cause:java/lang/Throwable
 org/apache/geode/security/NotAuthorizedException,true,419215768216387745,principal:java/security/Principal
-org/apache/geode/internal/admin/remote/DistributionLocatorId,true,6587390186971937865,bindAddress:java/lang/String,host:java/net/InetAddress,hostname:java/lang/String,hostnameForClients:java/lang/String,peerLocator:boolean,port:int,serverLocator:boolean
-org/apache/geode/management/internal/cli/domain/DataCommandRequest,false,command:java/lang/String,key:java/lang/String,keyClass:java/lang/String,loadOnCacheMiss:boolean,principal:java/lang/Object,putIfAbsent:boolean,query:java/lang/String,recursive:boolean,regionName:java/lang/String,removeAllKeys:java/lang/String,value:java/lang/String,valueClass:java/lang/String

-- 
To stop receiving notification emails like this one, please contact
['"commits@geode.apache.org" <co...@geode.apache.org>'].