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/05/11 17:42:38 UTC

[geode] branch develop updated: GEODE-6756: rework type and regionAttributes setter/getter (#3578)

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 1f458c9  GEODE-6756: rework type and regionAttributes setter/getter (#3578)
1f458c9 is described below

commit 1f458c9516d476f0889708210e76e892985f41c7
Author: Jinmei Liao <ji...@pivotal.io>
AuthorDate: Sat May 11 10:42:15 2019 -0700

    GEODE-6756: rework type and regionAttributes setter/getter (#3578)
    
    Co-authored-by: Jens Deppe <jd...@pivotal.io>
    
    * make type a required attribute for RegionConfig
    * Do the type to attributes mapping in the RegionConfigValidator for the ultimate attributes validation.
---
 .../rest/ClientClusterManagementSSLTest.java       |   8 +
 .../ClientClusterManagementServiceDunitTest.java   |   4 +
 .../rest/ClusterManagementServiceOnServerTest.java |   2 +
 .../rest/ListRegionManagementDunitTest.java        |   4 +
 .../internal/rest/RegionManagementDunitTest.java   |   5 +-
 .../internal/api/RegionAPIDUnitTest.java           |   3 +-
 .../RegionConfigRealizerIntegrationTest.java       |  17 +-
 .../internal/cli/commands/CreateRegionCommand.java |   4 +-
 .../validators/RegionConfigValidator.java          | 250 +++++++++++++++++++--
 .../cache/configuration/RegionConfigTest.java      |  19 ++
 ...nternalConfigurationPersistenceServiceTest.java |   2 +-
 .../realizers/RegionConfigRealizerTest.java        |   7 +-
 .../validators/RegionConfigValidatorTest.java      | 131 ++++++++++-
 .../cache/configuration/RegionAttributesType.java  |   2 +-
 .../geode/cache/configuration/RegionConfig.java    | 139 ------------
 15 files changed, 408 insertions(+), 189 deletions(-)

diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClientClusterManagementSSLTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClientClusterManagementSSLTest.java
index 5676b12..688bd85 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClientClusterManagementSSLTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClientClusterManagementSSLTest.java
@@ -38,6 +38,7 @@ import org.springframework.web.client.ResourceAccessException;
 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.configuration.RegionType;
 import org.apache.geode.examples.SimpleSecurityManager;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
 import org.apache.geode.management.GeodeClusterManagementServiceConfig;
@@ -94,6 +95,7 @@ public class ClientClusterManagementSSLTest {
   public void createRegion_Successful() throws Exception {
     RegionConfig region = new RegionConfig();
     region.setName("customer");
+    region.setType(RegionType.PARTITION);
     int httpPort = locator.getHttpPort();
 
     client.invoke(() -> {
@@ -121,6 +123,7 @@ public class ClientClusterManagementSSLTest {
   public void createRegion_NoSsl() throws Exception {
     RegionConfig region = new RegionConfig();
     region.setName("customer");
+    region.setType(RegionType.PARTITION);
     int httpPort = locator.getHttpPort();
 
     client.invoke(() -> {
@@ -140,6 +143,7 @@ public class ClientClusterManagementSSLTest {
   public void createRegion_WrongPassword() throws Exception {
     RegionConfig region = new RegionConfig();
     region.setName("customer");
+    region.setType(RegionType.PARTITION);
     int httpPort = locator.getHttpPort();
 
     client.invoke(() -> {
@@ -167,6 +171,7 @@ public class ClientClusterManagementSSLTest {
   public void createRegion_NoUser() throws Exception {
     RegionConfig region = new RegionConfig();
     region.setName("customer");
+    region.setType(RegionType.PARTITION);
     int httpPort = locator.getHttpPort();
 
     client.invoke(() -> {
@@ -192,6 +197,7 @@ public class ClientClusterManagementSSLTest {
   public void createRegion_NoPassword() throws Exception {
     RegionConfig region = new RegionConfig();
     region.setName("customer");
+    region.setType(RegionType.PARTITION);
     int httpPort = locator.getHttpPort();
 
     client.invoke(() -> {
@@ -218,6 +224,7 @@ public class ClientClusterManagementSSLTest {
   public void createRegion_NoPrivilege() throws Exception {
     RegionConfig region = new RegionConfig();
     region.setName("customer");
+    region.setType(RegionType.PARTITION);
     int httpPort = locator.getHttpPort();
 
     client.invoke(() -> {
@@ -252,6 +259,7 @@ public class ClientClusterManagementSSLTest {
       ClusterManagementService cmsClient = new ClientClusterManagementService(config);
       RegionConfig region = new RegionConfig();
       region.setName("orders");
+      region.setType(RegionType.PARTITION);
       cmsClient.create(region);
 
       // verify that the region is created on the server
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClientClusterManagementServiceDunitTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClientClusterManagementServiceDunitTest.java
index 17cb991..bb2b3ba 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClientClusterManagementServiceDunitTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClientClusterManagementServiceDunitTest.java
@@ -25,6 +25,7 @@ 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.cache.configuration.RegionType;
 import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
 import org.apache.geode.management.GeodeClusterManagementServiceConfig;
 import org.apache.geode.management.api.ClusterManagementResult;
@@ -63,6 +64,7 @@ public class ClientClusterManagementServiceDunitTest {
   public void createRegion() {
     RegionConfig region = new RegionConfig();
     region.setName("customer");
+    region.setType(RegionType.PARTITION);
 
     ClusterManagementResult result = cmsClient.create(region);
 
@@ -79,6 +81,7 @@ public class ClientClusterManagementServiceDunitTest {
   public void createRegionWithNullGroup() {
     RegionConfig region = new RegionConfig();
     region.setName("orders");
+    region.setType(RegionType.PARTITION);
 
     ClusterManagementResult result = cmsClient.create(region);
 
@@ -103,6 +106,7 @@ public class ClientClusterManagementServiceDunitTest {
   public void createRegionWithGroup() {
     RegionConfig region = new RegionConfig();
     region.setName("company");
+    region.setType(RegionType.PARTITION);
     region.setGroup(groupA);
 
     ClusterManagementResult result = cmsClient.create(region);
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClusterManagementServiceOnServerTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClusterManagementServiceOnServerTest.java
index 8ea4326..2ff5508 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClusterManagementServiceOnServerTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ClusterManagementServiceOnServerTest.java
@@ -34,6 +34,7 @@ import org.junit.Test;
 import org.springframework.web.client.ResourceAccessException;
 
 import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.cache.configuration.RegionType;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
 import org.apache.geode.management.GeodeClusterManagementServiceConfig;
 import org.apache.geode.management.api.ClusterManagementResult;
@@ -66,6 +67,7 @@ public class ClusterManagementServiceOnServerTest implements Serializable {
 
     regionConfig = new RegionConfig();
     regionConfig.setName("test");
+    regionConfig.setType(RegionType.PARTITION);
   }
 
   @Test
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ListRegionManagementDunitTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ListRegionManagementDunitTest.java
index 6f3c364..be31490 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ListRegionManagementDunitTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ListRegionManagementDunitTest.java
@@ -70,6 +70,7 @@ public class ListRegionManagementDunitTest {
     RegionConfig regionConfig = new RegionConfig();
     regionConfig.setName("customers1");
     regionConfig.setGroup("group1");
+    regionConfig.setType(RegionType.PARTITION);
     client.create(regionConfig);
     locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/customers1", 1);
 
@@ -83,11 +84,13 @@ public class ListRegionManagementDunitTest {
     regionConfig = new RegionConfig();
     regionConfig.setName("customers2");
     regionConfig.setGroup("group2");
+    regionConfig.setType(RegionType.PARTITION);
     client.create(regionConfig);
     locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/customers2", 2);
 
     regionConfig = new RegionConfig();
     regionConfig.setName("customers");
+    regionConfig.setType(RegionType.PARTITION);
     client.create(regionConfig);
     locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/customers", 2);
 
@@ -95,6 +98,7 @@ public class ListRegionManagementDunitTest {
     regionConfig = new RegionConfig();
     regionConfig.setName("customers3");
     regionConfig.setGroup("group1");
+    regionConfig.setType(RegionType.PARTITION);
     client.create(regionConfig);
     regionConfig.setGroup("group2");
     client.create(regionConfig);
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 1a14388..054db03 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
@@ -91,6 +91,7 @@ public class RegionManagementDunitTest {
   public void createRegionWithKeyValueConstraint() throws Exception {
     RegionConfig config = new RegionConfig();
     config.setName("customers2");
+    config.setType(RegionType.PARTITION);
     RegionAttributesType type = new RegionAttributesType();
     type.setKeyConstraint("java.lang.Boolean");
     type.setValueConstraint("java.lang.Integer");
@@ -124,8 +125,8 @@ public class RegionManagementDunitTest {
   }
 
   @Test
-  public void createsAPartitionedRegionByDefault() throws Exception {
-    String json = "{\"name\": \"orders\"}";
+  public void createsAPartitionedRegion() throws Exception {
+    String json = "{\"name\": \"orders\", \"type\": \"PARTITION\"}";
 
     ClusterManagementResult result = restClient.doPostAndAssert("/regions", json)
         .hasStatusCode(201)
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 07f2941..8d66409 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
@@ -100,11 +100,12 @@ public class RegionAPIDUnitTest {
   }
 
   @Test
-  public void defaultTypeIsPartition() throws Exception {
+  public void createPartitionedRegion() throws Exception {
     String regionName = testName.getMethodName();
     locator.invoke(() -> {
       RegionConfig config = new RegionConfig();
       config.setName(regionName);
+      config.setType(RegionType.PARTITION);
       ClusterManagementResult result = ClusterStartupRule.getLocator().getClusterManagementService()
           .create(config);
       assertThat(result.isSuccessful()).isTrue();
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
index 04f32d4..b87d36f 100644
--- 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
@@ -15,7 +15,6 @@
 package org.apache.geode.management.internal.configuration.realizers;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import org.junit.Before;
 import org.junit.Rule;
@@ -26,6 +25,7 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.configuration.RegionType;
+import org.apache.geode.management.internal.configuration.validators.RegionConfigValidator;
 import org.apache.geode.test.junit.rules.ServerStarterRule;
 
 public class RegionConfigRealizerIntegrationTest {
@@ -46,7 +46,7 @@ public class RegionConfigRealizerIntegrationTest {
   public void sanityCheck() throws Exception {
     config.setName("test");
     config.setType(RegionType.REPLICATE);
-
+    RegionConfigValidator.setShortcutAttributes(config);
     realizer.create(config, server.getCache());
 
     Region<Object, Object> region = server.getCache().getRegion("test");
@@ -59,21 +59,10 @@ public class RegionConfigRealizerIntegrationTest {
   public void create2ndTime() throws Exception {
     config.setName("foo");
     config.setType(RegionType.REPLICATE);
+    RegionConfigValidator.setShortcutAttributes(config);
     realizer.create(config, server.getCache());
 
     // the 2nd time with same name and type will not throw an error
     realizer.create(config, server.getCache());
   }
-
-  @Test
-  public void createDifferentRegion2ndTime() throws Exception {
-    config.setName("bar");
-    config.setType(RegionType.REPLICATE);
-    realizer.create(config, server.getCache());
-
-    // the 2nd time with different type will throw an error
-    config.setType(RegionType.PARTITION);
-    assertThatThrownBy(() -> realizer.create(config, server.getCache()))
-        .isInstanceOf(IllegalStateException.class);
-  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommand.java
index 10b2d51..f19199a 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommand.java
@@ -64,6 +64,7 @@ import org.apache.geode.management.internal.cli.functions.RegionCreateFunction;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.model.ResultModel;
 import org.apache.geode.management.internal.cli.util.RegionPath;
+import org.apache.geode.management.internal.configuration.validators.RegionConfigValidator;
 import org.apache.geode.management.internal.exceptions.EntityExistsException;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission;
@@ -225,6 +226,7 @@ public class CreateRegionCommand extends SingleGfshCommand {
         getConfigurationPersistenceService();
     if (regionShortcut != null) {
       regionConfig.setType(regionShortcut.name());
+      RegionConfigValidator.setShortcutAttributes(regionConfig);
     }
     // get the attributes from the template region
     else {
@@ -263,7 +265,7 @@ public class CreateRegionCommand extends SingleGfshCommand {
       if (templateRegionConfigs.size() == 1) {
         regionConfig = templateRegionConfigs.get(0);
       }
-      // found more than one configuration with this name. fail ff they have different attributes.
+      // found more than one configuration with this name. fail if they have different attributes.
       else {
         RegionConfig first = templateRegionConfigs.get(0);
         for (int i = 1; i < templateRegionConfigs.size(); i++) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidator.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidator.java
index f794e4a..8d7f264 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidator.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidator.java
@@ -17,6 +17,10 @@ package org.apache.geode.management.internal.configuration.validators;
 
 import org.apache.geode.cache.configuration.CacheConfig;
 import org.apache.geode.cache.configuration.CacheElement;
+import org.apache.geode.cache.configuration.EnumActionDestroyOverflow;
+import org.apache.geode.cache.configuration.RegionAttributesDataPolicy;
+import org.apache.geode.cache.configuration.RegionAttributesScope;
+import org.apache.geode.cache.configuration.RegionAttributesType;
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.configuration.RegionType;
 import org.apache.geode.internal.cache.InternalCache;
@@ -38,33 +42,239 @@ public class RegionConfigValidator implements ConfigurationValidator<RegionConfi
       throw new IllegalArgumentException("Name of the region has to be specified.");
     }
 
+    if (config.getType() == null) {
+      throw new IllegalArgumentException("Type of the region has to be specified.");
+    }
+
+    // validate if the type is a valid RegionType. Only types defined in RegionType are supported
+    // by management v2 api.
+    try {
+      RegionType.valueOf(config.getType());
+
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException(
+          String.format("Type %s is not supported in Management V2 API.", config.getType()));
+    }
+
     RegionNameValidation.validate(config.getName());
 
-    if (config.getType() == null) {
-      RegionType defaultRegion = RegionType.PARTITION;
-      config.setType(defaultRegion);
-    } else {
-      String type = config.getType();
-      // validate if the type is a valid RegionType. Only types defined in RegionType are supported
-      // by management v2 api.
-      try {
-        RegionType.valueOf(type);
-
-      } catch (IllegalArgumentException e) {
-        throw new IllegalArgumentException(
-            String.format("Type %s is not supported in Management V2 API.", type));
-      }
-
-      // additional authorization
-      if (config.getRegionAttributes().getDataPolicy().isPersistent()) {
-        cache.getSecurityService()
-            .authorize(ResourcePermission.Resource.CLUSTER, ResourcePermission.Operation.WRITE,
-                ResourcePermission.Target.DISK);
+    setShortcutAttributes(config);
+
+    // additional authorization
+    if (config.getRegionAttributes().getDataPolicy().isPersistent()) {
+      cache.getSecurityService()
+          .authorize(ResourcePermission.Resource.CLUSTER, ResourcePermission.Operation.WRITE,
+              ResourcePermission.Target.DISK);
+    }
+  }
+
+  public static void setShortcutAttributes(RegionConfig config) {
+    String type = config.getType();
+    RegionAttributesType regionAttributes;
+
+    if (config.getRegionAttributes() == null) {
+      regionAttributes = new RegionAttributesType();
+      config.setRegionAttributes(regionAttributes);
+    }
+
+    regionAttributes = config.getRegionAttributes();
+    switch (type) {
+      case "PARTITION": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PARTITION, regionAttributes);
+        break;
+      }
+      case "REPLICATE": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.REPLICATE, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.DISTRIBUTED_ACK, regionAttributes);
+        break;
+      }
+      case "PARTITION_REDUNDANT": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PARTITION, regionAttributes);
+        checkAndSetRedundancyCopy("1", regionAttributes);
+        break;
+      }
+      case "PARTITION_PERSISTENT": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION, regionAttributes);
+        break;
+      }
+      case "PARTITION_REDUNDANT_PERSISTENT": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION, regionAttributes);
+        checkAndSetRedundancyCopy("1", regionAttributes);
+        break;
+      }
+      case "PARTITION_OVERFLOW": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PARTITION, regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.OVERFLOW_TO_DISK, regionAttributes);
+        break;
+      }
+      case "PARTITION_REDUNDANT_OVERFLOW": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PARTITION, regionAttributes);
+        checkAndSetRedundancyCopy("1", regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.OVERFLOW_TO_DISK, regionAttributes);
+        break;
+      }
+      case "PARTITION_PERSISTENT_OVERFLOW": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION, regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.OVERFLOW_TO_DISK, regionAttributes);
+        break;
+      }
+      case "PARTITION_REDUNDANT_PERSISTENT_OVERFLOW": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION, regionAttributes);
+        checkAndSetRedundancyCopy("1", regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.OVERFLOW_TO_DISK, regionAttributes);
+        break;
+      }
+      case "PARTITION_HEAP_LRU": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PARTITION, regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.LOCAL_DESTROY, regionAttributes);
+        break;
+
+      }
+      case "PARTITION_REDUNDANT_HEAP_LRU": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PARTITION, regionAttributes);
+        checkAndSetRedundancyCopy("1", regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.LOCAL_DESTROY, regionAttributes);
+        break;
+      }
+
+      case "REPLICATE_PERSISTENT": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.DISTRIBUTED_ACK, regionAttributes);
+        break;
+      }
+      case "REPLICATE_OVERFLOW": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.REPLICATE, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.DISTRIBUTED_ACK, regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.OVERFLOW_TO_DISK, regionAttributes);
+        break;
+
       }
+      case "REPLICATE_PERSISTENT_OVERFLOW": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.DISTRIBUTED_ACK, regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.OVERFLOW_TO_DISK, regionAttributes);
+        break;
+      }
+      case "REPLICATE_HEAP_LRU": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PRELOADED, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.DISTRIBUTED_ACK, regionAttributes);
+        regionAttributes.setInterestPolicy("all");
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.LOCAL_DESTROY, regionAttributes);
+        break;
+      }
+      case "LOCAL": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.NORMAL, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.LOCAL, regionAttributes);
+        break;
+      }
+      case "LOCAL_PERSISTENT": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.LOCAL, regionAttributes);
+        break;
+      }
+      case "LOCAL_HEAP_LRU": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.NORMAL, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.LOCAL, regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.LOCAL_DESTROY, regionAttributes);
+        break;
+      }
+      case "LOCAL_OVERFLOW": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.NORMAL, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.LOCAL, regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.OVERFLOW_TO_DISK, regionAttributes);
+        break;
+      }
+      case "LOCAL_PERSISTENT_OVERFLOW": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.LOCAL, regionAttributes);
+        checkAndSetEvictionAction(EnumActionDestroyOverflow.OVERFLOW_TO_DISK, regionAttributes);
+        break;
+      }
+      case "PARTITION_PROXY": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PARTITION, regionAttributes);
+        checkAndSetLocalMaxMemory("0", regionAttributes);
+        break;
+      }
+      case "PARTITION_PROXY_REDUNDANT": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.PARTITION, regionAttributes);
+        checkAndSetLocalMaxMemory("0", regionAttributes);
+        checkAndSetRedundancyCopy("1", regionAttributes);
+        break;
+      }
+      case "REPLICATE_PROXY": {
+        checkAndSetDataPolicy(RegionAttributesDataPolicy.EMPTY, regionAttributes);
+        checkAndSetScope(RegionAttributesScope.DISTRIBUTED_ACK, regionAttributes);
+        break;
+      }
+      default:
+        throw new IllegalArgumentException("invalid type " + type);
     }
+  }
 
+  private static void checkAndSetLocalMaxMemory(String maxMemory,
+      RegionAttributesType regionAttributes) {
+    if (regionAttributes.getPartitionAttributes() == null
+        || regionAttributes.getPartitionAttributes().getLocalMaxMemory() == null) {
+      regionAttributes.setLocalMaxMemory(maxMemory);
+    }
+    String existing = regionAttributes.getPartitionAttributes().getLocalMaxMemory();
+    if (!existing.equals(maxMemory)) {
+      throw new IllegalArgumentException("Invalid local max memory: " + existing);
+    }
   }
 
+  private static void checkAndSetEvictionAction(EnumActionDestroyOverflow evictionAction,
+      RegionAttributesType regionAttributes) {
+    if (regionAttributes.getEvictionAttributes() == null
+        || regionAttributes.getEvictionAttributes().getLruHeapPercentage() == null
+        || regionAttributes.getEvictionAttributes().getLruHeapPercentage().getAction() == null) {
+      regionAttributes.setLruHeapPercentageEvictionAction(evictionAction);
+    }
+
+    EnumActionDestroyOverflow existing =
+        regionAttributes.getEvictionAttributes().getLruHeapPercentage().getAction();
+    if (existing != evictionAction) {
+      throw new IllegalArgumentException("Conflicting eviction action " + existing.toString());
+    }
+  }
+
+  private static void checkAndSetScope(RegionAttributesScope scope,
+      RegionAttributesType regionAttributes) {
+    RegionAttributesScope existing = regionAttributes.getScope();
+    if (existing == null) {
+      regionAttributes.setScope(scope);
+    } else if (existing != scope) {
+      throw new IllegalArgumentException("Conflicting scope " + existing.toString());
+    }
+  }
+
+  private static void checkAndSetDataPolicy(RegionAttributesDataPolicy policy,
+      RegionAttributesType regionAttributes) {
+    RegionAttributesDataPolicy existing = regionAttributes.getDataPolicy();
+    if (existing == null) {
+      regionAttributes.setDataPolicy(policy);
+    } else if (existing != policy) {
+      throw new IllegalArgumentException("Conflicting data policy "
+          + existing.toString());
+    }
+  }
+
+  // need to do this if user already set the redundant copy in the RegionAttributeType
+  private static void checkAndSetRedundancyCopy(String copies,
+      RegionAttributesType regionAttributes) {
+    if (regionAttributes.getPartitionAttributes() == null
+        || regionAttributes.getPartitionAttributes().getRedundantCopies() == null) {
+      regionAttributes.setRedundantCopy(copies);
+    }
+    RegionAttributesType.PartitionAttributes partitionAttributes =
+        regionAttributes.getPartitionAttributes();
+    if ("0".equals(partitionAttributes.getRedundantCopies())) {
+      throw new IllegalArgumentException(
+          "Conflicting redundant copy when region type is REDUNDANT");
+    }
+  }
+
+
   @Override
   public boolean exists(RegionConfig config, CacheConfig existing) {
     return CacheElement.exists(existing.getRegions(), config.getId());
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 ae71b84..8bdf380 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
@@ -16,6 +16,7 @@ package org.apache.geode.cache.configuration;
 
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
 
 import java.io.File;
 import java.net.URL;
@@ -26,7 +27,9 @@ import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.config.JAXBService;
+import org.apache.geode.management.internal.configuration.validators.RegionConfigValidator;
 import org.apache.geode.util.internal.GeodeJsonMapper;
 
 public class RegionConfigTest {
@@ -68,6 +71,7 @@ public class RegionConfigTest {
       RegionConfig config = new RegionConfig();
       config.setType(shortcut.name());
       config.setName(shortcut.name());
+      RegionConfigValidator.setShortcutAttributes(config);
       RegionConfig masterRegion = CacheElement.findElement(master.getRegions(), shortcut.name());
       assertThat(config).isEqualToComparingFieldByFieldRecursively(masterRegion);
     }
@@ -238,4 +242,19 @@ public class RegionConfigTest {
     assertThat(newDiskStore.getDiskDirs()).hasSize(2);
 
   }
+
+  @Test
+  public void setAttributesAndType() throws Exception {
+    RegionConfig config = new RegionConfig();
+    config.setType("REPLICATE");
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setKeyConstraint("java.lang.Boolean");
+    attributes.setValueConstraint("java.lang.Integer");
+    attributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+    config.setRegionAttributes(attributes);
+
+    RegionConfigValidator validator = new RegionConfigValidator(mock(InternalCache.class));
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+  }
 }
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 dc1cfb5..5e6d611 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
@@ -90,7 +90,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/configuration/realizers/RegionConfigRealizerTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/realizers/RegionConfigRealizerTest.java
index 3451d89..0e71b10 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
@@ -30,15 +30,18 @@ import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.internal.configuration.validators.RegionConfigValidator;
 
 public class RegionConfigRealizerTest {
   InternalCache cache;
   RegionFactory regionFactory;
   RegionConfigRealizer realizer;
+  RegionConfigValidator validator;
 
   @Before
   public void setup() {
     cache = mock(InternalCache.class);
+    validator = new RegionConfigValidator(cache);
     regionFactory = mock(RegionFactory.class);
     when(cache.createRegionFactory()).thenReturn(regionFactory);
     realizer = new RegionConfigRealizer();
@@ -49,7 +52,7 @@ public class RegionConfigRealizerTest {
     RegionConfig config = new RegionConfig();
     config.setName("regionName");
     config.setType(RegionShortcut.PARTITION.name());
-
+    validator.validate(config);
     realizer.create(config, cache);
 
     ArgumentCaptor<DataPolicy> dataPolicyArgumentCaptor = ArgumentCaptor.forClass(DataPolicy.class);
@@ -64,7 +67,7 @@ public class RegionConfigRealizerTest {
     RegionConfig config = new RegionConfig();
     config.setName("regionName");
     config.setType(RegionShortcut.REPLICATE.name());
-
+    validator.validate(config);
     realizer.create(config, cache);
 
     ArgumentCaptor<DataPolicy> dataPolicyArgumentCaptor = ArgumentCaptor.forClass(DataPolicy.class);
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidatorTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidatorTest.java
index 38dbf58..f98dbeb 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidatorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidatorTest.java
@@ -28,6 +28,10 @@ import static org.mockito.Mockito.when;
 import org.junit.Before;
 import org.junit.Test;
 
+import org.apache.geode.cache.configuration.EnumActionDestroyOverflow;
+import org.apache.geode.cache.configuration.RegionAttributesDataPolicy;
+import org.apache.geode.cache.configuration.RegionAttributesScope;
+import org.apache.geode.cache.configuration.RegionAttributesType;
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.configuration.RegionType;
 import org.apache.geode.internal.cache.InternalCache;
@@ -82,14 +86,6 @@ public class RegionConfigValidatorTest {
   }
 
   @Test
-  public void defaultsTypeToPartitioned() {
-    config.setName("regionName");
-    validator.validate(config);
-
-    assertThat(config.getType()).isEqualTo("PARTITION");
-  }
-
-  @Test
   public void noName() {
     assertThatThrownBy(() -> validator.validate(config)).isInstanceOf(
         IllegalArgumentException.class)
@@ -99,6 +95,7 @@ public class RegionConfigValidatorTest {
   @Test
   public void invalidName1() {
     config.setName("__test");
+    config.setType("REPLICATE");
     assertThatThrownBy(() -> validator.validate(config)).isInstanceOf(
         IllegalArgumentException.class)
         .hasMessageContaining("Region names may not begin with a double-underscore");
@@ -107,10 +104,128 @@ public class RegionConfigValidatorTest {
   @Test
   public void invalidName2() {
     config.setName("a!&b");
+    config.setType("REPLICATE");
     assertThatThrownBy(() -> validator.validate(config)).isInstanceOf(
         IllegalArgumentException.class)
         .hasMessageContaining(
             "Region names may only be alphanumeric and may contain hyphens or underscores");
   }
 
+  @Test
+  public void missingType() {
+    config.setName("test");
+    assertThatThrownBy(() -> validator.validate(config)).isInstanceOf(
+        IllegalArgumentException.class)
+        .hasMessageContaining(
+            "Type of the region has to be specified");
+  }
+
+  @Test
+  public void validatePartition() throws Exception {
+    config.setName("test");
+    config.setType("PARTITION");
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+    config.setRegionAttributes(attributes);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+  }
+
+  @Test
+  public void validateReplicate() throws Exception {
+    config.setName("test");
+    config.setType("REPLICATE");
+    RegionAttributesType attributes = new RegionAttributesType();
+    config.setRegionAttributes(attributes);
+
+    attributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+
+    attributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+    attributes.setScope(RegionAttributesScope.DISTRIBUTED_NO_ACK);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+  }
+
+  @Test
+  public void validatePartition_Redundant() throws Exception {
+    config.setName("test");
+    config.setType("PARTITION_REDUNDANT");
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+    config.setRegionAttributes(attributes);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+  }
+
+  @Test
+  public void validatePartition_Persistent() throws Exception {
+    config.setName("test");
+    config.setType("PARTITION_PERSISTENT");
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+    config.setRegionAttributes(attributes);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+  }
+
+  @Test
+  public void validatePartition_Redundant_Persistent() throws Exception {
+    config.setName("test");
+    config.setType("PARTITION_REDUNDANT_PERSISTENT");
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+    config.setRegionAttributes(attributes);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+    attributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_PARTITION);
+    attributes.setRedundantCopy("0");
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+
+    // valid redundancy copy
+    attributes.setRedundantCopy("2");
+    validator.validate(config);
+    assertThat(config.getRegionAttributes().getPartitionAttributes().getRedundantCopies())
+        .isEqualTo("2");
+  }
+
+  @Test
+  public void validatePartition_overflow() throws Exception {
+    config.setName("test");
+    config.setType("PARTITION_OVERFLOW");
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+    config.setRegionAttributes(attributes);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+
+    attributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+    attributes.setLruHeapPercentageEvictionAction(EnumActionDestroyOverflow.LOCAL_DESTROY);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+  }
+
+  @Test
+  public void validatePartition_proxy() throws Exception {
+    config.setName("test");
+    config.setType("PARTITION_PROXY");
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
+    config.setRegionAttributes(attributes);
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+
+    attributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
+    attributes.setLocalMaxMemory("5000");
+    assertThatThrownBy(() -> validator.validate(config))
+        .isInstanceOf(IllegalArgumentException.class);
+
+    // validator will use the type to set the local max memory to be 0
+    attributes.setLocalMaxMemory(null);
+    validator.validate(config);
+    assertThat(attributes.getPartitionAttributes().getLocalMaxMemory()).isEqualTo("0");
+  }
+
 }
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 63d72e7..4ea5687 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
@@ -1494,7 +1494,7 @@ public class RegionAttributesType implements Serializable {
     this.offHeap = value;
   }
 
-  public void setLruHeapPercentage(EnumActionDestroyOverflow action) {
+  public void setLruHeapPercentageEvictionAction(EnumActionDestroyOverflow action) {
     if (evictionAttributes == null) {
       evictionAttributes = new EvictionAttributes();
     }
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 9a0bb88..fe18b1a 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
@@ -333,145 +333,6 @@ public class RegionConfig extends CacheElement implements RestfulEndpoint {
   public void setType(String regionType) {
     if (regionType != null) {
       this.type = regionType.toUpperCase();
-      setShortcutAttributes();
-    }
-  }
-
-  private void setShortcutAttributes() {
-    if (regionAttributes == null) {
-      regionAttributes = new RegionAttributesType();
-    }
-
-    switch (type) {
-      case "PARTITION": {
-        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PARTITION);
-        break;
-      }
-      case "REPLICATE": {
-        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.REPLICATE);
-        regionAttributes.setScope(RegionAttributesScope.DISTRIBUTED_ACK);
-        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_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.setDataPolicy(RegionAttributesDataPolicy.NORMAL);
-        regionAttributes.setScope(RegionAttributesScope.LOCAL);
-        break;
-      }
-      case "LOCAL_PERSISTENT": {
-        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.PERSISTENT_REPLICATE);
-        regionAttributes.setScope(RegionAttributesScope.LOCAL);
-        break;
-      }
-      case "LOCAL_HEAP_LRU": {
-        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.NORMAL);
-        regionAttributes.setScope(RegionAttributesScope.LOCAL);
-        regionAttributes.setLruHeapPercentage(EnumActionDestroyOverflow.LOCAL_DESTROY);
-        break;
-      }
-      case "LOCAL_OVERFLOW": {
-        regionAttributes.setDataPolicy(RegionAttributesDataPolicy.NORMAL);
-        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 type " + type);
     }
   }