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

[geode] branch develop updated: GEODE-6306: extend the RegionConfigRealizer to honor all region attributes configurations

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

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


The following commit(s) were added to refs/heads/develop by this push:
     new c877ef8  GEODE-6306: extend the RegionConfigRealizer to honor all region attributes configurations
c877ef8 is described below

commit c877ef8720dc8f5a1a9735a3678060f337441673
Author: jinmeiliao <ji...@pivotal.io>
AuthorDate: Fri Jan 25 10:11:55 2019 -0800

    GEODE-6306: extend the RegionConfigRealizer to honor all region attributes configurations
    
    Co-authored-by: Jens Deppe <jd...@pivotal.io>
---
 .../internal/rest/RegionManagementDunitTest.java   |   3 +
 .../internal/api/RegionAPIDUnitTest.java           |   6 +-
 .../cli/commands/CreateRegionCommandDUnitTest.java |  12 +-
 .../CreateRegionCommandIntegrationTest.java        |  12 +
 .../RegionConfigMutatorIntegrationTest.java        |  52 ++++
 .../RegionConfigRealizerIntegrationTest.java       |  55 +++++
 .../geode/internal/cache/GemFireCacheImpl.java     |   1 +
 .../internal/cache/PartitionAttributesImpl.java    |  19 +-
 .../api/LocatorClusterManagementService.java       |  46 +++-
 .../geode/management/internal/cli/CliUtil.java     |   5 +
 .../cli/functions/RegionCreateFunction.java        | 269 +--------------------
 .../cli/functions/UpdateCacheFunction.java         |  13 +-
 .../management/internal/cli/util/RegionPath.java   |  35 +--
 .../domain/DeclarableTypeInstantiator.java         |  13 +
 .../mutators/ConfigurationMutatorFactory.java      |  35 ---
 .../realizers/ConfigurationRealizerFactory.java    |  35 ---
 .../realizers/RegionConfigRealizer.java            | 218 +++++++++++++++--
 .../validators/ConfigurationValidator.java         |  23 +-
 .../cache/configuration/RegionConfigTest.java      |  51 +++-
 ...nternalConfigurationPersistenceServiceTest.java |   2 +-
 .../internal/cli/util/RegionPathTest.java          |  82 +++++++
 .../mutators/ConfigurationMutatorFactoryTest.java  |  36 ---
 .../ConfigurationRealizerFactoryTest.java          |  35 ---
 .../realizers/RegionConfigRealizerTest.java        |  13 +-
 .../geode/cache/configuration/RegionConfigTest.xml | 149 ++++++++++++
 .../configuration/RegionAttributesDataPolicy.java  |  19 +-
 .../cache/configuration/RegionAttributesType.java  |  37 ++-
 .../geode/cache/configuration/RegionConfig.java    | 137 +++++++++++
 .../internal/api/ClusterManagementService.java     |   6 +-
 .../controllers/RegionManagementController.java    |   2 +-
 30 files changed, 882 insertions(+), 539 deletions(-)

diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/RegionManagementDunitTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/RegionManagementDunitTest.java
index 85eaf63..080425a 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/RegionManagementDunitTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/RegionManagementDunitTest.java
@@ -77,5 +77,8 @@ public class RegionManagementDunitTest {
               .getCacheConfig("cluster");
       assertThat(cacheConfig.getRegions().get(0).getName()).isEqualTo("customers");
     });
+
+    // verify that additional server can be started with the cluster configuration
+    cluster.startServerVM(2, locator.getPort());
   }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/api/RegionAPIDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/api/RegionAPIDUnitTest.java
index b685fdd..b45e43a 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/api/RegionAPIDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/api/RegionAPIDUnitTest.java
@@ -66,7 +66,7 @@ public class RegionAPIDUnitTest {
       config.setName(regionName);
       config.setRefid(RegionShortcut.PARTITION.toString());
       ClusterManagementResult result = ClusterStartupRule.getLocator().getClusterManagementService()
-          .create(config);
+          .create(config, "cluster");
       assertThat(result.isSuccessful()).isTrue();
     });
 
@@ -99,7 +99,7 @@ public class RegionAPIDUnitTest {
       config.setName(regionName);
       config.setRefid(RegionShortcut.REPLICATE.toString());
       ClusterManagementResult result = ClusterStartupRule.getLocator().getClusterManagementService()
-          .create(config);
+          .create(config, "cluster");
       assertThat(result.isSuccessful()).isTrue();
     });
 
@@ -124,7 +124,7 @@ public class RegionAPIDUnitTest {
       config.setName(regionName);
       config.setRefid(RegionShortcut.PARTITION.toString());
       ClusterManagementResult result = ClusterStartupRule.getLocator().getClusterManagementService()
-          .create(config);
+          .create(config, "cluster");
       assertThat(result.isSuccessful()).isTrue();
     });
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandDUnitTest.java
index 8261f9c..4ed5da7 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandDUnitTest.java
@@ -246,7 +246,7 @@ public class CreateRegionCommandDUnitTest {
     gfsh.executeAndAssertThat("create region --name=" + testName.getMethodName()
         + " --type=PARTITION --partition-resolver=InvalidPartitionResolver")
         .statusIsError()
-        .containsOutput("Could not find class");
+        .containsOutput("Error instantiating class");
   }
 
   @Test
@@ -255,13 +255,13 @@ public class CreateRegionCommandDUnitTest {
         + " --type=REPLICATE --entry-time-to-live-custom-expiry=InvalidCustomExpiry" +
         " --enable-statistics=true")
         .statusIsError()
-        .containsOutput("Could not find class");
+        .containsOutput("Error instantiating class");
 
     gfsh.executeAndAssertThat("create region --name=" + testName.getMethodName()
         + " --type=REPLICATE --entry-idle-time-custom-expiry=InvalidCustomExpiry" +
         " --enable-statistics=true")
         .statusIsError()
-        .containsOutput("Could not find class");
+        .containsOutput("Error instantiating class");
   }
 
   @Test
@@ -269,7 +269,7 @@ public class CreateRegionCommandDUnitTest {
     gfsh.executeAndAssertThat("create region --name=" + testName.getMethodName()
         + " --type=REPLICATE --cache-loader=InvalidCacheLoader")
         .statusIsError()
-        .containsOutput("Could not find class");
+        .containsOutput("Error instantiating class");
   }
 
   @Test
@@ -277,7 +277,7 @@ public class CreateRegionCommandDUnitTest {
     gfsh.executeAndAssertThat("create region --name=" + testName.getMethodName()
         + " --type=REPLICATE --cache-writer=InvalidCacheWriter")
         .statusIsError()
-        .containsOutput("Could not find class");
+        .containsOutput("Error instantiating class");
   }
 
   @Test
@@ -286,7 +286,7 @@ public class CreateRegionCommandDUnitTest {
         + " --type=REPLICATE --cache-listener=" + TestCacheListener.class.getName()
         + ",InvalidCacheListener")
         .statusIsError()
-        .containsOutput("Could not find class")
+        .containsOutput("Error instantiating class")
         .doesNotContainOutput("TestCacheListener");
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandIntegrationTest.java
index a067052..de189fe 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandIntegrationTest.java
@@ -625,4 +625,16 @@ public class CreateRegionCommandIntegrationTest {
     gfsh.executeAndAssertThat("create region --name=FOO --type=REPLICATE --cache-listener=abc{abc}")
         .statusIsError().containsOutput("Invalid JSON: {abc}");
   }
+
+  @Test
+  public void createSubRegion() throws Exception {
+    gfsh.executeAndAssertThat("create region --name=region --type=REPLICATE").statusIsSuccess();
+    gfsh.executeAndAssertThat("create region --name=region/region1 --type=REPLICATE")
+        .statusIsSuccess();
+
+    Region subregion = server.getCache().getRegion("/region/region1");
+    assertThat(subregion).isNotNull();
+
+    gfsh.executeAndAssertThat("destroy region --name=/region").statusIsSuccess();
+  }
 }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/configuration/mutators/RegionConfigMutatorIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/configuration/mutators/RegionConfigMutatorIntegrationTest.java
new file mode 100644
index 0000000..eba45d4
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/configuration/mutators/RegionConfigMutatorIntegrationTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.configuration.mutators;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.cache.configuration.CacheElement;
+import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.test.junit.rules.LocatorStarterRule;
+
+public class RegionConfigMutatorIntegrationTest {
+
+  @Rule
+  public LocatorStarterRule locator = new LocatorStarterRule().withAutoStart();
+
+  private RegionConfigMutator mutator;
+  private RegionConfig config;
+
+  @Before
+  public void before() throws Exception {
+    config = new RegionConfig();
+    mutator = new RegionConfigMutator();
+  }
+
+  @Test
+  public void sanity() throws Exception {
+    config.setRefid("REPLICATE");
+    config.setName("test");
+    CacheConfig cacheConfig =
+        locator.getLocator().getConfigurationPersistenceService().getCacheConfig("cluster", true);
+
+    mutator.add(config, cacheConfig);
+    assertThat(CacheElement.findElement(cacheConfig.getRegions(), config.getId())).isNotNull();
+  }
+}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizerIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizerIntegrationTest.java
new file mode 100644
index 0000000..a391f0a
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizerIntegrationTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.configuration.realizers;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.test.junit.rules.ServerStarterRule;
+
+public class RegionConfigRealizerIntegrationTest {
+
+  @Rule
+  public ServerStarterRule server = new ServerStarterRule().withAutoStart();
+
+  private RegionConfigRealizer realizer;
+  private RegionConfig config;
+
+  @Before
+  public void setup() {
+    config = new RegionConfig();
+    realizer = new RegionConfigRealizer();
+  }
+
+  @Test
+  public void sanityCheck() throws Exception {
+    config.setName("test");
+    config.setRefid("REPLICATE");
+
+    realizer.create(config, server.getCache());
+
+    Region<Object, Object> region = server.getCache().getRegion("test");
+    assertThat(region).isNotNull();
+    assertThat(region.getAttributes().getDataPolicy()).isEqualTo(DataPolicy.REPLICATE);
+    assertThat(region.getAttributes().getScope()).isEqualTo(Scope.DISTRIBUTED_ACK);
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index 1268098..fa6fc2a 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -1000,6 +1000,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     } // synchronized
   }
 
+  @Override
   public HttpService getHttpService() {
     return httpService;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionAttributesImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionAttributesImpl.java
index bfc9ded..df919e2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionAttributesImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionAttributesImpl.java
@@ -39,10 +39,10 @@ import org.apache.geode.cache.configuration.DeclarableType;
 import org.apache.geode.cache.configuration.RegionAttributesType;
 import org.apache.geode.cache.partition.PartitionListener;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.offheap.OffHeapStorage;
+import org.apache.geode.management.internal.configuration.domain.DeclarableTypeInstantiator;
 
 /**
  * Internal implementation of PartitionAttributes. New attributes existing only in this class and
@@ -792,7 +792,7 @@ public class PartitionAttributesImpl implements PartitionAttributes, Cloneable,
   }
 
   public static PartitionAttributesImpl fromConfig(
-      RegionAttributesType.PartitionAttributes configAttributes) {
+      RegionAttributesType.PartitionAttributes configAttributes, Cache cache) {
     PartitionAttributesImpl partitionAttributes = new PartitionAttributesImpl();
     if (configAttributes == null) {
       return null;
@@ -821,11 +821,8 @@ public class PartitionAttributesImpl implements PartitionAttributes, Cloneable,
     }
 
     if (configAttributes.getPartitionResolver() != null) {
-      try {
-        partitionAttributes.setPartitionResolver((PartitionResolver) ClassPathLoader.getLatest()
-            .forName(configAttributes.getPartitionResolver().getClassName()).newInstance());
-      } catch (Exception e) {
-      }
+      partitionAttributes.setPartitionResolver(
+          DeclarableTypeInstantiator.newInstance(configAttributes.getPartitionResolver(), cache));
     }
 
     if (configAttributes.getRecoveryDelay() != null) {
@@ -840,12 +837,8 @@ public class PartitionAttributesImpl implements PartitionAttributes, Cloneable,
     if (configAttributes.getPartitionListeners() != null) {
       List<DeclarableType> configListeners = configAttributes.getPartitionListeners();
       for (int i = 0; i < configListeners.size(); i++) {
-        try {
-          partitionAttributes.addPartitionListener((PartitionListener) ClassPathLoader.getLatest()
-              .forName(configListeners.get(i).getClassName())
-              .newInstance());
-        } catch (Exception e) {
-        }
+        partitionAttributes.addPartitionListener(
+            DeclarableTypeInstantiator.newInstance(configListeners.get(i), cache));
       }
     }
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/api/LocatorClusterManagementService.java b/geode-core/src/main/java/org/apache/geode/management/internal/api/LocatorClusterManagementService.java
index b745251..0177354 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/api/LocatorClusterManagementService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/api/LocatorClusterManagementService.java
@@ -18,14 +18,17 @@
 package org.apache.geode.management.internal.api;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.logging.log4j.Logger;
 import sun.reflect.generics.reflectiveObjects.NotImplementedException;
 
+import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.cache.configuration.CacheConfig;
 import org.apache.geode.cache.configuration.CacheElement;
+import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.ConfigurationPersistenceService;
@@ -36,7 +39,8 @@ import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.cli.functions.UpdateCacheFunction;
 import org.apache.geode.management.internal.configuration.mutators.ConfigurationMutator;
-import org.apache.geode.management.internal.configuration.mutators.ConfigurationMutatorFactory;
+import org.apache.geode.management.internal.configuration.mutators.RegionConfigMutator;
+import org.apache.geode.management.internal.configuration.validators.ConfigurationValidator;
 import org.apache.geode.management.internal.exceptions.EntityExistsException;
 import org.apache.geode.management.internal.exceptions.NoMembersException;
 
@@ -44,19 +48,40 @@ public class LocatorClusterManagementService implements ClusterManagementService
   private static Logger logger = LogService.getLogger();
   private DistributionManager distributionManager;
   private ConfigurationPersistenceService persistenceService;
+  private HashMap<Class, ConfigurationMutator> mutators;
+  private HashMap<Class, ConfigurationValidator> validators;
 
   public LocatorClusterManagementService(DistributionManager distributionManager,
       ConfigurationPersistenceService persistenceService) {
+    this(distributionManager, persistenceService, new HashMap(), new HashMap());
+    // initialize the list of mutators
+    mutators.put(RegionConfig.class, new RegionConfigMutator());
+
+    // initialize the list of validators
+  }
+
+  @VisibleForTesting
+  public LocatorClusterManagementService(DistributionManager distributionManager,
+      ConfigurationPersistenceService persistenceService, HashMap mutators, HashMap validators) {
     this.distributionManager = distributionManager;
     this.persistenceService = persistenceService;
+    this.mutators = mutators;
+    this.validators = validators;
   }
 
   @Override
-  public ClusterManagementResult create(CacheElement config) {
+  public ClusterManagementResult create(CacheElement config, String group) {
+    if (group == null) {
+      group = "cluster";
+    }
+
     ClusterManagementResult result = new ClusterManagementResult();
-    String group = "cluster";
-    ConfigurationMutator configurationMutator =
-        (new ConfigurationMutatorFactory()).generate(config);
+    ConfigurationMutator configurationMutator = mutators.get(config.getClass());
+
+    ConfigurationValidator validator = validators.get(config.getClass());
+    if (validator != null) {
+      validator.validate(config);
+    }
     final boolean configurationPersistenceEnabled = persistenceService != null;
 
     // exit early if config element already exists in cache config
@@ -84,13 +109,14 @@ public class LocatorClusterManagementService implements ClusterManagementService
 
     // persist configuration in cache config
     if (configurationPersistenceEnabled) {
-      persistenceService.updateCacheConfig(group, cacheConfigForGroup -> {
+      String finalGroup = group;
+      persistenceService.updateCacheConfig(finalGroup, cacheConfigForGroup -> {
         try {
           configurationMutator.add(config, cacheConfigForGroup);
           result.setClusterConfigPersisted(true,
-              "successfully persisted config for " + group);
+              "successfully persisted config for " + finalGroup);
         } catch (Exception e) {
-          String message = "failed to update cluster config for " + group;
+          String message = "failed to update cluster config for " + finalGroup;
           logger.error(message, e);
           result.setClusterConfigPersisted(false, message);
           return null;
@@ -104,12 +130,12 @@ public class LocatorClusterManagementService implements ClusterManagementService
   }
 
   @Override
-  public ClusterManagementResult delete(CacheElement config) {
+  public ClusterManagementResult delete(CacheElement config, String group) {
     throw new NotImplementedException();
   }
 
   @Override
-  public ClusterManagementResult update(CacheElement config) {
+  public ClusterManagementResult update(CacheElement config, String group) {
     throw new NotImplementedException();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
index 589b9eb..b2d4dd6 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
@@ -66,6 +66,7 @@ import org.apache.geode.management.internal.cli.exceptions.UserErrorException;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.management.internal.configuration.domain.DeclarableTypeInstantiator;
 
 /**
  * This class contains utility methods used by classes used to build the Command Line Interface
@@ -468,6 +469,10 @@ public class CliUtil {
     return loadedClass;
   }
 
+  /**
+   * @deprecated use {@link DeclarableTypeInstantiator}
+   */
+  @Deprecated
   public static <K> K newInstance(Class<K> klass, String neededFor) {
     K instance;
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
index 154db6b..558c642 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
@@ -14,43 +14,23 @@
  */
 package org.apache.geode.management.internal.cli.functions;
 
-import java.util.Arrays;
-import java.util.List;
-
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheListener;
-import org.apache.geode.cache.CacheLoader;
-import org.apache.geode.cache.CacheWriter;
-import org.apache.geode.cache.CustomExpiry;
-import org.apache.geode.cache.DataPolicy;
-import org.apache.geode.cache.ExpirationAction;
-import org.apache.geode.cache.ExpirationAttributes;
-import org.apache.geode.cache.PartitionResolver;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionExistsException;
-import org.apache.geode.cache.RegionFactory;
-import org.apache.geode.cache.Scope;
-import org.apache.geode.cache.configuration.DeclarableType;
-import org.apache.geode.cache.configuration.RegionAttributesType;
-import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.ResultSender;
-import org.apache.geode.compression.Compressor;
-import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.internal.cache.EvictionAttributesImpl;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.PartitionAttributesImpl;
 import org.apache.geode.internal.cache.execute.InternalFunction;
 import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.commands.RegionCommandsUtils;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.util.RegionPath;
 import org.apache.geode.management.internal.configuration.domain.XmlEntity;
+import org.apache.geode.management.internal.configuration.realizers.RegionConfigRealizer;
 
 /**
  *
@@ -66,6 +46,8 @@ public class RegionCreateFunction implements InternalFunction {
 
   public static RegionCreateFunction INSTANCE = new RegionCreateFunction();
 
+  private static final RegionConfigRealizer realizer = new RegionConfigRealizer();
+
   @Override
   public boolean isHA() {
     return false;
@@ -93,12 +75,12 @@ public class RegionCreateFunction implements InternalFunction {
     }
 
     try {
-      Region<?, ?> createdRegion =
-          createRegion(cache, regionCreateArgs.getConfig(), regionCreateArgs.getRegionPath());
-      XmlEntity xmlEntity = getXmlEntityForRegion(createdRegion);
+      RegionPath regionPath = new RegionPath(regionCreateArgs.getRegionPath());
+      realizer.create(regionCreateArgs.getConfig(), regionCreateArgs.getRegionPath(), cache);
+      XmlEntity xmlEntity = new XmlEntity(CacheXml.REGION, "name", regionPath.getRootRegionName());
       resultSender.lastResult(new CliFunctionResult(memberNameOrId, xmlEntity.getXmlDefinition(),
           CliStrings.format(CliStrings.CREATE_REGION__MSG__REGION_0_CREATED_ON_1,
-              createdRegion.getFullPath(), memberNameOrId)));
+              regionCreateArgs.getRegionPath(), memberNameOrId)));
     } catch (IllegalStateException e) {
       String exceptionMsg = e.getMessage();
       String localizedString =
@@ -125,15 +107,6 @@ public class RegionCreateFunction implements InternalFunction {
     }
   }
 
-  private XmlEntity getXmlEntityForRegion(Region<?, ?> region) {
-    Region<?, ?> curRegion = region;
-    while (curRegion != null && curRegion.getParentRegion() != null) {
-      curRegion = curRegion.getParentRegion();
-    }
-
-    return new XmlEntity(CacheXml.REGION, "name", curRegion.getName());
-  }
-
   private CliFunctionResult handleException(final String memberNameOrId, final String exceptionMsg,
       final Exception e) {
     if (e != null && logger.isDebugEnabled()) {
@@ -148,234 +121,6 @@ public class RegionCreateFunction implements InternalFunction {
     return new CliFunctionResult(memberNameOrId, CliFunctionResult.StatusState.ERROR);
   }
 
-  private <K, V> Region<?, ?> createRegion(Cache cache, RegionConfig config, String regionPath)
-      throws RuntimeException {
-    RegionAttributesType regionAttributes = config.getRegionAttributes();
-    Region<K, V> createdRegion;
-    RegionFactory<K, V> factory = cache.createRegionFactory();
-
-    validateAndSetCustomClasses(regionAttributes, factory);
-
-    if (regionAttributes.getPartitionAttributes() != null) {
-      factory.setPartitionAttributes(
-          PartitionAttributesImpl.fromConfig(regionAttributes.getPartitionAttributes()));
-    }
-
-    factory
-        .setDataPolicy(DataPolicy.fromString(regionAttributes.getDataPolicy().value().toUpperCase()
-            .replace("-", "_")));
-
-    if (regionAttributes.getScope() != null) {
-      factory.setScope(Scope.fromString(regionAttributes.getScope().value().toUpperCase()
-          .replace("-", "_")));
-    }
-
-    validateAndSetExpirationAttributes(regionAttributes, factory);
-
-    if (regionAttributes.getEvictionAttributes() != null) {
-      try {
-        factory.setEvictionAttributes(
-            EvictionAttributesImpl.fromConfig(regionAttributes.getEvictionAttributes()));
-      } catch (Exception e) {
-        throw new IllegalArgumentException(
-            CliStrings.CREATE_REGION__MSG__OBJECT_SIZER_MUST_BE_OBJECTSIZER_AND_DECLARABLE);
-      }
-    }
-
-    if (regionAttributes.getDiskStoreName() != null) {
-      factory.setDiskStoreName(regionAttributes.getDiskStoreName());
-    }
-
-    if (regionAttributes.isDiskSynchronous() != null) {
-      factory.setDiskSynchronous(regionAttributes.isDiskSynchronous());
-    }
-
-    if (regionAttributes.isOffHeap() != null) {
-      factory.setOffHeap(regionAttributes.isOffHeap());
-    }
-
-    if (regionAttributes.isStatisticsEnabled() != null) {
-      factory.setStatisticsEnabled(regionAttributes.isStatisticsEnabled());
-    }
-
-    if (regionAttributes.isEnableAsyncConflation() != null) {
-      factory.setEnableAsyncConflation(regionAttributes.isEnableAsyncConflation());
-    }
-
-    if (regionAttributes.isEnableSubscriptionConflation() != null) {
-      factory.setEnableSubscriptionConflation(regionAttributes.isEnableSubscriptionConflation());
-    }
-
-    if (regionAttributes.getGatewaySenderIds() != null) {
-      Arrays.stream(regionAttributes.getGatewaySenderIds().split(","))
-          .forEach(gsi -> factory.addGatewaySenderId(gsi));
-    }
-
-    if (regionAttributes.getAsyncEventQueueIds() != null) {
-      Arrays.stream(regionAttributes.getAsyncEventQueueIds().split(","))
-          .forEach(gsi -> factory.addAsyncEventQueueId(gsi));
-    }
-
-    factory.setConcurrencyChecksEnabled(regionAttributes.isConcurrencyChecksEnabled());
-
-    if (regionAttributes.getConcurrencyLevel() != null) {
-      factory.setConcurrencyLevel(Integer.valueOf(regionAttributes.getConcurrencyLevel()));
-    }
-
-    if (regionAttributes.isCloningEnabled() != null) {
-      factory.setCloningEnabled(regionAttributes.isCloningEnabled());
-    }
-
-    if (regionAttributes.isMulticastEnabled() != null) {
-      factory.setMulticastEnabled(regionAttributes.isMulticastEnabled());
-    }
-
-    RegionPath regionPathData = new RegionPath(regionPath);
-    String regionName = regionPathData.getName();
-    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);
-    }
-
-    return createdRegion;
-  }
-
-  private <K, V> void validateAndSetExpirationAttributes(RegionAttributesType regionAttributes,
-      RegionFactory<K, V> factory) {
-    if (regionAttributes.getEntryIdleTime() != null) {
-      RegionAttributesType.ExpirationAttributesType eitl = regionAttributes.getEntryIdleTime();
-      factory.setEntryIdleTimeout(
-          new ExpirationAttributes(Integer.valueOf(eitl.getTimeout()),
-              ExpirationAction.fromXmlString(eitl.getAction())));
-
-      try {
-        if (eitl.getCustomExpiry() != null) {
-          factory.setCustomEntryIdleTimeout((CustomExpiry) ClassPathLoader.getLatest()
-              .forName(eitl.getCustomExpiry().getClassName())
-              .newInstance());
-        }
-      } catch (Exception e) {
-      }
-    }
-
-    if (regionAttributes.getEntryTimeToLive() != null) {
-      RegionAttributesType.ExpirationAttributesType ettl = regionAttributes.getEntryTimeToLive();
-      factory.setEntryTimeToLive(
-          new ExpirationAttributes(Integer.valueOf(ettl.getTimeout()),
-              ExpirationAction.fromXmlString(ettl.getAction())));
-
-      try {
-        if (ettl.getCustomExpiry() != null) {
-          factory.setCustomEntryTimeToLive((CustomExpiry) ClassPathLoader.getLatest()
-              .forName(ettl.getCustomExpiry().getClassName())
-              .newInstance());
-        }
-      } catch (Exception e) {
-      }
-    }
-
-    if (regionAttributes.getRegionIdleTime() != null) {
-      RegionAttributesType.ExpirationAttributesType ritl = regionAttributes.getRegionIdleTime();
-      factory.setRegionIdleTimeout(
-          new ExpirationAttributes(Integer.valueOf(ritl.getTimeout()),
-              ExpirationAction.fromXmlString(ritl.getAction())));
-    }
-
-    if (regionAttributes.getRegionTimeToLive() != null) {
-      RegionAttributesType.ExpirationAttributesType rttl = regionAttributes.getRegionTimeToLive();
-      factory.setRegionTimeToLive(
-          new ExpirationAttributes(Integer.valueOf(rttl.getTimeout()),
-              ExpirationAction.fromXmlString(rttl.getAction())));
-    }
-  }
-
-  private <K, V> void validateAndSetCustomClasses(RegionAttributesType regionAttributes,
-      RegionFactory<K, V> factory) {
-    if (regionAttributes.getEntryIdleTime() != null
-        && regionAttributes.getEntryIdleTime().getCustomExpiry() != null) {
-      String customExpiry = regionAttributes.getEntryIdleTime().getCustomExpiry().getClassName();
-      String neededFor = CliStrings.ENTRY_IDLE_TIME_CUSTOM_EXPIRY;
-      Class<CustomExpiry> customExpiryClass = CliUtil.forName(customExpiry, neededFor);
-      CliUtil.newInstance(customExpiryClass, neededFor);
-    }
-
-    if (regionAttributes.getEntryTimeToLive() != null
-        && regionAttributes.getEntryTimeToLive().getCustomExpiry() != null) {
-      String customExpiry = regionAttributes.getEntryTimeToLive().getCustomExpiry().getClassName();
-      String neededFor = CliStrings.ENTRY_TTL_CUSTOM_EXPIRY;
-      Class<CustomExpiry> customExpiryClass = CliUtil.forName(customExpiry, neededFor);
-      CliUtil.newInstance(customExpiryClass, neededFor);
-    }
-
-    if (regionAttributes.getPartitionAttributes() != null
-        && regionAttributes.getPartitionAttributes().getPartitionResolver() != null) {
-      String partitionResolver =
-          regionAttributes.getPartitionAttributes().getPartitionResolver().getClassName();
-      String neededFor = CliStrings.CREATE_REGION__PARTITION_RESOLVER;
-      Class<PartitionResolver> partitionResolverClass =
-          CliUtil.forName(partitionResolver, neededFor);
-      CliUtil.newInstance(partitionResolverClass, neededFor);
-    }
-
-    if (regionAttributes.getCacheLoader() != null) {
-      String cacheLoader =
-          regionAttributes.getCacheLoader().getClassName();
-      String neededFor = CliStrings.CREATE_REGION__CACHELOADER;
-      Class<CacheLoader> cacheLoaderClass =
-          CliUtil.forName(cacheLoader, neededFor);
-      CacheLoader loader = CliUtil.newInstance(cacheLoaderClass, neededFor);
-      factory.setCacheLoader(loader);
-    }
-
-    if (regionAttributes.getCacheWriter() != null) {
-      String cacheWriter =
-          regionAttributes.getCacheWriter().getClassName();
-      String neededFor = CliStrings.CREATE_REGION__CACHEWRITER;
-      Class<CacheWriter> cacheWriterClass =
-          CliUtil.forName(cacheWriter, neededFor);
-      CacheWriter writer = CliUtil.newInstance(cacheWriterClass, neededFor);
-      factory.setCacheWriter(writer);
-    }
-
-    if (regionAttributes.getCacheListeners() != null) {
-      List<DeclarableType> configListeners = regionAttributes.getCacheListeners();
-      CacheListener[] listeners = new CacheListener[configListeners.size()];
-      String neededFor = CliStrings.CREATE_REGION__CACHELISTENER;
-      for (int i = 0; i < configListeners.size(); i++) {
-        String listener = configListeners.get(i).getClassName();
-        Class<CacheListener> cacheListenerClass = CliUtil.forName(listener, neededFor);
-        listeners[i] = CliUtil.newInstance(cacheListenerClass, neededFor);
-      }
-      factory.initCacheListeners(listeners);
-    }
-
-    final String keyConstraint = (String) regionAttributes.getKeyConstraint();
-    final String valueConstraint = regionAttributes.getValueConstraint();
-    if (keyConstraint != null && !keyConstraint.isEmpty()) {
-      Class<K> keyConstraintClass =
-          CliUtil.forName(keyConstraint, CliStrings.CREATE_REGION__KEYCONSTRAINT);
-      factory.setKeyConstraint(keyConstraintClass);
-    }
-
-    if (valueConstraint != null && !valueConstraint.isEmpty()) {
-      Class<V> valueConstraintClass =
-          CliUtil.forName(valueConstraint, CliStrings.CREATE_REGION__VALUECONSTRAINT);
-      factory.setValueConstraint(valueConstraintClass);
-    }
-
-    if (regionAttributes.getCompressor() != null) {
-      Class<Compressor> compressorKlass =
-          CliUtil.forName(regionAttributes.getCompressor().getClassName(),
-              CliStrings.CREATE_REGION__COMPRESSOR);
-      factory.setCompressor(
-          CliUtil.newInstance(compressorKlass, CliStrings.CREATE_REGION__COMPRESSOR));
-    }
-  }
-
-
   @Override
   public String getId() {
     return ID;
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/UpdateCacheFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/UpdateCacheFunction.java
index 5fc3e54..fdec13b 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/UpdateCacheFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/UpdateCacheFunction.java
@@ -18,15 +18,24 @@
 package org.apache.geode.management.internal.cli.functions;
 
 import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.collections.map.HashedMap;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.configuration.CacheElement;
+import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.management.cli.CliFunction;
 import org.apache.geode.management.internal.configuration.realizers.ConfigurationRealizer;
-import org.apache.geode.management.internal.configuration.realizers.ConfigurationRealizerFactory;
+import org.apache.geode.management.internal.configuration.realizers.RegionConfigRealizer;
 
 public class UpdateCacheFunction extends CliFunction<List> {
+  private static Map<Class, ConfigurationRealizer> realizers = new HashedMap();
+  static {
+    realizers.put(RegionConfig.class, new RegionConfigRealizer());
+  }
+
   public enum CacheElementOperation {
     ADD, DELETE, UPDATE
   }
@@ -37,7 +46,7 @@ public class UpdateCacheFunction extends CliFunction<List> {
     CacheElementOperation operation = (CacheElementOperation) context.getArguments().get(1);
     Cache cache = context.getCache();
 
-    ConfigurationRealizer realizer = (new ConfigurationRealizerFactory()).generate(cacheElement);
+    ConfigurationRealizer realizer = realizers.get(cacheElement.getClass());
     switch (operation) {
       case ADD:
         realizer.create(cacheElement, cache);
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/RegionPath.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/RegionPath.java
index 1d3c5eb..f909f8b 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/RegionPath.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/RegionPath.java
@@ -18,10 +18,7 @@ import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 
-import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
-import org.apache.geode.management.ManagementException;
-import org.apache.geode.management.ManagementService;
 
 /**
  * Class to handle Region path.
@@ -74,6 +71,10 @@ public class RegionPath {
   }
 
   public String[] getRegionsOnParentPath() {
+    if (getParent() == null) {
+      return new String[] {};
+    }
+
     String[] regionsOnPath = getParent().split(Region.SEPARATOR);
 
     // Ignore preceding separator if there is one
@@ -87,6 +88,14 @@ public class RegionPath {
     return regions.toArray(new String[] {});
   }
 
+  public String getRootRegionName() {
+    if (isRoot()) {
+      return getName();
+    } else {
+      return getRegionsOnParentPath()[0];
+    }
+  }
+
   /**
    * @return Parent RegionPath of this RegionPath. null if this is a root region
    */
@@ -101,26 +110,6 @@ public class RegionPath {
     return regionParentPath == null;
   }
 
-  public boolean existsInCache(Cache cache) {
-    return cache != null && cache.getRegion(regionPath) != null;
-  }
-
-  public boolean existsInCluster(Cache cache) {
-    boolean existsInCluster = false;
-
-    if (cache != null) {
-      ManagementService managementService = ManagementService.getExistingManagementService(cache);
-      if (managementService.isManager()) {
-        existsInCluster = managementService != null
-            && managementService.getDistributedRegionMXBean(regionPath) != null;
-      } else {
-        throw new ManagementException("Not a cache from Manager member.");
-      }
-    }
-
-    return existsInCluster;
-  }
-
   @Override
   public String toString() {
     return "RegionPath [regionPath=" + regionPath + "]";
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/domain/DeclarableTypeInstantiator.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/domain/DeclarableTypeInstantiator.java
index 3921dc8..a71a4c4 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/domain/DeclarableTypeInstantiator.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/domain/DeclarableTypeInstantiator.java
@@ -18,6 +18,7 @@ import java.util.Properties;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Declarable;
+import org.apache.geode.cache.configuration.ClassNameType;
 import org.apache.geode.cache.configuration.DeclarableType;
 import org.apache.geode.cache.configuration.ObjectType;
 import org.apache.geode.cache.configuration.ParameterType;
@@ -43,6 +44,7 @@ public abstract class DeclarableTypeInstantiator {
     }
   }
 
+
   public static <T> T newInstance(ObjectType objectType, Cache cache) {
     if (objectType.getString() != null) {
       return (T) objectType.getString();
@@ -69,4 +71,15 @@ public abstract class DeclarableTypeInstantiator {
       throw new RuntimeException("Error instantiating class: <" + type.getClassName() + ">", e);
     }
   }
+
+  public static <V> V newInstance(ClassNameType className) {
+    try {
+      Class<V> loadedClass =
+          (Class<V>) ClassPathLoader.getLatest().forName(className.getClassName());
+      return loadedClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Error instantiating class: <" + className.getClassName() + ">",
+          e);
+    }
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/mutators/ConfigurationMutatorFactory.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/mutators/ConfigurationMutatorFactory.java
deleted file mode 100644
index 77e8dce..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/mutators/ConfigurationMutatorFactory.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.configuration.mutators;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.geode.cache.configuration.CacheElement;
-import org.apache.geode.cache.configuration.RegionConfig;
-
-public class ConfigurationMutatorFactory {
-  private final Map<Class, ConfigurationMutator> configurationMutatorMap = new HashMap<>();
-
-  public ConfigurationMutatorFactory() {
-    configurationMutatorMap.put(RegionConfig.class, new RegionConfigMutator());
-  }
-
-  public ConfigurationMutator generate(CacheElement config) {
-    return configurationMutatorMap.get(config.getClass());
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/realizers/ConfigurationRealizerFactory.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/realizers/ConfigurationRealizerFactory.java
deleted file mode 100644
index 5d93893..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/realizers/ConfigurationRealizerFactory.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.configuration.realizers;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.geode.cache.configuration.CacheElement;
-import org.apache.geode.cache.configuration.RegionConfig;
-
-public class ConfigurationRealizerFactory {
-  private final Map<Class, ConfigurationRealizer> realizerMap = new HashMap<>();
-
-  public ConfigurationRealizerFactory() {
-    this.realizerMap.put(RegionConfig.class, new RegionConfigRealizer());
-  }
-
-  public ConfigurationRealizer generate(CacheElement config) {
-    return realizerMap.get(config.getClass());
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizer.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizer.java
index 5803c53..67ceb2c 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizer.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizer.java
@@ -17,50 +17,216 @@
 
 package org.apache.geode.management.internal.configuration.realizers;
 
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheListener;
 import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.ExpirationAction;
+import org.apache.geode.cache.ExpirationAttributes;
+import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
-import org.apache.geode.cache.configuration.RegionAttributesDataPolicy;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.cache.configuration.DeclarableType;
 import org.apache.geode.cache.configuration.RegionAttributesType;
 import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.internal.cache.EvictionAttributesImpl;
 import org.apache.geode.internal.cache.PartitionAttributesImpl;
+import org.apache.geode.management.internal.cli.CliUtil;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.util.RegionPath;
+import org.apache.geode.management.internal.configuration.domain.DeclarableTypeInstantiator;
 
 public class RegionConfigRealizer implements ConfigurationRealizer<RegionConfig> {
   public RegionConfigRealizer() {}
 
+  /**
+   * this is used to create only root regions.
+   *
+   * @param regionConfig the name in the regionConfig can not contain sub-regions.
+   */
   @Override
   public void create(RegionConfig regionConfig, Cache cache) {
-    String regionPath = regionConfig.getName();
-    if (regionConfig.getRegionAttributes() == null) {
-      regionConfig.setRegionAttributes(new RegionAttributesType());
-    }
-
-    RegionAttributesType regionAttributes = regionConfig.getRegionAttributes();
-    switch (regionConfig.getRefid()) {
-      case "PARTITION":
-        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
-        RegionAttributesType.PartitionAttributes partitionAttributes =
-            new RegionAttributesType.PartitionAttributes();
-        partitionAttributes.setRedundantCopies("1");
-        regionAttributes.setPartitionAttributes(partitionAttributes);
-        break;
-      case "REPLICATE":
-        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
-        break;
-      default:
-        break;
+    RegionFactory factory = getRegionFactory(cache, regionConfig.getRegionAttributes());
+    factory.create(regionConfig.getName());
+  }
+
+  /**
+   * this method supports creating root region and sub-regions.
+   * We need this because CreateRegionCommand should still
+   * support creating sub regions.
+   *
+   * @param regionConfig the name in regionConfig is ignored.
+   * @param regionPath this is the full path of the region
+   */
+  public void create(RegionConfig regionConfig, String regionPath, Cache cache) {
+    RegionFactory factory = getRegionFactory(cache, regionConfig.getRegionAttributes());
+    RegionPath regionPathData = new RegionPath(regionPath);
+    String regionName = regionPathData.getName();
+    String parentRegionPath = regionPathData.getParent();
+    if (parentRegionPath == null) {
+      factory.create(regionName);
+      return;
     }
 
+    Region parentRegion = cache.getRegion(parentRegionPath);
+    factory.createSubregion(parentRegion, regionName);
+  }
+
+  private RegionFactory getRegionFactory(Cache cache, RegionAttributesType regionAttributes) {
     RegionFactory factory = cache.createRegionFactory();
+
+    factory.setDataPolicy(DataPolicy.fromString(regionAttributes.getDataPolicy().name()));
+
+    if (regionAttributes.getScope() != null) {
+      factory.setScope(Scope.fromString(regionAttributes.getScope().name()));
+    }
+
+    if (regionAttributes.getCacheLoader() != null) {
+      ((RegionFactory<Object, Object>) factory)
+          .setCacheLoader(DeclarableTypeInstantiator.newInstance(regionAttributes.getCacheLoader(),
+              cache));
+    }
+
+    if (regionAttributes.getCacheWriter() != null) {
+      ((RegionFactory<Object, Object>) factory)
+          .setCacheWriter(DeclarableTypeInstantiator.newInstance(regionAttributes.getCacheWriter(),
+              cache));
+    }
+
+    if (regionAttributes.getCacheListeners() != null) {
+      List<DeclarableType> configListeners = regionAttributes.getCacheListeners();
+      CacheListener[] listeners = new CacheListener[configListeners.size()];
+      for (int i = 0; i < configListeners.size(); i++) {
+        listeners[i] = DeclarableTypeInstantiator.newInstance(configListeners.get(i), cache);
+      }
+      ((RegionFactory<Object, Object>) factory).initCacheListeners(listeners);
+    }
+
+    final String keyConstraint = regionAttributes.getKeyConstraint();
+    final String valueConstraint = regionAttributes.getValueConstraint();
+    if (keyConstraint != null && !keyConstraint.isEmpty()) {
+      Class<Object> keyConstraintClass =
+          CliUtil.forName(keyConstraint, CliStrings.CREATE_REGION__KEYCONSTRAINT);
+      ((RegionFactory<Object, Object>) factory).setKeyConstraint(keyConstraintClass);
+    }
+
+    if (valueConstraint != null && !valueConstraint.isEmpty()) {
+      Class<Object> valueConstraintClass =
+          CliUtil.forName(valueConstraint, CliStrings.CREATE_REGION__VALUECONSTRAINT);
+      ((RegionFactory<Object, Object>) factory).setValueConstraint(valueConstraintClass);
+    }
+
+    if (regionAttributes.getCompressor() != null) {
+      ((RegionFactory<Object, Object>) factory)
+          .setCompressor(DeclarableTypeInstantiator.newInstance(regionAttributes.getCompressor()));
+    }
+
     if (regionAttributes.getPartitionAttributes() != null) {
       factory.setPartitionAttributes(
-          PartitionAttributesImpl.fromConfig(regionAttributes.getPartitionAttributes()));
+          PartitionAttributesImpl.fromConfig(regionAttributes.getPartitionAttributes(), cache));
+    }
+
+    if (regionAttributes.getEntryIdleTime() != null) {
+      RegionAttributesType.ExpirationAttributesType eitl = regionAttributes.getEntryIdleTime();
+      ((RegionFactory<Object, Object>) factory).setEntryIdleTimeout(
+          new ExpirationAttributes(Integer.valueOf(eitl.getTimeout()),
+              ExpirationAction.fromXmlString(eitl.getAction())));
+
+
+      if (eitl.getCustomExpiry() != null) {
+        ((RegionFactory<Object, Object>) factory).setCustomEntryIdleTimeout(
+            DeclarableTypeInstantiator.newInstance(eitl.getCustomExpiry(),
+                cache));
+      }
+    }
+
+    if (regionAttributes.getEntryTimeToLive() != null) {
+      RegionAttributesType.ExpirationAttributesType ettl = regionAttributes.getEntryTimeToLive();
+      ((RegionFactory<Object, Object>) factory).setEntryTimeToLive(
+          new ExpirationAttributes(Integer.valueOf(ettl.getTimeout()),
+              ExpirationAction.fromXmlString(ettl.getAction())));
+
+      if (ettl.getCustomExpiry() != null) {
+        ((RegionFactory<Object, Object>) factory)
+            .setCustomEntryTimeToLive(DeclarableTypeInstantiator.newInstance(ettl.getCustomExpiry(),
+                cache));
+      }
+    }
+
+    if (regionAttributes.getRegionIdleTime() != null) {
+      RegionAttributesType.ExpirationAttributesType ritl = regionAttributes.getRegionIdleTime();
+      ((RegionFactory<Object, Object>) factory).setRegionIdleTimeout(
+          new ExpirationAttributes(Integer.valueOf(ritl.getTimeout()),
+              ExpirationAction.fromXmlString(ritl.getAction())));
+    }
+
+    if (regionAttributes.getRegionTimeToLive() != null) {
+      RegionAttributesType.ExpirationAttributesType rttl = regionAttributes.getRegionTimeToLive();
+      ((RegionFactory<Object, Object>) factory).setRegionTimeToLive(
+          new ExpirationAttributes(Integer.valueOf(rttl.getTimeout()),
+              ExpirationAction.fromXmlString(rttl.getAction())));
+    }
+
+    if (regionAttributes.getEvictionAttributes() != null) {
+      try {
+        factory.setEvictionAttributes(
+            EvictionAttributesImpl.fromConfig(regionAttributes.getEvictionAttributes()));
+      } catch (Exception e) {
+        throw new IllegalArgumentException(
+            CliStrings.CREATE_REGION__MSG__OBJECT_SIZER_MUST_BE_OBJECTSIZER_AND_DECLARABLE);
+      }
+    }
+
+    if (regionAttributes.getDiskStoreName() != null) {
+      factory.setDiskStoreName(regionAttributes.getDiskStoreName());
+    }
+
+    if (regionAttributes.isDiskSynchronous() != null) {
+      factory.setDiskSynchronous(regionAttributes.isDiskSynchronous());
+    }
+
+    if (regionAttributes.isOffHeap() != null) {
+      factory.setOffHeap(regionAttributes.isOffHeap());
+    }
+
+    if (regionAttributes.isStatisticsEnabled() != null) {
+      factory.setStatisticsEnabled(regionAttributes.isStatisticsEnabled());
+    }
+
+    if (regionAttributes.isEnableAsyncConflation() != null) {
+      factory.setEnableAsyncConflation(regionAttributes.isEnableAsyncConflation());
     }
 
-    factory
-        .setDataPolicy(DataPolicy.fromString(regionAttributes.getDataPolicy().value().toUpperCase()
-            .replace("-", "_")));
-    factory.create(regionPath);
+    if (regionAttributes.isEnableSubscriptionConflation() != null) {
+      factory.setEnableSubscriptionConflation(regionAttributes.isEnableSubscriptionConflation());
+    }
+
+    if (regionAttributes.getGatewaySenderIds() != null) {
+      Arrays.stream(regionAttributes.getGatewaySenderIds().split(","))
+          .forEach(gsi -> factory.addGatewaySenderId(gsi));
+    }
+
+    if (regionAttributes.getAsyncEventQueueIds() != null) {
+      Arrays.stream(regionAttributes.getAsyncEventQueueIds().split(","))
+          .forEach(gsi -> factory.addAsyncEventQueueId(gsi));
+    }
+
+    factory.setConcurrencyChecksEnabled(regionAttributes.isConcurrencyChecksEnabled());
+
+    if (regionAttributes.getConcurrencyLevel() != null) {
+      factory.setConcurrencyLevel(Integer.valueOf(regionAttributes.getConcurrencyLevel()));
+    }
+
+    if (regionAttributes.isCloningEnabled() != null) {
+      factory.setCloningEnabled(regionAttributes.isCloningEnabled());
+    }
+
+    if (regionAttributes.isMulticastEnabled() != null) {
+      factory.setMulticastEnabled(regionAttributes.isMulticastEnabled());
+    }
+    return factory;
   }
 
   @Override
@@ -73,4 +239,6 @@ public class RegionConfigRealizer implements ConfigurationRealizer<RegionConfig>
 
   @Override
   public void delete(RegionConfig config, Cache cache) {}
+
+
 }
diff --git a/geode-management/src/main/java/org/apache/geode/management/internal/api/ClusterManagementService.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/ConfigurationValidator.java
similarity index 52%
copy from geode-management/src/main/java/org/apache/geode/management/internal/api/ClusterManagementService.java
copy to geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/ConfigurationValidator.java
index e3c82e4..fec118e 100644
--- a/geode-management/src/main/java/org/apache/geode/management/internal/api/ClusterManagementService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/ConfigurationValidator.java
@@ -12,28 +12,11 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.management.internal.configuration.validators;
 
-package org.apache.geode.management.internal.api;
-
-import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.configuration.CacheElement;
 
-/**
- * this is responsible for applying and persisting cache configuration changes on locators
- * and/or servers.
- */
-@Experimental
-public interface ClusterManagementService {
-  /**
-   * This method will try to create the element on all the applicable members in the cluster and
-   * persist the configuration in the cluster configuration if persistence is enabled.
-   *
-   * @param config this holds the configuration attributes of the element you are trying to create
-   *        on the cluster
-   */
-  ClusterManagementResult create(CacheElement config);
-
-  ClusterManagementResult delete(CacheElement config);
+public interface ConfigurationValidator<T extends CacheElement> {
 
-  ClusterManagementResult update(CacheElement config);
+  public void validate(T config);
 }
diff --git a/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java b/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java
index ea76196..737c04c 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java
@@ -17,29 +17,64 @@ package org.apache.geode.cache.configuration;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
 
+import java.io.File;
+import java.net.URL;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Before;
 import org.junit.Test;
 
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.internal.config.JAXBService;
+
 public class RegionConfigTest {
+
+  private JAXBService service;
+  private CacheConfig cacheConfig, master;
+  private RegionConfig regionConfig;
+  private URL xmlResource;
+
+  @Before
+  public void before() throws Exception {
+    service = new JAXBService(CacheConfig.class);
+    regionConfig = new RegionConfig();
+    xmlResource = RegionConfigTest.class.getResource("RegionConfigTest.xml");
+    assertThat(xmlResource).isNotNull();
+    master =
+        service.unMarshall(FileUtils.readFileToString(new File(xmlResource.getFile()), "UTF-8"));
+  }
+
   @Test
   public void regionNameCannotBeNull() {
-    RegionConfig config = new RegionConfig();
-    assertThatThrownBy(() -> config.setName(null))
+    assertThatThrownBy(() -> regionConfig.setName(null))
         .isInstanceOf(IllegalArgumentException.class);
   }
 
   @Test
   public void regionNameSwallowsSlash() {
-    RegionConfig config = new RegionConfig();
-    config.setName("/regionA");
-    assertThat(config.getName()).isEqualTo("regionA");
+    regionConfig.setName("/regionA");
+    assertThat(regionConfig.getName()).isEqualTo("regionA");
   }
 
   @Test
   public void subRegionsUnsupported() {
-    RegionConfig config = new RegionConfig();
-    assertThatThrownBy(() -> config.setName("/Parent/Child"))
+    regionConfig = new RegionConfig();
+    assertThatThrownBy(() -> regionConfig.setName("/Parent/Child"))
         .isInstanceOf(IllegalArgumentException.class);
-    assertThatThrownBy(() -> config.setName("Parent/Child"))
+    assertThatThrownBy(() -> regionConfig.setName("Parent/Child"))
         .isInstanceOf(IllegalArgumentException.class);
   }
+
+  @Test
+  public void checkDefaultRegionAttributesForShortcuts() throws Exception {
+    RegionShortcut[] shortcuts = RegionShortcut.values();
+    for (RegionShortcut shortcut : shortcuts) {
+      RegionConfig config = new RegionConfig();
+      config.setRefid(shortcut.name());
+      config.setName(shortcut.name());
+      RegionConfig masterRegion = CacheElement.findElement(master.getRegions(), shortcut.name());
+      assertThat(config).isEqualToComparingFieldByFieldRecursively(masterRegion);
+    }
+  }
+
 }
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalConfigurationPersistenceServiceTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalConfigurationPersistenceServiceTest.java
index 33b9174..574cc07 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalConfigurationPersistenceServiceTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalConfigurationPersistenceServiceTest.java
@@ -89,7 +89,7 @@ public class InternalConfigurationPersistenceServiceTest {
 
     System.out.println(configuration.getCacheXmlContent());
     assertThat(configuration.getCacheXmlContent())
-        .contains("<region name=\"regionA\" refid=\"REPLICATE\"/>");
+        .contains("<region name=\"regionA\" refid=\"REPLICATE\">");
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/util/RegionPathTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/util/RegionPathTest.java
new file mode 100644
index 0000000..f11b5d4
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/util/RegionPathTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.util;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Test;
+
+public class RegionPathTest {
+
+  @Test
+  public void rootName() {
+    RegionPath path = new RegionPath("test");
+    assertThat(path.getParent()).isNull();
+    assertThat(path.getName()).isEqualTo("test");
+    assertThat(path.isRoot()).isTrue();
+    assertThat(path.getRegionsOnParentPath()).isEmpty();
+    assertThat(path.getRootRegionName()).isEqualTo("test");
+  }
+
+  @Test
+  public void rootNameWithSlash() {
+    RegionPath path = new RegionPath("/test");
+    assertThat(path.getParent()).isNull();
+    assertThat(path.getName()).isEqualTo("test");
+    assertThat(path.isRoot()).isTrue();
+    assertThat(path.getRegionsOnParentPath()).isEmpty();
+    assertThat(path.getRootRegionName()).isEqualTo("test");
+  }
+
+  @Test
+  public void subRegionName() {
+    RegionPath path = new RegionPath("test1/test2");
+    assertThat(path.getParent()).isEqualTo("/test1");
+    assertThat(path.getName()).isEqualTo("test2");
+    assertThat(path.isRoot()).isFalse();
+    assertThat(path.getRegionsOnParentPath()).containsExactly("test1");
+    assertThat(path.getRootRegionName()).isEqualTo("test1");
+  }
+
+  @Test
+  public void subRegionNameWithSlash() {
+    RegionPath path = new RegionPath("/test1/test2");
+    assertThat(path.getParent()).isEqualTo("/test1");
+    assertThat(path.getName()).isEqualTo("test2");
+    assertThat(path.isRoot()).isFalse();
+    assertThat(path.getRegionsOnParentPath()).containsExactly("test1");
+    assertThat(path.getRootRegionName()).isEqualTo("test1");
+  }
+
+  @Test
+  public void subSubRegionName() {
+    RegionPath path = new RegionPath("test1/test2/test3");
+    assertThat(path.getParent()).isEqualTo("/test1/test2");
+    assertThat(path.getName()).isEqualTo("test3");
+    assertThat(path.isRoot()).isFalse();
+    assertThat(path.getRegionsOnParentPath()).containsExactly("test1", "test2");
+    assertThat(path.getRootRegionName()).isEqualTo("test1");
+  }
+
+  @Test
+  public void subSubRegionNameWithSlash() {
+    RegionPath path = new RegionPath("/test1/test2/test3");
+    assertThat(path.getParent()).isEqualTo("/test1/test2");
+    assertThat(path.getName()).isEqualTo("test3");
+    assertThat(path.isRoot()).isFalse();
+    assertThat(path.getRegionsOnParentPath()).containsExactly("test1", "test2");
+    assertThat(path.getRootRegionName()).isEqualTo("test1");
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/mutators/ConfigurationMutatorFactoryTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/mutators/ConfigurationMutatorFactoryTest.java
deleted file mode 100644
index 44d94a1..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/mutators/ConfigurationMutatorFactoryTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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.configuration.mutators;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-import org.junit.Test;
-
-import org.apache.geode.cache.configuration.RegionConfig;
-
-public class ConfigurationMutatorFactoryTest {
-
-  @Test
-  public void generatesConfigurationMutatorFromConfigObjectType() {
-    ConfigurationMutatorFactory subject = new ConfigurationMutatorFactory();
-
-    RegionConfig config = new RegionConfig();
-    ConfigurationMutator configurationMutator = subject.generate(config);
-
-    assertThat(configurationMutator).isInstanceOf(RegionConfigMutator.class);
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/realizers/ConfigurationRealizerFactoryTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/realizers/ConfigurationRealizerFactoryTest.java
deleted file mode 100644
index c0abcc4..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/realizers/ConfigurationRealizerFactoryTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.configuration.realizers;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-import org.junit.Test;
-
-import org.apache.geode.cache.configuration.RegionConfig;
-
-public class ConfigurationRealizerFactoryTest {
-  @Test
-  public void generatesConfigurationRealizerFromConfigObjectType() {
-    ConfigurationRealizerFactory subject = new ConfigurationRealizerFactory();
-
-    RegionConfig config = new RegionConfig();
-    ConfigurationRealizer realizer = subject.generate(config);
-
-    assertThat(realizer).isInstanceOf(RegionConfigRealizer.class);
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizerTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizerTest.java
index 7972f5b..55b127c 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizerTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizerTest.java
@@ -26,7 +26,6 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
 import org.apache.geode.cache.DataPolicy;
-import org.apache.geode.cache.PartitionAttributes;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.internal.cache.InternalCache;
@@ -48,21 +47,13 @@ public class RegionConfigRealizerTest {
     config.setName("regionName");
     config.setRefid("PARTITION");
 
-    RegionConfigRealizer subject = new RegionConfigRealizer();
-    subject.create(config, cache);
+    RegionConfigRealizer realizer = new RegionConfigRealizer();
+    realizer.create(config, cache);
 
     ArgumentCaptor<DataPolicy> dataPolicyArgumentCaptor = ArgumentCaptor.forClass(DataPolicy.class);
     verify(regionFactory).setDataPolicy(dataPolicyArgumentCaptor.capture());
     assertThat(dataPolicyArgumentCaptor.getValue()).isEqualTo(DataPolicy.PARTITION);
 
-    ArgumentCaptor<PartitionAttributes> partitionAttributesArgumentCaptor =
-        ArgumentCaptor.forClass(PartitionAttributes.class);
-    verify(regionFactory).setPartitionAttributes(partitionAttributesArgumentCaptor.capture());
-
-    PartitionAttributes partitionAttributes = partitionAttributesArgumentCaptor.getValue();
-    assertThat(partitionAttributes).isNotNull();
-    assertThat(partitionAttributes.getRedundantCopies()).isEqualTo(1);
-
     verify(regionFactory).create("regionName");
   }
 
diff --git a/geode-core/src/test/resources/org/apache/geode/cache/configuration/RegionConfigTest.xml b/geode-core/src/test/resources/org/apache/geode/cache/configuration/RegionConfigTest.xml
new file mode 100644
index 0000000..ba8af9c
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/cache/configuration/RegionConfigTest.xml
@@ -0,0 +1,149 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  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.
+-->
+<cache>
+    <region name="PARTITION" refid="PARTITION">
+        <region-attributes data-policy="partition"/>
+    </region>
+    <region name="PARTITION_REDUNDANT" refid="PARTITION_REDUNDANT">
+        <region-attributes data-policy="partition">
+            <partition-attributes redundant-copies="1"/>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_PERSISTENT" refid="PARTITION_PERSISTENT">
+        <region-attributes data-policy="persistent-partition"/>
+    </region>
+    <region name="PARTITION_REDUNDANT_PERSISTENT" refid="PARTITION_REDUNDANT_PERSISTENT">
+        <region-attributes data-policy="persistent-partition">
+            <partition-attributes redundant-copies="1"/>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_OVERFLOW" refid="PARTITION_OVERFLOW">
+        <region-attributes data-policy="partition">
+            <eviction-attributes>
+                <lru-heap-percentage action="overflow-to-disk"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_REDUNDANT_OVERFLOW" refid="PARTITION_REDUNDANT_OVERFLOW">
+        <region-attributes data-policy="partition">
+            <partition-attributes redundant-copies="1"/>
+            <eviction-attributes>
+                <lru-heap-percentage action="overflow-to-disk"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_PERSISTENT_OVERFLOW" refid="PARTITION_PERSISTENT_OVERFLOW">
+        <region-attributes data-policy="persistent-partition">
+            <eviction-attributes>
+                <lru-heap-percentage action="overflow-to-disk"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_REDUNDANT_PERSISTENT_OVERFLOW"
+            refid="PARTITION_REDUNDANT_PERSISTENT_OVERFLOW">
+        <region-attributes data-policy="persistent-partition">
+            <partition-attributes redundant-copies="1"/>
+            <eviction-attributes>
+                <lru-heap-percentage action="overflow-to-disk"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_HEAP_LRU" refid="PARTITION_HEAP_LRU">
+        <region-attributes data-policy="partition">
+            <eviction-attributes>
+                <lru-heap-percentage action="local-destroy"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_REDUNDANT_HEAP_LRU" refid="PARTITION_REDUNDANT_HEAP_LRU">
+        <region-attributes data-policy="partition">
+            <partition-attributes redundant-copies="1"/>
+            <eviction-attributes>
+                <lru-heap-percentage action="local-destroy"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="REPLICATE" refid="REPLICATE">
+        <region-attributes data-policy="replicate" scope="distributed-ack"/>
+    </region>
+    <region name="REPLICATE_PERSISTENT" refid="REPLICATE_PERSISTENT">
+        <region-attributes data-policy="persistent-replicate" scope="distributed-ack"/>
+    </region>
+    <region name="REPLICATE_OVERFLOW" refid="REPLICATE_OVERFLOW">
+        <region-attributes data-policy="replicate" scope="distributed-ack">
+            <eviction-attributes>
+                <lru-heap-percentage action="overflow-to-disk"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="REPLICATE_PERSISTENT_OVERFLOW" refid="REPLICATE_PERSISTENT_OVERFLOW">
+        <region-attributes data-policy="persistent-replicate" scope="distributed-ack">
+            <eviction-attributes>
+                <lru-heap-percentage action="overflow-to-disk"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="REPLICATE_HEAP_LRU" refid="REPLICATE_HEAP_LRU">
+        <region-attributes data-policy="preloaded" scope="distributed-ack">
+            <subscription-attributes interest-policy="all"/>
+            <eviction-attributes>
+                <lru-heap-percentage action="local-destroy"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="LOCAL" refid="LOCAL">
+        <region-attributes scope="local"/>
+    </region>
+    <region name="LOCAL_PERSISTENT" refid="LOCAL_PERSISTENT">
+        <region-attributes data-policy="persistent-replicate" scope="local"/>
+    </region>
+    <region name="LOCAL_HEAP_LRU" refid="LOCAL_HEAP_LRU">
+        <region-attributes scope="local">
+            <eviction-attributes>
+                <lru-heap-percentage action="local-destroy"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="LOCAL_OVERFLOW" refid="LOCAL_OVERFLOW">
+        <region-attributes scope="local">
+            <eviction-attributes>
+                <lru-heap-percentage action="overflow-to-disk"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="LOCAL_PERSISTENT_OVERFLOW" refid="LOCAL_PERSISTENT_OVERFLOW">
+        <region-attributes data-policy="persistent-replicate" scope="local">
+            <eviction-attributes>
+                <lru-heap-percentage action="overflow-to-disk"/>
+            </eviction-attributes>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_PROXY" refid="PARTITION_PROXY">
+        <region-attributes data-policy="partition">
+            <partition-attributes local-max-memory="0"/>
+        </region-attributes>
+    </region>
+    <region name="PARTITION_PROXY_REDUNDANT" refid="PARTITION_PROXY_REDUNDANT">
+        <region-attributes data-policy="partition">
+            <partition-attributes local-max-memory="0" redundant-copies="1"/>
+        </region-attributes>
+    </region>
+    <region name="REPLICATE_PROXY" refid="REPLICATE_PROXY">
+        <region-attributes data-policy="empty" scope="distributed-ack"/>
+    </region>
+</cache>
\ No newline at end of file
diff --git a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionAttributesDataPolicy.java b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionAttributesDataPolicy.java
index aa17d5b..6230f24 100644
--- a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionAttributesDataPolicy.java
+++ b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionAttributesDataPolicy.java
@@ -54,14 +54,19 @@ import org.apache.geode.annotations.Experimental;
 @XmlEnum
 @Experimental
 public enum RegionAttributesDataPolicy implements Serializable {
-
   @XmlEnumValue("empty")
-  EMPTY("empty"), @XmlEnumValue("normal")
-  NORMAL("normal"), @XmlEnumValue("partition")
-  PARTITION("partition"), @XmlEnumValue("persistent-replicate")
-  PERSISTENT_REPLICATE("persistent-replicate"), @XmlEnumValue("preloaded")
-  PRELOADED("preloaded"), @XmlEnumValue("replicate")
-  REPLICATE("replicate"), @XmlEnumValue("persistent-partition")
+  EMPTY("empty"),
+  @XmlEnumValue("normal")
+  NORMAL("normal"),
+  @XmlEnumValue("partition")
+  PARTITION("partition"),
+  @XmlEnumValue("persistent-replicate")
+  PERSISTENT_REPLICATE("persistent-replicate"),
+  @XmlEnumValue("preloaded")
+  PRELOADED("preloaded"),
+  @XmlEnumValue("replicate")
+  REPLICATE("replicate"),
+  @XmlEnumValue("persistent-partition")
   PERSISTENT_PARTITION("persistent-partition");
   private final String value;
 
diff --git a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionAttributesType.java b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionAttributesType.java
index fde2711..94e2e7d 100644
--- a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionAttributesType.java
+++ b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionAttributesType.java
@@ -345,7 +345,7 @@ import org.apache.geode.cache.ExpirationAction;
 public class RegionAttributesType implements Serializable {
 
   @XmlElement(name = "key-constraint", namespace = "http://geode.apache.org/schema/cache")
-  protected Object keyConstraint;
+  protected String keyConstraint;
   @XmlElement(name = "value-constraint", namespace = "http://geode.apache.org/schema/cache")
   protected String valueConstraint;
   @XmlElement(name = "region-time-to-live", namespace = "http://geode.apache.org/schema/cache")
@@ -440,7 +440,7 @@ public class RegionAttributesType implements Serializable {
    * {@link Object }
    *
    */
-  public Object getKeyConstraint() {
+  public String getKeyConstraint() {
     return keyConstraint;
   }
 
@@ -451,7 +451,7 @@ public class RegionAttributesType implements Serializable {
    * {@link Object }
    *
    */
-  public void setKeyConstraint(Object value) {
+  public void setKeyConstraint(String value) {
     this.keyConstraint = value;
   }
 
@@ -1437,6 +1437,37 @@ public class RegionAttributesType implements Serializable {
     this.offHeap = value;
   }
 
+  public void setLruHeapPercentage(EnumActionDestroyOverflow action) {
+    if (evictionAttributes == null) {
+      evictionAttributes = new EvictionAttributes();
+    }
+    EvictionAttributes.LruHeapPercentage lruHeapPercentage =
+        new EvictionAttributes.LruHeapPercentage();
+    lruHeapPercentage.setAction(action);
+    evictionAttributes.setLruHeapPercentage(lruHeapPercentage);
+  }
+
+  public void setInterestPolicy(String interestPolicy) {
+    if (subscriptionAttributes == null) {
+      subscriptionAttributes = new SubscriptionAttributes();
+    }
+    subscriptionAttributes.setInterestPolicy(interestPolicy);
+  }
+
+  public void setRedundantCopy(String copies) {
+    if (partitionAttributes == null) {
+      partitionAttributes = new PartitionAttributes();
+    }
+    partitionAttributes.setRedundantCopies(copies);
+  }
+
+  public void setLocalMaxMemory(String maxMemory) {
+    if (partitionAttributes == null) {
+      partitionAttributes = new PartitionAttributes();
+    }
+    partitionAttributes.setLocalMaxMemory(maxMemory);
+  }
+
 
   /**
    * <p>
diff --git a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
index a39db22..d97a0d1 100644
--- a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
+++ b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
@@ -352,6 +352,143 @@ public class RegionConfig implements CacheElement {
    */
   public void setRefid(String value) {
     this.refid = value;
+    setShortcutAttributes();
+  }
+
+  private void setShortcutAttributes() {
+    if (regionAttributes == null) {
+      regionAttributes = new RegionAttributesType();
+    }
+
+    switch (refid) {
+      case "PARTITION": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+        break;
+      }
+      case "PARTITION_REDUNDANT": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+        regionAttributes.setRedundantCopy("1");
+        break;
+      }
+      case "PARTITION_PERSISTENT": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION);
+        break;
+      }
+      case "PARTITION_REDUNDANT_PERSISTENT": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION);
+        regionAttributes.setRedundantCopy("1");
+        break;
+      }
+      case "PARTITION_OVERFLOW": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
+        break;
+      }
+      case "PARTITION_REDUNDANT_OVERFLOW": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+        regionAttributes.setRedundantCopy("1");
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
+        break;
+      }
+      case "PARTITION_PERSISTENT_OVERFLOW": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION);
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
+        break;
+
+      }
+      case "PARTITION_REDUNDANT_PERSISTENT_OVERFLOW": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION);
+        regionAttributes.setRedundantCopy("1");
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
+        break;
+
+      }
+      case "PARTITION_HEAP_LRU": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.LOCAL_DESTROY);
+        break;
+
+      }
+      case "PARTITION_REDUNDANT_HEAP_LRU": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+        regionAttributes.setRedundantCopy("1");
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.LOCAL_DESTROY);
+        break;
+      }
+      case "REPLICATE": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+        regionAttributes.setScope(RegionAttributesScope.DISTRIBUTED_ACK);
+        break;
+      }
+      case "REPLICATE_PERSISTENT": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE);
+        regionAttributes.setScope(RegionAttributesScope.DISTRIBUTED_ACK);
+        break;
+      }
+      case "REPLICATE_OVERFLOW": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+        regionAttributes.setScope(RegionAttributesScope.DISTRIBUTED_ACK);
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
+        break;
+
+      }
+      case "REPLICATE_PERSISTENT_OVERFLOW": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE);
+        regionAttributes.setScope(RegionAttributesScope.DISTRIBUTED_ACK);
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
+        break;
+      }
+      case "REPLICATE_HEAP_LRU": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PRELOADED);
+        regionAttributes.setScope(RegionAttributesScope.DISTRIBUTED_ACK);
+        regionAttributes.setInterestPolicy("all");
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.LOCAL_DESTROY);
+        break;
+      }
+      case "LOCAL": {
+        regionAttributes.setScope(RegionAttributesScope.LOCAL);
+        break;
+      }
+      case "LOCAL_PERSISTENT": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE);
+        regionAttributes.setScope(RegionAttributesScope.LOCAL);
+        break;
+      }
+      case "LOCAL_HEAP_LRU": {
+        regionAttributes.setScope(RegionAttributesScope.LOCAL);
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.LOCAL_DESTROY);
+        break;
+      }
+      case "LOCAL_OVERFLOW": {
+        regionAttributes.setScope(RegionAttributesScope.LOCAL);
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
+        break;
+      }
+      case "LOCAL_PERSISTENT_OVERFLOW": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE);
+        regionAttributes.setScope(RegionAttributesScope.LOCAL);
+        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
+        break;
+      }
+      case "PARTITION_PROXY": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+        regionAttributes.setLocalMaxMemory("0");
+        break;
+      }
+      case "PARTITION_PROXY_REDUNDANT": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+        regionAttributes.setLocalMaxMemory("0");
+        regionAttributes.setRedundantCopy("1");
+        break;
+      }
+      case "REPLICATE_PROXY": {
+        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.EMPTY);
+        regionAttributes.setScope(RegionAttributesScope.DISTRIBUTED_ACK);
+        break;
+      }
+      default:
+        throw new IllegalArgumentException("invalid refid " + refid);
+    }
   }
 
   @Override
diff --git a/geode-management/src/main/java/org/apache/geode/management/internal/api/ClusterManagementService.java b/geode-management/src/main/java/org/apache/geode/management/internal/api/ClusterManagementService.java
index e3c82e4..b9c392e 100644
--- a/geode-management/src/main/java/org/apache/geode/management/internal/api/ClusterManagementService.java
+++ b/geode-management/src/main/java/org/apache/geode/management/internal/api/ClusterManagementService.java
@@ -31,9 +31,9 @@ public interface ClusterManagementService {
    * @param config this holds the configuration attributes of the element you are trying to create
    *        on the cluster
    */
-  ClusterManagementResult create(CacheElement config);
+  ClusterManagementResult create(CacheElement config, String group);
 
-  ClusterManagementResult delete(CacheElement config);
+  ClusterManagementResult delete(CacheElement config, String group);
 
-  ClusterManagementResult update(CacheElement config);
+  ClusterManagementResult update(CacheElement config, String group);
 }
diff --git a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
index f2ec03e..f1ebae2 100644
--- a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
+++ b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
@@ -37,7 +37,7 @@ public class RegionManagementController extends AbstractManagementController {
   public ResponseEntity<ClusterManagementResult> createRegion(
       @RequestBody RegionConfig regionConfig) {
     ClusterManagementResult result =
-        clusterManagementService.create(regionConfig);
+        clusterManagementService.create(regionConfig, "cluster");
     return new ResponseEntity<>(result,
         result.isSuccessful() ? HttpStatus.CREATED : HttpStatus.INTERNAL_SERVER_ERROR);
   }