You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by mm...@apache.org on 2018/11/29 23:30:13 UTC

[geode] branch geode-5971-createregion created (now af3e912)

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

mmartell pushed a change to branch geode-5971-createregion
in repository https://gitbox.apache.org/repos/asf/geode.git.


      at af3e912  WIP - add partition attributes and disk store tests

This branch includes the following new commits:

     new 1a1cb17  WIP
     new 8e50565  wip - checks that eviction attributes are not persisted when empty
     new af3e912  WIP - add partition attributes and disk store tests

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[geode] 03/03: WIP - add partition attributes and disk store tests

Posted by mm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mmartell pushed a commit to branch geode-5971-createregion
in repository https://gitbox.apache.org/repos/asf/geode.git

commit af3e912881afb7597cad9aabd40969428f626a9a
Author: Peter Tran <pt...@pivotal.io>
AuthorDate: Thu Nov 29 15:29:52 2018 -0800

    WIP - add partition attributes and disk store tests
    
    Signed-off-by: Aditya Anchuri <aa...@pivotal.io>
---
 .../cli/commands/CreateRegionCommandDUnitTest.java |  95 ++++++++++
 ...egionCommandPersistsConfigurationDUnitTest.java | 203 +++++++++++++++++++--
 .../apache/geode/cache/PartitionAttributes.java    |  19 ++
 3 files changed, 306 insertions(+), 11 deletions(-)

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 3dbecfc..7a6568f 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
@@ -20,10 +20,12 @@ import static org.assertj.core.api.Assertions.assertThat;
 import java.io.File;
 import java.io.Serializable;
 import java.util.Arrays;
+import java.util.List;
 import java.util.stream.Collectors;
 
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -31,13 +33,18 @@ import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
 import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Declarable;
+import org.apache.geode.cache.EntryOperation;
 import org.apache.geode.cache.PartitionResolver;
 import org.apache.geode.cache.Region;
+import org.apache.geode.cache.asyncqueue.AsyncEvent;
+import org.apache.geode.cache.asyncqueue.AsyncEventListener;
 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.util.CacheListenerAdapter;
 import org.apache.geode.compression.SnappyCompressor;
+import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.test.compiler.JarBuilder;
@@ -59,6 +66,26 @@ public class CreateRegionCommandDUnitTest {
       implements Serializable {
   }
 
+  public static class DummyAEQListener implements AsyncEventListener, Declarable {
+    @Override
+    public boolean processEvents(List<AsyncEvent> events) {
+      return false;
+    }
+  }
+
+  public static class DummyPartitionResolver implements PartitionResolver, Declarable {
+    @Override
+    public Object getRoutingObject(EntryOperation opDetails) {
+      return null;
+    }
+
+    @Override
+    public String getName() {
+      return "dummy";
+    }
+  }
+
+
   @ClassRule
   public static ClusterStartupRule lsRule = new ClusterStartupRule();
 
@@ -230,6 +257,7 @@ public class CreateRegionCommandDUnitTest {
     gfsh.executeAndAssertThat("destroy region --name=/TEMPLATE").statusIsSuccess();
   }
 
+
   @Test
   public void cannotSetRegionExpirationForPartitionedTemplate() {
     gfsh.executeAndAssertThat("create region --name=/TEMPLATE --type=PARTITION")
@@ -561,6 +589,73 @@ public class CreateRegionCommandDUnitTest {
         .containsOutput("Region /startWithLocalRegion already exists on the cluster");
   }
 
+  /**
+   * Ignored this test until we refactor the FetchRegionAttributesFunction to not use
+   * AttributesFactory, and instead use RegionConfig, which we will do as part of implementing
+   * GEODE-6103
+   */
+  @Ignore
+  @Test
+  public void testCreateRegionFromTemplateWithAsyncEventListeners() {
+    String queueId = "queue1";
+    gfsh.executeAndAssertThat(
+        "create async-event-queue --id=" + queueId
+            + " --listener=" + CreateRegionCommandDUnitTest.DummyAEQListener.class.getName())
+        .statusIsSuccess();
+
+    String regionName = testName.getMethodName();
+    gfsh.executeAndAssertThat(
+        "create region --name=" + regionName
+            + " --type=REPLICATE"
+            + " --async-event-queue-id=" + queueId)
+        .statusIsSuccess();
+
+    gfsh.executeAndAssertThat(
+        "create region --name=" + regionName + "-from-template"
+            + " --template-region=" + regionName)
+        .statusIsSuccess();
+
+    server1.invoke(() -> {
+      Region regionFromTemplate = ClusterStartupRule.getCache()
+          .getRegion(regionName + "-from-template");
+      assertThat(regionFromTemplate).isNotNull();
+      assertThat(((InternalRegion) regionFromTemplate).getAsyncEventQueueIds())
+          .contains(queueId);
+    });
+  }
+
+  /**
+   * Ignored this test until we refactor the FetchRegionAttributesFunction to not use
+   * AttributesFactory, and instead use RegionConfig, which we will do as part of implementing
+   * GEODE-6103
+   */
+  @Ignore
+  @Test
+  public void testCreateRegionFromTemplateWithPartitionResolver() {
+    String regionName = testName.getMethodName();
+    String regionFromTemplateName = regionName + "-from-template";
+
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionName
+        + " --type=PARTITION"
+        + " --partition-resolver=" + DummyPartitionResolver.class.getName()).statusIsSuccess();
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionFromTemplateName
+        + " --template-region=" + regionName).statusIsSuccess();
+
+    server1.invoke(() -> {
+      Region regionFromTemplate = ClusterStartupRule.getCache()
+          .getRegion(regionName + "-from-template");
+      assertThat(regionFromTemplate).isNotNull();
+      assertThat(((InternalRegion) regionFromTemplate).getPartitionAttributes()
+          .getPartitionResolver())
+          .isNotNull();
+      assertThat(((InternalRegion) regionFromTemplate).getPartitionAttributes()
+          .getPartitionResolver().getName())
+          .isEqualTo(DummyPartitionResolver.class.getName());
+    });
+  }
+
   private String getUniversalClassCode(String classname) {
     String code = "package io.pivotal;" + "import org.apache.geode.cache.CacheLoader;"
         + "import org.apache.geode.cache.CacheLoaderException;"
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
index 0a0dbcc..4c414f9 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
@@ -412,35 +412,215 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
   }
 
   @Test
-  public void placeholderAEQ() {}
+  public void createRegionPersistsAEQConfig() {
+    String queueId = "queue1";
+    gfsh.executeAndAssertThat(
+        "create async-event-queue --id=" + queueId
+            + " --listener=" + CreateRegionCommandDUnitTest.DummyAEQListener.class.getName())
+        .statusIsSuccess();
+
+    String regionName = testName.getMethodName();
+    gfsh.executeAndAssertThat(
+        "create region --name=" + regionName
+            + " --type=REPLICATE"
+            + " --async-event-queue-id=" + queueId)
+        .statusIsSuccess();
+
+    locator.invoke(() -> {
+      InternalConfigurationPersistenceService cc =
+          ClusterStartupRule.getLocator().getConfigurationPersistenceService();
+      CacheConfig config = cc.getCacheConfig("cluster");
+
+      List<RegionConfig> regions = config.getRegions();
+      assertThat(regions).isNotEmpty();
+      assertThat(regions).hasSize(1);
+      RegionConfig regionConfig = CacheElement.findElement(regions, regionName);
+      assertThat(regionConfig.getRegionAttributes().get(0).getAsyncEventQueueIds())
+          .contains(queueId);
+    });
+  }
 
   @Test
-  public void placeholderColocation() {}
+  public void createRegionWithColocation() {
+    String regionName = testName.getMethodName();
+    String colocatedRegionName = regionName + "-colocated";
+    String colocatedRegionFromTemplateName = colocatedRegionName + "-from-template";
+
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionName
+        + " --type=PARTITION");
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + colocatedRegionName
+        + " --colocated-with=" + regionName
+        + " --type=PARTITION");
+
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + colocatedRegionFromTemplateName
+        + " --template-region=" + colocatedRegionName);
+
+    locator.invoke(() -> {
+      InternalConfigurationPersistenceService cc =
+          ClusterStartupRule.getLocator().getConfigurationPersistenceService();
+      CacheConfig config = cc.getCacheConfig("cluster");
+
+      List<RegionConfig> regions = config.getRegions();
+      assertThat(regions).isNotEmpty();
+      assertThat(regions).hasSize(3);
+
+      RegionConfig colocatedConfig = CacheElement.findElement(regions, colocatedRegionName);
+      assertThat(
+          colocatedConfig.getRegionAttributes().get(0).getPartitionAttributes().getColocatedWith())
+              .isEqualTo("/" + regionName);
+
+      RegionConfig colocatedConfigFromTemplate = CacheElement.findElement(regions,
+          colocatedRegionFromTemplateName);
+      assertThat(
+          colocatedConfigFromTemplate.getRegionAttributes().get(0).getPartitionAttributes()
+              .getColocatedWith())
+                  .isEqualTo("/" + regionName);
+    });
+  }
 
   @Test
-  public void placeholderDiskstores() {
-    // test disk-synchronous
+  public void createRegionPersistsDiskstores() throws Exception {
+    String regionName = testName.getMethodName();
+    String store = "Store1";
+    gfsh.executeAndAssertThat("create disk-store"
+        + " --name=" + store
+        + " --dir=/tmp/foo").statusIsSuccess();
+
+    // Give disk store time to get created
+    Thread.sleep(2000);
+
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionName
+        + " --type=REPLICATE_PERSISTENT"
+        + " --disk-store=" + store
+    ).statusIsSuccess();
+
+    String regionNameFromTemplate = regionName + "-from-template";
+    gfsh.executeAndAssertThat("create region --name=" + regionNameFromTemplate
+        + " --template-region=" + regionName)
+        .statusIsSuccess();
+
+    locator.invoke(() -> {
+      InternalConfigurationPersistenceService cc =
+          ClusterStartupRule.getLocator().getConfigurationPersistenceService();
+      CacheConfig config = cc.getCacheConfig("cluster");
+
+      List<RegionConfig> regions = config.getRegions();
+      assertThat(regions).isNotEmpty();
+      assertThat(regions).hasSize(2);
+
+      List<String> regionNames = Arrays.asList(regionName, regionNameFromTemplate);
+      regionNames.forEach(name -> {
+        RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), name);
+        assertThat(regionConfig).isNotNull();
+        assertThat(regionConfig.getName()).isEqualTo(name);
+
+        RegionAttributesType regionAttributes = regionConfig.getRegionAttributes().get(0);
+        assertThat(regionAttributes.getDiskStoreName())
+            .isEqualTo(store);
+      });
+    });
   }
 
   @Test
-  public void placeholderPartitionedRegion() {
-    // test disk-synchronous String regionName = testName.getMethodName();
+  public void createRegionPersistsPartitionAttributes() {
     String regionName = testName.getMethodName();
+    String regionFromTemplateName = regionName + "-from-template";
+
     gfsh.executeAndAssertThat("create region"
         + " --name=" + regionName
         + " --type=PARTITION"
-        + " --partition-resolver=" + DummyPartitionResolver.class.getName()
         + " --recovery-delay=1"
+        + " --local-max-memory=1000"
         + " --redundant-copies=1"
         + " --startup-recovery-delay=1"
         + " --total-max-memory=100"
-        + " --total-num-buckets=1"
-        + " --eviction-max-memory=700"
-        + " --eviction-entry-count=7"
-        + " --eviction-object-sizer=" + DummyObjectSizer.class.getName()).statusIsSuccess();
+        + " --total-num-buckets=1").statusIsSuccess();
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionFromTemplateName
+        + " --template-region=" + regionName);
+
+    locator.invoke(() -> {
+      InternalConfigurationPersistenceService cc =
+          ClusterStartupRule.getLocator().getConfigurationPersistenceService();
+      CacheConfig config = cc.getCacheConfig("cluster");
+
+      List<RegionConfig> regions = config.getRegions();
+      assertThat(regions).isNotEmpty();
+      assertThat(regions).hasSize(2);
+
+      List<String> regionNames = Arrays.asList(regionName, regionFromTemplateName);
+      regionNames.forEach(name -> {
+        RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), name);
+        assertThat(regionConfig).isNotNull();
+        assertThat(regionConfig.getName()).isEqualTo(name);
+
+        RegionAttributesType regionAttributes = regionConfig.getRegionAttributes().get(0);
+        RegionAttributesType.PartitionAttributes partitionAttributes =
+            regionAttributes.getPartitionAttributes();
+
+        assertThat(partitionAttributes.getRecoveryDelay())
+            .describedAs("Recovery delay should be 1 for region " + name)
+            .isEqualTo("1");
+        assertThat(partitionAttributes.getLocalMaxMemory())
+            .describedAs("Local max memory should be 1000 for region " + name)
+            .isEqualTo("1000");
+        assertThat(partitionAttributes.getRedundantCopies())
+            .describedAs("Redundant copies should be 1 for region " + name)
+            .isEqualTo("1");
+        assertThat(partitionAttributes.getStartupRecoveryDelay())
+            .describedAs("Startup recovery delay should be 1 for region " + name)
+            .isEqualTo("1");
+        assertThat(partitionAttributes.getTotalMaxMemory())
+            .describedAs("Total max memory should be 100 for region " + name)
+            .isEqualTo("100");
+        assertThat(partitionAttributes.getTotalNumBuckets())
+            .describedAs("Total num buckets should be 1 for region " + name)
+            .isEqualTo("1");
+      });
+    });
   }
 
   @Test
+  public void createRegionPersistsPartitionResolver() {
+    String regionName = testName.getMethodName();
+
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionName
+        + " --type=PARTITION"
+        + " --partition-resolver=" + DummyPartitionResolver.class.getName()).statusIsSuccess();
+
+    locator.invoke(() -> {
+      InternalConfigurationPersistenceService cc =
+          ClusterStartupRule.getLocator().getConfigurationPersistenceService();
+      CacheConfig config = cc.getCacheConfig("cluster");
+
+      List<RegionConfig> regions = config.getRegions();
+      assertThat(regions).isNotEmpty();
+      assertThat(regions).hasSize(1);
+
+      List<String> regionNames = Arrays.asList(regionName);
+      regionNames.forEach(name -> {
+        RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), name);
+        assertThat(regionConfig).isNotNull();
+        assertThat(regionConfig.getName()).isEqualTo(name);
+
+        RegionAttributesType regionAttributes = regionConfig.getRegionAttributes().get(0);
+        RegionAttributesType.PartitionAttributes partitionAttributes =
+            regionAttributes.getPartitionAttributes();
+
+        assertThat(partitionAttributes.getPartitionResolver().getClassName())
+            .isEqualTo(DummyPartitionResolver.class.getName());
+      });
+    });
+  }
+
+  // TODO test empty partition attributes
+
+  @Test
   public void placeholderCustomExpiryClass() {
     // + " --entry-idle-time-custom-expiry=" + DummyCustomExpiry.class.getName()
     // assertThat(attr.getEntryIdleTime().getExpirationAttributes().getCustomExpiry().toString())
@@ -449,6 +629,7 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
 
   }
 
+  // TODO + " --eviction-entry-count=7"
   @Test
   public void placeHolderDisableCloning() {
     // " --enable-cloning=false"
diff --git a/geode-core/src/main/java/org/apache/geode/cache/PartitionAttributes.java b/geode-core/src/main/java/org/apache/geode/cache/PartitionAttributes.java
index d137fa3..2cab775 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/PartitionAttributes.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/PartitionAttributes.java
@@ -18,6 +18,8 @@ package org.apache.geode.cache;
 import java.util.List;
 import java.util.Properties;
 
+import org.apache.geode.cache.configuration.DeclarableType;
+import org.apache.geode.cache.configuration.RegionAttributesType;
 import org.apache.geode.cache.partition.PartitionListener;
 
 /**
@@ -153,4 +155,21 @@ public interface PartitionAttributes<K, V> {
    */
   List<FixedPartitionAttributes> getFixedPartitionAttributes();
 
+  default RegionAttributesType.PartitionAttributes convertToConfigPartitionAttributes() {
+    RegionAttributesType.PartitionAttributes configAttributes =
+        new RegionAttributesType.PartitionAttributes();
+    configAttributes.setColocatedWith(getColocatedWith());
+    configAttributes.setLocalMaxMemory(Integer.toString(getLocalMaxMemory()));
+    if (getPartitionResolver() != null) {
+      configAttributes.setPartitionResolver(new DeclarableType(getPartitionResolver().getName()));
+    }
+    configAttributes.setRecoveryDelay(Long.toString(getRecoveryDelay()));
+    configAttributes.setStartupRecoveryDelay(Long.toString(getStartupRecoveryDelay()));
+    configAttributes.setRedundantCopies(Integer.toString(getRedundantCopies()));
+    configAttributes.setTotalMaxMemory(Long.toString(getTotalMaxMemory()));
+    configAttributes.setTotalNumBuckets(Long.toString(getTotalNumBuckets()));
+
+    return configAttributes;
+  }
+
 }


[geode] 02/03: wip - checks that eviction attributes are not persisted when empty

Posted by mm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mmartell pushed a commit to branch geode-5971-createregion
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 8e50565b74586944e9fcee1e4fdeec8c8ef7e033
Author: Aditya Anchuri <aa...@pivotal.io>
AuthorDate: Wed Nov 28 14:46:11 2018 -0800

    wip - checks that eviction attributes are not persisted when empty
    
    Signed-off-by: Peter Tran <pt...@pivotal.io>
---
 ...egionCommandPersistsConfigurationDUnitTest.java | 33 ++++++++++++++++++++++
 .../org/apache/geode/cache/EvictionAttributes.java |  4 +++
 .../cache/configuration/RegionConfigFactory.java   |  4 ++-
 3 files changed, 40 insertions(+), 1 deletion(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
index f4ea2ce..0a0dbcc 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
@@ -379,6 +379,39 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
   }
 
   @Test
+  public void createRegionDoesNotPersistEmptyEvictionAttributes() {
+    String regionName = testName.getMethodName();
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionName
+        + " --type=REPLICATE").statusIsSuccess();
+
+    locator.invoke(() -> {
+      InternalConfigurationPersistenceService cc =
+          ClusterStartupRule.getLocator().getConfigurationPersistenceService();
+      CacheConfig config = cc.getCacheConfig("cluster");
+
+      List<RegionConfig> regions = config.getRegions();
+      assertThat(regions).isNotEmpty();
+      assertThat(regions).hasSize(1);
+
+      List<String> regionNames = Arrays.asList(regionName);
+      regionNames.forEach(name -> {
+        RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), name);
+        assertThat(regionConfig).isNotNull();
+        assertThat(regionConfig.getName()).isEqualTo(name);
+        assertThat(regionConfig.getRegionAttributes())
+            .describedAs("Expecting region attributes to exist")
+            .hasSize(1);
+
+        RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+        assertThat(attr.getEvictionAttributes())
+            .describedAs("Eviction attributes should be null for " + name)
+            .isNull();
+      });
+    });
+  }
+
+  @Test
   public void placeholderAEQ() {}
 
   @Test
diff --git a/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java b/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
index bc23920..b333b2e 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
@@ -540,4 +540,8 @@ public abstract class EvictionAttributes implements DataSerializable {
     return configAttributes;
   }
 
+  public boolean isEmpty() {
+    return getAction() == EvictionAction.NONE && getAlgorithm() == EvictionAlgorithm.NONE;
+  }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/configuration/RegionConfigFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/configuration/RegionConfigFactory.java
index 8aa98f7..c78f68b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/configuration/RegionConfigFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/configuration/RegionConfigFactory.java
@@ -226,7 +226,9 @@ public class RegionConfigFactory {
     if (args.getEvictionAttributes() != null) {
       addAttribute(regionConfig, a -> a.setEvictionAttributes(
           args.getEvictionAttributes().convertToConfigEvictionAttributes()));
-    } else if (regionAttributes != null && regionAttributes.getEvictionAttributes() != null) {
+    } else if (regionAttributes != null &&
+        regionAttributes.getEvictionAttributes() != null &&
+        !regionAttributes.getEvictionAttributes().isEmpty()) {
       addAttribute(regionConfig, a -> a.setEvictionAttributes(
           regionAttributes.getEvictionAttributes().convertToConfigEvictionAttributes()));
     }


[geode] 01/03: WIP

Posted by mm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mmartell pushed a commit to branch geode-5971-createregion
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 1a1cb17ad92d82a48989637aa48035d4c29878f5
Author: Aditya Anchuri <aa...@pivotal.io>
AuthorDate: Wed Nov 28 11:49:40 2018 -0800

    WIP
    
    Signed-off-by: Peter Tran <pt...@pivotal.io>
---
 ...egionCommandPersistsConfigurationDUnitTest.java | 169 ++++++++++++++++-----
 .../org/apache/geode/cache/EvictionAttributes.java |  35 +++++
 .../cache/configuration/RegionConfigFactory.java   |  93 +++++++++---
 .../internal/cli/commands/CreateRegionCommand.java |   1 +
 .../configuration/RegionConfigFactoryTest.java     |   1 -
 5 files changed, 241 insertions(+), 58 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
index 444640d..f4ea2ce 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateRegionCommandPersistsConfigurationDUnitTest.java
@@ -16,6 +16,7 @@ package org.apache.geode.management.internal.cli.commands;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
+import java.util.Arrays;
 import java.util.List;
 
 import org.junit.Before;
@@ -26,17 +27,13 @@ import org.junit.rules.TestName;
 
 import org.apache.geode.cache.CacheLoader;
 import org.apache.geode.cache.CacheLoaderException;
-import org.apache.geode.cache.CacheWriter;
-import org.apache.geode.cache.CacheWriterException;
 import org.apache.geode.cache.CustomExpiry;
 import org.apache.geode.cache.Declarable;
-import org.apache.geode.cache.EntryEvent;
 import org.apache.geode.cache.EntryOperation;
 import org.apache.geode.cache.ExpirationAttributes;
 import org.apache.geode.cache.LoaderHelper;
 import org.apache.geode.cache.PartitionResolver;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionEvent;
 import org.apache.geode.cache.configuration.CacheConfig;
 import org.apache.geode.cache.configuration.CacheElement;
 import org.apache.geode.cache.configuration.ExpirationAttributesType;
@@ -123,7 +120,7 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
   public void before() throws Exception {
     locator = clusterRule.startLocatorVM(0);
     server1 = clusterRule.startServerVM(1, locator.getPort());
-    server2 = clusterRule.startServerVM(2, locator.getPort());
+    // server2 = clusterRule.startServerVM(2, locator.getPort());
 
     gfsh.connectAndVerify(locator);
   }
@@ -257,9 +254,8 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
         + " --cache-writer=" + DummyCacheWriter.class.getName()
         + " --compressor=" + DummyCompressor.class.getName()
         + " --enable-async-conflation=false"
-        + " --enable-cloning=false"
-        + " --enable-concurrency-checks=true"
-        + " --enable-multicast=true"
+        + " --enable-concurrency-checks=false"
+        + " --enable-multicast=false"
         + " --concurrency-level=1"
         + " --enable-statistics=true"
         + " --enable-subscription-conflation=true"
@@ -267,26 +263,19 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
         + " --entry-idle-time-expiration-action=local-destroy"
         + " --entry-time-to-live-expiration=200"
         + " --entry-time-to-live-expiration-action=local-destroy"
-        + " --entry-idle-time-custom-expiry=" + DummyCustomExpiry.class.getName()
         + " --eviction-action=local-destroy"
-//        + " --eviction-entry-count=7" TODO
-        + " --eviction-max-memory=700"
-        + " --eviction-object-sizer=" + DummyObjectSizer.class.getName()
         + " --key-constraint=" + Object.class.getName()
-        + " --local-max-memory=500"
         + " --off-heap=false"
-        + " --partition-resolver=" + DummyPartitionResolver.class.getName()
         + " --region-idle-time-expiration=100"
         + " --region-idle-time-expiration-action=local-destroy"
         + " --region-time-to-live-expiration=200"
         + " --region-time-to-live-expiration-action=local-destroy"
-        + " --recovery-delay=1"
-        + " --redundant-copies=1"
-        + " --startup-recovery-delay=1"
-        + " --total-max-memory=100"
-        + " --total-num-buckets=1"
-        + " --value-constraint=" + Object.class.getName()
-    ).statusIsSuccess();
+        + " --value-constraint=" + Object.class.getName()).statusIsSuccess();
+
+    String regionNameFromTemplate = regionName + "-from-template";
+    gfsh.executeAndAssertThat("create region --name=" + regionNameFromTemplate
+        + " --template-region=" + regionName)
+        .statusIsSuccess();
 
     locator.invoke(() -> {
       InternalConfigurationPersistenceService cc =
@@ -295,28 +284,105 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
 
       List<RegionConfig> regions = config.getRegions();
       assertThat(regions).isNotEmpty();
-      RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), regionName);
-
-      assertThat(regionConfig).isNotNull();
-      assertThat(regionConfig.getName()).isEqualTo(regionName);
-      assertThat(regionConfig.getRegionAttributes()).hasSize(1);
-
-      RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
-      assertThat(attr.isStatisticsEnabled()).isTrue();
-      assertThat(attr.isEnableAsyncConflation()).isTrue();
-
-      ExpirationAttributesType entryIdleTimeExp = attr.getEntryIdleTime().getExpirationAttributes();
-      assertThat(entryIdleTimeExp.getTimeout()).isEqualTo("100");
+      assertThat(regions).hasSize(2);
+
+      List<String> regionNames = Arrays.asList(regionName, regionNameFromTemplate);
+      regionNames.forEach(name -> {
+        RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), name);
+        assertThat(regionConfig).isNotNull();
+        assertThat(regionConfig.getName()).isEqualTo(name);
+        assertThat(regionConfig.getRegionAttributes())
+            .describedAs("Expecting region attributes to exist")
+            .hasSize(1);
+
+        RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+        assertThat(attr.getCacheListeners().get(0).toString())
+            .describedAs("Expecting one cache listener for region " + name)
+            .isEqualTo(DummyCacheListener.class.getName());
+        assertThat(attr.getCacheLoader().toString())
+            .describedAs("Expecting a DummyCacheLoader for region " + name)
+            .isEqualTo(DummyCacheLoader.class.getName());
+        assertThat(attr.getCacheWriter().toString())
+            .describedAs("Expecting a DummyCacheWriter for region " + name)
+            .isEqualTo(DummyCacheWriter.class.getName());
+        assertThat(attr.getCompressor().toString())
+            .describedAs("Expecting a DummyCompressor for region " + name)
+            .isEqualTo(DummyCompressor.class.getName());
+        assertThat(attr.isEnableAsyncConflation())
+            .describedAs("Expecting async conflation to not be enabled for region "
+                + name)
+            .isFalse();
+        assertThat(attr.isConcurrencyChecksEnabled())
+            .describedAs("Expecting concurrency checks not to be enabled for region "
+                + name)
+            .isFalse();
+        assertThat(attr.isMulticastEnabled())
+            .describedAs("Expecting multicast is not enabled for region " + name)
+            .isFalse();
+        assertThat(attr.getConcurrencyLevel())
+            .describedAs("Expecting concurrency level to be 1 for region " + name)
+            .isEqualTo("1");
+        assertThat(attr.isStatisticsEnabled())
+            .describedAs("Expecting statistics to be enabled for region " + name)
+            .isTrue();
+        assertThat(attr.isEnableSubscriptionConflation())
+            .describedAs("Expecting subscription conflation to be enabled for region "
+                + name)
+            .isTrue();
+        assertThat(attr.getEntryIdleTime().getExpirationAttributes().getTimeout())
+            .describedAs("Entry idle time timeout should be 100 for region " + name)
+            .isEqualTo("100");
+        assertThat(attr.getEntryIdleTime().getExpirationAttributes().getAction())
+            .describedAs("Entry idle time expiration action should be local-destroy for region "
+                + name)
+            .isEqualTo("local-destroy");
+        assertThat(attr.getEntryTimeToLive().getExpirationAttributes().getTimeout())
+            .describedAs("Expecting entry time to live expiration to be 200 for region "
+                + name)
+            .isEqualTo("200");
+        assertThat(attr.getEntryTimeToLive().getExpirationAttributes().getAction())
+            .describedAs("Entry time to live expiration action should be local-destroy "
+                + "for region " + name)
+            .isEqualTo("local-destroy");
+        assertThat(attr.getEvictionAttributes().getLruHeapPercentage().getAction().value())
+            .describedAs("Eviction action should be local-destroy for region " + name)
+            .isEqualTo("local-destroy");
+        assertThat(attr.getKeyConstraint())
+            .describedAs("Expected key constraint to be " + Object.class.getName() +
+                " for region " + name)
+            .isEqualTo(Object.class.getName());
+        assertThat(attr.isOffHeap())
+            .describedAs("Expected off heap to be false for region " + name)
+            .isFalse();
+        assertThat(attr.getRegionIdleTime().getExpirationAttributes().getTimeout())
+            .describedAs("Expecting region idle time expiration to be 100 for region "
+                + name)
+            .isEqualTo("100");
+        assertThat(attr.getRegionIdleTime().getExpirationAttributes().getAction())
+            .describedAs("Expecting region idle time expiration action to be "
+                + "local-destroy for region " + name)
+            .isEqualTo("local-destroy");
+        assertThat(attr.getRegionTimeToLive().getExpirationAttributes().getTimeout())
+            .describedAs("Expecting region idle time timeout to be 200 for "
+                + "region " + name)
+            .isEqualTo("200");
+        assertThat(attr.getRegionTimeToLive().getExpirationAttributes().getAction())
+            .describedAs("Expecting region ttl action to be local-destroy for "
+                + "region " + name)
+            .isEqualTo("local-destroy");
+        assertThat(attr.getValueConstraint())
+            .describedAs("Expecting value constraint to be Object.class for "
+                + "region " + name)
+            .isEqualTo(Object.class.getName());
+      });
     });
   }
 
   @Test
-  public void placeholderAEQ() {
-  }
+  public void placeholderAEQ() {}
 
   @Test
-  public void placeholderColocation() {
-  }
+  public void placeholderColocation() {}
 
   @Test
   public void placeholderDiskstores() {
@@ -325,6 +391,33 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
 
   @Test
   public void placeholderPartitionedRegion() {
-    // test disk-synchronous
+    // test disk-synchronous String regionName = testName.getMethodName();
+    String regionName = testName.getMethodName();
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionName
+        + " --type=PARTITION"
+        + " --partition-resolver=" + DummyPartitionResolver.class.getName()
+        + " --recovery-delay=1"
+        + " --redundant-copies=1"
+        + " --startup-recovery-delay=1"
+        + " --total-max-memory=100"
+        + " --total-num-buckets=1"
+        + " --eviction-max-memory=700"
+        + " --eviction-entry-count=7"
+        + " --eviction-object-sizer=" + DummyObjectSizer.class.getName()).statusIsSuccess();
+  }
+
+  @Test
+  public void placeholderCustomExpiryClass() {
+    // + " --entry-idle-time-custom-expiry=" + DummyCustomExpiry.class.getName()
+    // assertThat(attr.getEntryIdleTime().getExpirationAttributes().getCustomExpiry().toString())
+    // .describedAs("Entry expiration custom expiration should be DummyCustomExpiry")
+    // .isEqualTo(DummyCustomExpiry.class.getName());
+
+  }
+
+  @Test
+  public void placeHolderDisableCloning() {
+    // " --enable-cloning=false"
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java b/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
index 66a1bb7..bc23920 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
@@ -16,6 +16,8 @@
 package org.apache.geode.cache;
 
 import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.configuration.EnumActionDestroyOverflow;
+import org.apache.geode.cache.configuration.RegionAttributesType;
 import org.apache.geode.cache.control.ResourceManager;
 import org.apache.geode.cache.util.ObjectSizer;
 import org.apache.geode.internal.cache.EvictionAttributesImpl;
@@ -505,4 +507,37 @@ public abstract class EvictionAttributes implements DataSerializable {
         .setAction(evictionAction).setMaximum(maximumMegabytes).setObjectSizer(null);
   }
 
+  public RegionAttributesType.EvictionAttributes convertToConfigEvictionAttributes() {
+    RegionAttributesType.EvictionAttributes configAttributes =
+        new RegionAttributesType.EvictionAttributes();
+    EnumActionDestroyOverflow action = EnumActionDestroyOverflow.fromValue(this.getAction()
+        .toString());
+    EvictionAlgorithm algorithm = getAlgorithm();
+    String objectSizerClass = getObjectSizer().getClass().toString();
+    Integer maximum = getMaximum();
+
+    if (algorithm.isLRUHeap()) {
+      RegionAttributesType.EvictionAttributes.LruHeapPercentage heapPercentage =
+          new RegionAttributesType.EvictionAttributes.LruHeapPercentage();
+      heapPercentage.setAction(action);
+      heapPercentage.setClassName(objectSizerClass);
+      configAttributes.setLruHeapPercentage(heapPercentage);
+    } else if (algorithm.isLRUMemory()) {
+      RegionAttributesType.EvictionAttributes.LruMemorySize memorySize =
+          new RegionAttributesType.EvictionAttributes.LruMemorySize();
+      memorySize.setAction(action);
+      memorySize.setClassName(objectSizerClass);
+      memorySize.setMaximum(maximum.toString());
+      configAttributes.setLruMemorySize(memorySize);
+    } else {
+      RegionAttributesType.EvictionAttributes.LruEntryCount entryCount =
+          new RegionAttributesType.EvictionAttributes.LruEntryCount();
+      entryCount.setAction(action);
+      entryCount.setMaximum(maximum.toString());
+      configAttributes.setLruEntryCount(entryCount);
+    }
+
+    return configAttributes;
+  }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/configuration/RegionConfigFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/configuration/RegionConfigFactory.java
index 61a7127..8aa98f7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/configuration/RegionConfigFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/configuration/RegionConfigFactory.java
@@ -18,6 +18,7 @@ import java.util.List;
 import java.util.Optional;
 import java.util.stream.Collectors;
 
+import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.configuration.ClassNameType;
 import org.apache.geode.cache.configuration.DeclarableType;
 import org.apache.geode.cache.configuration.ExpirationAttributesType;
@@ -30,6 +31,7 @@ public class RegionConfigFactory {
     RegionConfig regionConfig = new RegionConfig();
     regionConfig.setName(getLeafRegion(args.getRegionPath()));
 
+    RegionAttributes<?, ?> regionAttributes = args.getRegionAttributes();
     if (args.getRegionShortcut() != null) {
       regionConfig.setRefid(args.getRegionShortcut().toString());
     }
@@ -44,6 +46,9 @@ public class RegionConfigFactory {
 
     if (args.getStatisticsEnabled() != null) {
       addAttribute(regionConfig, a -> a.setStatisticsEnabled(args.getStatisticsEnabled()));
+    } else if (regionAttributes != null) {
+      addAttribute(regionConfig, a -> a.setStatisticsEnabled(regionAttributes
+          .getStatisticsEnabled()));
     }
 
     if (args.getEntryExpirationIdleTime() != null) {
@@ -51,6 +56,31 @@ public class RegionConfigFactory {
       entryIdleTime.setExpirationAttributes(
           args.getEntryExpirationIdleTime().getExpirationAttributes().toConfigType());
       addAttribute(regionConfig, a -> a.setEntryIdleTime(entryIdleTime));
+    } else if (regionAttributes != null &&
+        regionAttributes.getEntryIdleTimeout() != null) {
+      RegionAttributesType.EntryIdleTime entryIdleTime = new RegionAttributesType.EntryIdleTime();
+      entryIdleTime.setExpirationAttributes(regionAttributes
+          .getEntryIdleTimeout().toConfigType());
+      addAttribute(regionConfig, a -> a.setEntryIdleTime(entryIdleTime));
+    }
+
+    if (args.getEntryIdleTimeCustomExpiry() != null) {
+      Object maybeEntryIdleAttr = getRegionAttributeValue(regionConfig, a -> a.getEntryIdleTime());
+      RegionAttributesType.EntryIdleTime entryIdleTime =
+          maybeEntryIdleAttr != null ? (RegionAttributesType.EntryIdleTime) maybeEntryIdleAttr
+              : new RegionAttributesType.EntryIdleTime();
+      ExpirationAttributesType expirationAttributes =
+          entryIdleTime.getExpirationAttributes() == null ? new ExpirationAttributesType()
+              : entryIdleTime.getExpirationAttributes();
+
+      DeclarableType customExpiry = new DeclarableType();
+      customExpiry.setClassName(args.getEntryIdleTimeCustomExpiry().getClassName());
+      expirationAttributes.setCustomExpiry(customExpiry);
+      entryIdleTime.setExpirationAttributes(expirationAttributes);
+
+      if (maybeEntryIdleAttr == null) {
+        addAttribute(regionConfig, a -> a.setEntryIdleTime(entryIdleTime));
+      }
     }
 
     if (args.getEntryExpirationTTL() != null) {
@@ -59,6 +89,13 @@ public class RegionConfigFactory {
       entryExpTime.setExpirationAttributes(
           args.getEntryExpirationTTL().getExpirationAttributes().toConfigType());
       addAttribute(regionConfig, a -> a.setEntryTimeToLive(entryExpTime));
+    } else if (regionAttributes != null
+        && regionAttributes.getEntryTimeToLive() != null) {
+      RegionAttributesType.EntryTimeToLive entryExpTime =
+          new RegionAttributesType.EntryTimeToLive();
+      entryExpTime.setExpirationAttributes(
+          regionAttributes.getEntryTimeToLive().toConfigType());
+      addAttribute(regionConfig, a -> a.setEntryTimeToLive(entryExpTime));
     }
 
     if (args.getRegionExpirationIdleTime() != null) {
@@ -67,6 +104,13 @@ public class RegionConfigFactory {
       regionIdleTime.setExpirationAttributes(
           args.getRegionExpirationIdleTime().getExpirationAttributes().toConfigType());
       addAttribute(regionConfig, a -> a.setRegionIdleTime(regionIdleTime));
+    } else if (regionAttributes != null &&
+        regionAttributes.getRegionIdleTimeout() != null) {
+      RegionAttributesType.RegionIdleTime regionIdleTime =
+          new RegionAttributesType.RegionIdleTime();
+      regionIdleTime.setExpirationAttributes(
+          regionAttributes.getRegionIdleTimeout().toConfigType());
+      addAttribute(regionConfig, a -> a.setRegionIdleTime(regionIdleTime));
     }
 
     if (args.getRegionExpirationTTL() != null) {
@@ -75,6 +119,13 @@ public class RegionConfigFactory {
       regionExpTime.setExpirationAttributes(
           args.getRegionExpirationTTL().getExpirationAttributes().toConfigType());
       addAttribute(regionConfig, a -> a.setRegionTimeToLive(regionExpTime));
+    } else if (regionAttributes != null &&
+        regionAttributes.getRegionTimeToLive() != null) {
+      RegionAttributesType.RegionTimeToLive regionExpTime =
+          new RegionAttributesType.RegionTimeToLive();
+      regionExpTime.setExpirationAttributes(
+          regionAttributes.getRegionTimeToLive().toConfigType());
+      addAttribute(regionConfig, a -> a.setRegionTimeToLive(regionExpTime));
     }
 
     if (args.getEntryTTLCustomExpiry() != null) {
@@ -96,25 +147,6 @@ public class RegionConfigFactory {
       }
     }
 
-    if (args.getEntryIdleTimeCustomExpiry() != null) {
-      Object maybeEntryIdleAttr = getRegionAttributeValue(regionConfig, a -> a.getEntryIdleTime());
-      RegionAttributesType.EntryIdleTime entryIdleTime =
-          maybeEntryIdleAttr != null ? (RegionAttributesType.EntryIdleTime) maybeEntryIdleAttr
-              : new RegionAttributesType.EntryIdleTime();
-      ExpirationAttributesType expirationAttributes =
-          entryIdleTime.getExpirationAttributes() == null ? new ExpirationAttributesType()
-              : entryIdleTime.getExpirationAttributes();
-
-      DeclarableType customExpiry = new DeclarableType();
-      customExpiry.setClassName(args.getEntryIdleTimeCustomExpiry().getClassName());
-      expirationAttributes.setCustomExpiry(customExpiry);
-      entryIdleTime.setExpirationAttributes(expirationAttributes);
-
-      if (maybeEntryIdleAttr == null) {
-        addAttribute(regionConfig, a -> a.setEntryIdleTime(entryIdleTime));
-      }
-    }
-
     if (args.getDiskStore() != null) {
       addAttribute(regionConfig, a -> a.setDiskStoreName(args.getDiskStore()));
     }
@@ -125,28 +157,45 @@ public class RegionConfigFactory {
 
     if (args.getEnableAsyncConflation() != null) {
       addAttribute(regionConfig, a -> a.setEnableAsyncConflation(args.getEnableAsyncConflation()));
+    } else if (regionAttributes != null) {
+      addAttribute(regionConfig, a -> a.setEnableAsyncConflation(regionAttributes
+          .getEnableAsyncConflation()));
     }
 
     if (args.getEnableSubscriptionConflation() != null) {
       addAttribute(regionConfig,
           a -> a.setEnableSubscriptionConflation(args.getEnableSubscriptionConflation()));
+    } else if (regionAttributes != null) {
+      addAttribute(regionConfig, a -> a.setEnableSubscriptionConflation(regionAttributes
+          .getEnableSubscriptionConflation()));
     }
 
     if (args.getConcurrencyChecksEnabled() != null) {
       addAttribute(regionConfig, a -> a.setConcurrencyChecksEnabled(
           args.getConcurrencyChecksEnabled()));
+    } else if (regionAttributes != null) {
+      addAttribute(regionConfig, a -> a.setConcurrencyChecksEnabled(regionAttributes
+          .getConcurrencyChecksEnabled()));
     }
 
     if (args.getCloningEnabled() != null) {
       addAttribute(regionConfig, a -> a.setCloningEnabled(args.getCloningEnabled()));
+    } else if (regionAttributes != null) {
+      addAttribute(regionConfig, a -> a.setCloningEnabled(regionAttributes
+          .getCloningEnabled()));
     }
 
     if (args.getOffHeap() != null) {
       addAttribute(regionConfig, a -> a.setOffHeap(args.getOffHeap()));
+    } else if (regionAttributes != null) {
+      addAttribute(regionConfig, a -> a.setOffHeap(regionAttributes.getOffHeap()));
     }
 
     if (args.getMcastEnabled() != null) {
       addAttribute(regionConfig, a -> a.setMulticastEnabled(args.getMcastEnabled()));
+    } else if (regionAttributes != null) {
+      addAttribute(regionConfig, a -> a.setMulticastEnabled(regionAttributes
+          .getMulticastEnabled()));
     }
 
     if (args.getPartitionArgs() != null) {
@@ -177,6 +226,9 @@ public class RegionConfigFactory {
     if (args.getEvictionAttributes() != null) {
       addAttribute(regionConfig, a -> a.setEvictionAttributes(
           args.getEvictionAttributes().convertToConfigEvictionAttributes()));
+    } else if (regionAttributes != null && regionAttributes.getEvictionAttributes() != null) {
+      addAttribute(regionConfig, a -> a.setEvictionAttributes(
+          regionAttributes.getEvictionAttributes().convertToConfigEvictionAttributes()));
     }
 
     if (args.getAsyncEventQueueIds() != null && !args.getAsyncEventQueueIds().isEmpty()) {
@@ -211,6 +263,9 @@ public class RegionConfigFactory {
 
     if (args.getConcurrencyLevel() != null) {
       addAttribute(regionConfig, a -> a.setConcurrencyLevel(args.getConcurrencyLevel().toString()));
+    } else if (regionAttributes != null) {
+      addAttribute(regionConfig, a -> a.setConcurrencyLevel(Integer.toString(
+          regionAttributes.getConcurrencyLevel())));
     }
 
     return regionConfig;
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 fb16bff..82ae692 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
@@ -73,6 +73,7 @@ import org.apache.geode.management.internal.cli.util.RegionPath;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission;
 
+// public class CreateRegionCommand extends InternalGfshCommand {
 public class CreateRegionCommand extends SingleGfshCommand {
   @CliCommand(value = CliStrings.CREATE_REGION, help = CliStrings.CREATE_REGION__HELP)
   @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION,
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/configuration/RegionConfigFactoryTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/configuration/RegionConfigFactoryTest.java
index 924e225..4003c44 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/configuration/RegionConfigFactoryTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/configuration/RegionConfigFactoryTest.java
@@ -23,7 +23,6 @@ import org.junit.Test;
 
 import org.apache.geode.cache.EvictionAction;
 import org.apache.geode.cache.ExpirationAction;
-import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.configuration.ClassNameType;
 import org.apache.geode.cache.configuration.DeclarableType;