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 2018/12/06 19:13:35 UTC

[geode] 02/03: GEODE-5971: RegionConfig can only have one RegionAttributesType

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

commit 846af4e2df0b6c76585e8a16e1eeb26e71e9c2dd
Author: Jinmei Liao <ji...@pivotal.io>
AuthorDate: Mon Dec 3 22:51:49 2018 -0800

    GEODE-5971: RegionConfig can only have one RegionAttributesType
    
    Signed-off-by: Jinmei Liao <ji...@pivotal.io>
---
 .../cli/CreateMappingCommandDUnitTest.java         |   2 +-
 .../cli/DestroyMappingCommandDunitTest.java        |   2 +-
 .../jdbc/internal/cli/CreateMappingCommand.java    |  28 +++---
 .../jdbc/internal/cli/DestroyMappingCommand.java   |  22 ++---
 .../internal/cli/CreateMappingCommandTest.java     |  24 ++---
 .../internal/cli/DestroyMappingCommandTest.java    |   4 +-
 ...egionCommandPersistsConfigurationDUnitTest.java | 110 +++++++++++++--------
 .../commands/DestroyRegionCommandDUnitTest.java    |   7 +-
 .../main/java/org/apache/geode/cache/Scope.java    |   4 +
 .../geode/cache/configuration/RegionConfig.java    |  45 ++-------
 .../cli/domain/RegionAttributeGetFunction.java     |  22 -----
 .../cli/domain/RegionAttributeSetFunction.java     |  22 -----
 .../internal/cli/domain/RegionConfigFactory.java   |  36 ++++---
 .../cli/functions/RegionCreateFunction.java        |  14 ---
 .../sanctioned-geode-core-serializables.txt        |   2 +-
 .../geode/cache/configuration/CacheConfigTest.java |  18 ++++
 .../cli/domain/RegionConfigFactoryTest.java        |  76 ++++++--------
 17 files changed, 187 insertions(+), 251 deletions(-)

diff --git a/geode-connectors/src/distributedTest/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandDUnitTest.java b/geode-connectors/src/distributedTest/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandDUnitTest.java
index d71f257..2bd74ef 100644
--- a/geode-connectors/src/distributedTest/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandDUnitTest.java
+++ b/geode-connectors/src/distributedTest/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandDUnitTest.java
@@ -142,7 +142,7 @@ public class CreateMappingCommandDUnitTest {
     RegionConfig regionConfig = cacheConfig.getRegions().stream()
         .filter(region -> region.getName().equals(convertRegionPathToName(regionName))).findFirst()
         .orElse(null);
-    RegionAttributesType attributes = regionConfig.getRegionAttributes().get(0);
+    RegionAttributesType attributes = regionConfig.getRegionAttributes();
     assertThat(attributes.getCacheLoader().getClassName()).isEqualTo(JdbcLoader.class.getName());
     if (synchronous) {
       assertThat(attributes.getCacheWriter().getClassName()).isEqualTo(JdbcWriter.class.getName());
diff --git a/geode-connectors/src/distributedTest/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommandDunitTest.java b/geode-connectors/src/distributedTest/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommandDunitTest.java
index a7b8d38..2cfcc7e 100644
--- a/geode-connectors/src/distributedTest/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommandDunitTest.java
+++ b/geode-connectors/src/distributedTest/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommandDunitTest.java
@@ -203,7 +203,7 @@ public class DestroyMappingCommandDunitTest implements Serializable {
         InternalLocator.getLocator().getConfigurationPersistenceService().getCacheConfig(null);
     RegionConfig regionConfig = cacheConfig.getRegions().stream()
         .filter(region -> region.getName().equals(REGION_NAME)).findFirst().orElse(null);
-    RegionAttributesType attributes = regionConfig.getRegionAttributes().get(0);
+    RegionAttributesType attributes = regionConfig.getRegionAttributes();
     assertThat(attributes.getCacheLoader()).isNull();
     if (synchronous) {
       assertThat(attributes.getCacheWriter()).isNull();
diff --git a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommand.java b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommand.java
index dd47895..5515f42 100644
--- a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommand.java
+++ b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommand.java
@@ -147,8 +147,7 @@ public class CreateMappingCommand extends SingleGfshCommand {
 
   private void checkForCacheLoader(String regionName, RegionConfig regionConfig)
       throws PreconditionException {
-    RegionAttributesType regionAttributes = regionConfig.getRegionAttributes().stream()
-        .filter(attributes -> attributes.getCacheLoader() != null).findFirst().orElse(null);
+    RegionAttributesType regionAttributes = regionConfig.getRegionAttributes();
     if (regionAttributes != null) {
       DeclarableType loaderDeclarable = regionAttributes.getCacheLoader();
       if (loaderDeclarable != null) {
@@ -162,8 +161,7 @@ public class CreateMappingCommand extends SingleGfshCommand {
   private void checkForCacheWriter(String regionName, boolean synchronous,
       RegionConfig regionConfig) throws PreconditionException {
     if (synchronous) {
-      RegionAttributesType writerAttributes = regionConfig.getRegionAttributes().stream()
-          .filter(attributes -> attributes.getCacheWriter() != null).findFirst().orElse(null);
+      RegionAttributesType writerAttributes = regionConfig.getRegionAttributes();
       if (writerAttributes != null) {
         DeclarableType writerDeclarable = writerAttributes.getCacheWriter();
         if (writerDeclarable != null) {
@@ -200,7 +198,7 @@ public class CreateMappingCommand extends SingleGfshCommand {
       return false;
     }
 
-    RegionAttributesType attributes = getRegionAttributes(regionConfig);
+    RegionAttributesType attributes = getRegionAttribute(regionConfig);
     addMappingToRegion(regionMapping, regionConfig);
     if (!synchronous) {
       createAsyncQueue(cacheConfig, attributes, queueName);
@@ -210,6 +208,14 @@ public class CreateMappingCommand extends SingleGfshCommand {
     return true;
   }
 
+  private RegionAttributesType getRegionAttribute(RegionConfig config) {
+    if (config.getRegionAttributes() == null) {
+      config.setRegionAttributes(new RegionAttributesType());
+    }
+
+    return config.getRegionAttributes();
+  }
+
   @CliAvailabilityIndicator({CREATE_MAPPING})
   public boolean commandAvailable() {
     return isOnlineCommandAvailable();
@@ -272,16 +278,4 @@ public class CreateMappingCommand extends SingleGfshCommand {
     writer.setClassName(JdbcWriter.class.getName());
     attributes.setCacheWriter(writer);
   }
-
-  private RegionAttributesType getRegionAttributes(RegionConfig regionConfig) {
-    RegionAttributesType attributes;
-    List<RegionAttributesType> attributesList = regionConfig.getRegionAttributes();
-    if (attributesList.isEmpty()) {
-      attributes = new RegionAttributesType();
-      attributesList.add(attributes);
-    } else {
-      attributes = attributesList.get(0);
-    }
-    return attributes;
-  }
 }
diff --git a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommand.java b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommand.java
index 0f32a8f..36098bf 100644
--- a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommand.java
+++ b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommand.java
@@ -84,13 +84,21 @@ public class DestroyMappingCommand extends SingleGfshCommand {
     boolean modified = false;
     modified |= removeJdbcMappingFromRegion(regionConfig);
     modified |= removeJdbcQueueFromCache(cacheConfig, regionName);
-    RegionAttributesType attributes = getRegionAttributes(regionConfig);
+    RegionAttributesType attributes = getRegionAttribute(regionConfig);
     modified |= removeJdbcLoader(attributes);
     modified |= removeJdbcWriter(attributes);
     modified |= removeJdbcAsyncEventQueueId(attributes, regionName);
     return modified;
   }
 
+  private RegionAttributesType getRegionAttribute(RegionConfig config) {
+    if (config.getRegionAttributes() == null) {
+      config.setRegionAttributes(new RegionAttributesType());
+    }
+
+    return config.getRegionAttributes();
+  }
+
   private boolean removeJdbcLoader(RegionAttributesType attributes) {
     DeclarableType cacheLoader = attributes.getCacheLoader();
     if (cacheLoader != null) {
@@ -159,18 +167,6 @@ public class DestroyMappingCommand extends SingleGfshCommand {
         .filter(region -> region.getName().equals(regionName)).findFirst().orElse(null);
   }
 
-  private RegionAttributesType getRegionAttributes(RegionConfig regionConfig) {
-    RegionAttributesType attributes;
-    List<RegionAttributesType> attributesList = regionConfig.getRegionAttributes();
-    if (attributesList.isEmpty()) {
-      attributes = new RegionAttributesType();
-      attributesList.add(attributes);
-    } else {
-      attributes = attributesList.get(0);
-    }
-    return attributes;
-  }
-
   @CliAvailabilityIndicator({DESTROY_MAPPING})
   public boolean commandAvailable() {
     return isOnlineCommandAvailable();
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandTest.java
index cded45f..02211fa 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateMappingCommandTest.java
@@ -97,11 +97,9 @@ public class CreateMappingCommandTest {
 
     matchingRegion = mock(RegionConfig.class);
     when(matchingRegion.getName()).thenReturn(regionName);
-    List<RegionAttributesType> attributesList = new ArrayList<>();
     matchingRegionAttributes = mock(RegionAttributesType.class);
     when(matchingRegionAttributes.getDataPolicy()).thenReturn(RegionAttributesDataPolicy.REPLICATE);
-    attributesList.add(matchingRegionAttributes);
-    when(matchingRegion.getRegionAttributes()).thenReturn(attributesList);
+    when(matchingRegion.getRegionAttributes()).thenReturn(matchingRegionAttributes);
 
     arguments[0] = mapping;
     arguments[1] = false;
@@ -205,13 +203,11 @@ public class CreateMappingCommandTest {
     List<RegionConfig> list = new ArrayList<>();
     list.add(matchingRegion);
     when(cacheConfig.getRegions()).thenReturn(list);
-    List<RegionAttributesType> attributes = new ArrayList<>();
     RegionAttributesType loaderAttribute = mock(RegionAttributesType.class);
     DeclarableType loaderDeclarable = mock(DeclarableType.class);
     when(loaderDeclarable.getClassName()).thenReturn(null);
     when(loaderAttribute.getCacheLoader()).thenReturn(loaderDeclarable);
-    attributes.add(loaderAttribute);
-    when(matchingRegion.getRegionAttributes()).thenReturn(attributes);
+    when(matchingRegion.getRegionAttributes()).thenReturn(loaderAttribute);
     List<CacheElement> customList = new ArrayList<>();
     RegionMapping existingMapping = mock(RegionMapping.class);
     customList.add(existingMapping);
@@ -235,13 +231,11 @@ public class CreateMappingCommandTest {
     List<RegionConfig> list = new ArrayList<>();
     list.add(matchingRegion);
     when(cacheConfig.getRegions()).thenReturn(list);
-    List<RegionAttributesType> attributes = new ArrayList<>();
     RegionAttributesType loaderAttribute = mock(RegionAttributesType.class);
     DeclarableType loaderDeclarable = mock(DeclarableType.class);
     when(loaderDeclarable.getClassName()).thenReturn("MyCacheLoaderClass");
     when(loaderAttribute.getCacheLoader()).thenReturn(loaderDeclarable);
-    attributes.add(loaderAttribute);
-    when(matchingRegion.getRegionAttributes()).thenReturn(attributes);
+    when(matchingRegion.getRegionAttributes()).thenReturn(loaderAttribute);
 
     ResultModel result = createRegionMappingCommand.createMapping(regionName, dataSourceName,
         tableName, pdxClass, false);
@@ -262,13 +256,11 @@ public class CreateMappingCommandTest {
     List<RegionConfig> list = new ArrayList<>();
     list.add(matchingRegion);
     when(cacheConfig.getRegions()).thenReturn(list);
-    List<RegionAttributesType> attributes = new ArrayList<>();
     RegionAttributesType writerAttribute = mock(RegionAttributesType.class);
     DeclarableType writerDeclarable = mock(DeclarableType.class);
     when(writerDeclarable.getClassName()).thenReturn("MyCacheWriterClass");
     when(writerAttribute.getCacheWriter()).thenReturn(writerDeclarable);
-    attributes.add(writerAttribute);
-    when(matchingRegion.getRegionAttributes()).thenReturn(attributes);
+    when(matchingRegion.getRegionAttributes()).thenReturn(writerAttribute);
 
     ResultModel result = createRegionMappingCommand.createMapping(regionName, dataSourceName,
         tableName, pdxClass, true);
@@ -289,11 +281,9 @@ public class CreateMappingCommandTest {
     List<RegionConfig> list = new ArrayList<>();
     list.add(matchingRegion);
     when(cacheConfig.getRegions()).thenReturn(list);
-    List<RegionAttributesType> attributes = new ArrayList<>();
     RegionAttributesType loaderAttribute = mock(RegionAttributesType.class);
     when(loaderAttribute.getCacheLoader()).thenReturn(null);
-    attributes.add(loaderAttribute);
-    when(matchingRegion.getRegionAttributes()).thenReturn(attributes);
+    when(matchingRegion.getRegionAttributes()).thenReturn(loaderAttribute);
     List<AsyncEventQueue> asyncEventQueues = new ArrayList<>();
     AsyncEventQueue matchingQueue = mock(AsyncEventQueue.class);
     String queueName = createRegionMappingCommand.createAsyncEventQueueName(regionName);
@@ -319,11 +309,9 @@ public class CreateMappingCommandTest {
     List<RegionConfig> list = new ArrayList<>();
     list.add(matchingRegion);
     when(cacheConfig.getRegions()).thenReturn(list);
-    List<RegionAttributesType> attributes = new ArrayList<>();
     RegionAttributesType loaderAttribute = mock(RegionAttributesType.class);
     when(loaderAttribute.getCacheLoader()).thenReturn(null);
-    attributes.add(loaderAttribute);
-    when(matchingRegion.getRegionAttributes()).thenReturn(attributes);
+    when(matchingRegion.getRegionAttributes()).thenReturn(loaderAttribute);
     List<AsyncEventQueue> asyncEventQueues = new ArrayList<>();
     AsyncEventQueue matchingQueue = mock(AsyncEventQueue.class);
     String queueName = createRegionMappingCommand.createAsyncEventQueueName(regionName);
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommandTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommandTest.java
index 9018055..a633ff0 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommandTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyMappingCommandTest.java
@@ -86,11 +86,9 @@ public class DestroyMappingCommandTest {
 
     matchingRegion = mock(RegionConfig.class);
     when(matchingRegion.getName()).thenReturn(regionName);
-    List<RegionAttributesType> attributesList = new ArrayList<>();
     matchingRegionAttributes = mock(RegionAttributesType.class);
     when(matchingRegionAttributes.getDataPolicy()).thenReturn(RegionAttributesDataPolicy.REPLICATE);
-    attributesList.add(matchingRegionAttributes);
-    when(matchingRegion.getRegionAttributes()).thenReturn(attributesList);
+    when(matchingRegion.getRegionAttributes()).thenReturn(matchingRegionAttributes);
   }
 
   @Test
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 275fac9..c4e14f0 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
@@ -38,6 +38,7 @@ import org.apache.geode.cache.configuration.CacheConfig;
 import org.apache.geode.cache.configuration.CacheElement;
 import org.apache.geode.cache.configuration.ExpirationAttributesType;
 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.util.CacheListenerAdapter;
@@ -168,9 +169,9 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
       RegionConfig regionConfig = regions.get(0);
       assertThat(regionConfig).isNotNull();
       assertThat(regionConfig.getName()).isEqualTo(regionName);
-      assertThat(regionConfig.getRegionAttributes()).hasSize(1);
+      assertThat(regionConfig.getRegionAttributes()).isNotNull();
 
-      RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+      RegionAttributesType attr = regionConfig.getRegionAttributes();
       assertThat(attr.isStatisticsEnabled()).isTrue();
       assertThat(attr.isEnableAsyncConflation()).isTrue();
 
@@ -218,9 +219,9 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
 
       assertThat(regionConfig).isNotNull();
       assertThat(regionConfig.getName()).isEqualTo(regionName);
-      assertThat(regionConfig.getRegionAttributes()).hasSize(1);
+      assertThat(regionConfig.getRegionAttributes()).isNotNull();
 
-      RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+      RegionAttributesType attr = regionConfig.getRegionAttributes();
       assertThat(attr.isStatisticsEnabled()).isTrue();
       assertThat(attr.isEnableAsyncConflation()).isTrue();
 
@@ -277,11 +278,9 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
         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);
+        assertThat(regionConfig.getRegionAttributes()).isNotNull();
 
-        RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+        RegionAttributesType attr = regionConfig.getRegionAttributes();
         assertThat(attr.getCacheListeners().get(0).toString())
             .describedAs("Expecting one cache listener for region " + name)
             .isEqualTo(DummyCacheListener.class.getName());
@@ -389,11 +388,9 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
         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);
+        assertThat(regionConfig.getRegionAttributes()).isNotNull();
 
-        RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+        RegionAttributesType attr = regionConfig.getRegionAttributes();
         assertThat(attr.getEvictionAttributes())
             .describedAs("Eviction attributes should be null for " + name)
             .isNull();
@@ -425,7 +422,7 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
       assertThat(regions).isNotEmpty();
       assertThat(regions).hasSize(1);
       RegionConfig regionConfig = CacheElement.findElement(regions, regionName);
-      assertThat(regionConfig.getRegionAttributes().get(0).getAsyncEventQueueIds())
+      assertThat(regionConfig.getRegionAttributes().getAsyncEventQueueIds())
           .contains(queueId);
     });
   }
@@ -459,13 +456,13 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
 
       RegionConfig colocatedConfig = CacheElement.findElement(regions, colocatedRegionName);
       assertThat(
-          colocatedConfig.getRegionAttributes().get(0).getPartitionAttributes().getColocatedWith())
+          colocatedConfig.getRegionAttributes().getPartitionAttributes().getColocatedWith())
               .isEqualTo("/" + regionName);
 
       RegionConfig colocatedConfigFromTemplate = CacheElement.findElement(regions,
           colocatedRegionFromTemplateName);
       assertThat(
-          colocatedConfigFromTemplate.getRegionAttributes().get(0).getPartitionAttributes()
+          colocatedConfigFromTemplate.getRegionAttributes().getPartitionAttributes()
               .getColocatedWith())
                   .isEqualTo("/" + regionName);
     });
@@ -508,7 +505,7 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
         assertThat(regionConfig).isNotNull();
         assertThat(regionConfig.getName()).isEqualTo(name);
 
-        RegionAttributesType regionAttributes = regionConfig.getRegionAttributes().get(0);
+        RegionAttributesType regionAttributes = regionConfig.getRegionAttributes();
         assertThat(regionAttributes.getDiskStoreName())
             .isEqualTo(store);
         assertThat(regionAttributes.isDiskSynchronous())
@@ -550,7 +547,7 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
         assertThat(regionConfig).isNotNull();
         assertThat(regionConfig.getName()).isEqualTo(name);
 
-        RegionAttributesType regionAttributes = regionConfig.getRegionAttributes().get(0);
+        RegionAttributesType regionAttributes = regionConfig.getRegionAttributes();
         RegionAttributesType.PartitionAttributes partitionAttributes =
             regionAttributes.getPartitionAttributes();
 
@@ -598,7 +595,7 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
       assertThat(regionConfig).isNotNull();
       assertThat(regionConfig.getName()).isEqualTo(regionName);
 
-      RegionAttributesType regionAttributes = regionConfig.getRegionAttributes().get(0);
+      RegionAttributesType regionAttributes = regionConfig.getRegionAttributes();
       RegionAttributesType.PartitionAttributes partitionAttributes =
           regionAttributes.getPartitionAttributes();
 
@@ -633,11 +630,9 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
           CacheElement.findElement(config.getRegions(), regionFromTemplateName);
       assertThat(regionConfig).isNotNull();
       assertThat(regionConfig.getName()).isEqualTo(regionFromTemplateName);
-      assertThat(regionConfig.getRegionAttributes())
-          .describedAs("Expecting region attributes to exist")
-          .hasSize(1);
+      assertThat(regionConfig.getRegionAttributes()).isNotNull();
 
-      RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+      RegionAttributesType attr = regionConfig.getRegionAttributes();
       assertThat(attr.getPartitionAttributes())
           .describedAs("Partition attributes should be null for " + regionFromTemplateName)
           .isNull();
@@ -664,11 +659,9 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
           CacheElement.findElement(config.getRegions(), regionName);
       assertThat(regionConfig).isNotNull();
       assertThat(regionConfig.getName()).isEqualTo(regionName);
-      assertThat(regionConfig.getRegionAttributes())
-          .describedAs("Expecting region attributes to exist")
-          .hasSize(1);
+      assertThat(regionConfig.getRegionAttributes()).isNotNull();
 
-      RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+      RegionAttributesType attr = regionConfig.getRegionAttributes();
       assertThat(attr.getRegionTimeToLive())
           .describedAs("Expiration attributes should be null for " + regionName)
           .isNull();
@@ -712,11 +705,9 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
         RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), name);
         assertThat(regionConfig).isNotNull();
         assertThat(regionConfig.getName()).isEqualTo(name);
-        assertThat(regionConfig.getRegionAttributes())
-            .describedAs("Expecting region attributes to exist for " + name)
-            .hasSize(1);
+        assertThat(regionConfig.getRegionAttributes()).isNotNull();
 
-        RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+        RegionAttributesType attr = regionConfig.getRegionAttributes();
         assertThat(attr.isCloningEnabled())
             .describedAs("Cloning should be disabled for " + name)
             .isFalse();
@@ -746,10 +737,8 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
       RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), regionName);
       assertThat(regionConfig).isNotNull();
       assertThat(regionConfig.getName()).isEqualTo(regionName);
-      assertThat(regionConfig.getRegionAttributes())
-          .describedAs("Expecting region attributes to exist for " + regionName)
-          .hasSize(1);
-      RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+      assertThat(regionConfig.getRegionAttributes()).isNotNull();
+      RegionAttributesType attr = regionConfig.getRegionAttributes();
       assertThat(attr.getEntryIdleTime().getExpirationAttributes().getCustomExpiry().toString())
           .describedAs("Entry expiration custom expiration should be DummyCustomExpiry")
           .isEqualTo(DummyCustomExpiry.class.getName());
@@ -757,7 +746,7 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
   }
 
   @Test
-  public void createRegionPersistsImplicitTemplateAttributes() {
+  public void createRegionPersistsDataPolicy() {
     String regionName = testName.getMethodName();
     gfsh.executeAndAssertThat("create region"
         + " --name=" + regionName
@@ -776,14 +765,57 @@ public class CreateRegionCommandPersistsConfigurationDUnitTest {
       RegionConfig regionConfig = CacheElement.findElement(config.getRegions(), regionName);
       assertThat(regionConfig).isNotNull();
       assertThat(regionConfig.getName()).isEqualTo(regionName);
-      assertThat(regionConfig.getRegionAttributes())
-          .describedAs("Expecting region attributes to exist for " + regionName)
-          .hasSize(1);
+      assertThat(regionConfig.getRegionAttributes()).isNotNull();
 
-      RegionAttributesType attr = regionConfig.getRegionAttributes().get(0);
+      RegionAttributesType attr = regionConfig.getRegionAttributes();
       assertThat(attr.getDataPolicy())
           .describedAs("Data policy for partitioned region should be persisted correctly")
           .isEqualTo(RegionAttributesDataPolicy.PARTITION);
     });
   }
+
+  @Test
+  public void createRegionPersistsScope() {
+    String regionName = testName.getMethodName();
+    String regionName2 = regionName + "2";
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionName
+        + " --type=PARTITION")
+        .statusIsSuccess();
+    gfsh.executeAndAssertThat("create region"
+        + " --name=" + regionName2
+        + " --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(2);
+
+      RegionConfig regionConfig1 = CacheElement.findElement(config.getRegions(), regionName);
+      assertThat(regionConfig1).isNotNull();
+      assertThat(regionConfig1.getName()).isEqualTo(regionName);
+      assertThat(regionConfig1.getRegionAttributes()).isNotNull();
+
+      RegionAttributesType attr1 = regionConfig1.getRegionAttributes();
+      assertThat(attr1.getScope())
+          .describedAs("Scope for partitioned region should be null")
+          .isNull();
+
+      RegionConfig regionConfig2 = CacheElement.findElement(config.getRegions(), regionName2);
+      assertThat(regionConfig2).isNotNull();
+      assertThat(regionConfig2.getName()).isEqualTo(regionName2);
+      assertThat(regionConfig2.getRegionAttributes()).isNotNull();
+
+      RegionAttributesType attr2 = regionConfig2.getRegionAttributes();
+      assertThat(attr2.getScope())
+          .describedAs(
+              "Scope for replicated region should be persisted as distributed-ack by default")
+          .isEqualTo(RegionAttributesScope.DISTRIBUTED_ACK);
+    });
+  }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
index 0849ba3..511fe14 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
@@ -117,13 +117,14 @@ public class DestroyRegionCommandDUnitTest {
       Configuration group1Config = service.getConfiguration("group1");
       assertThat(group1Config.getCacheXmlContent())
           .containsOnlyOnce("<region name=\"region1\">")
-          .containsOnlyOnce("<region-attributes data-policy=\"empty\" scope=\"distributed-ack\"/>");
+          .containsOnlyOnce("data-policy=\"empty\"")
+          .containsOnlyOnce("scope=\"distributed-ack\"");
 
       Configuration clusterConfig = service.getConfiguration("group2");
       assertThat(clusterConfig.getCacheXmlContent())
           .containsOnlyOnce("<region name=\"region1\">")
-          .containsOnlyOnce(
-              "<region-attributes data-policy=\"replicate\" scope=\"distributed-ack\"/>");
+          .containsOnlyOnce("data-policy=\"replicate\"")
+          .containsOnlyOnce("scope=\"distributed-ack\"");
     });
 
     gfsh.executeAndAssertThat("destroy region --name=region1").statusIsSuccess()
diff --git a/geode-core/src/main/java/org/apache/geode/cache/Scope.java b/geode-core/src/main/java/org/apache/geode/cache/Scope.java
index 604f6a7..7517f56 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/Scope.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/Scope.java
@@ -157,6 +157,10 @@ public class Scope implements Serializable {
     return this.name;
   }
 
+  public String toConfigTypeString() {
+    return this.name.toLowerCase().replace("_", "-");
+  }
+
   /**
    * Parse the given string into a Scope
    *
diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
index 8dbb387..2638ecf 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
@@ -28,8 +28,6 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlType;
 
-import org.w3c.dom.Element;
-
 import org.apache.geode.annotations.Experimental;
 
 
@@ -151,9 +149,8 @@ import org.apache.geode.annotations.Experimental;
     propOrder = {"regionAttributes", "indexes", "entries", "regionElements", "regions"})
 @Experimental
 public class RegionConfig implements CacheElement {
-
   @XmlElement(name = "region-attributes", namespace = "http://geode.apache.org/schema/cache")
-  protected List<RegionAttributesType> regionAttributes;
+  protected RegionAttributesType regionAttributes;
   @XmlElement(name = "index", namespace = "http://geode.apache.org/schema/cache")
   protected List<RegionConfig.Index> indexes;
   @XmlElement(name = "entry", namespace = "http://geode.apache.org/schema/cache")
@@ -174,34 +171,12 @@ public class RegionConfig implements CacheElement {
     this.refid = refid;
   }
 
-  /**
-   * Gets the value of the regionAttributes property.
-   *
-   * <p>
-   * This accessor method returns a reference to the live list,
-   * not a snapshot. Therefore any modification you make to the
-   * returned list will be present inside the JAXB object.
-   * This is why there is not a <CODE>set</CODE> method for the regionAttributes property.
-   *
-   * <p>
-   * For example, to add a new item, do as follows:
-   *
-   * <pre>
-   * getRegionAttributes().add(newItem);
-   * </pre>
-   *
-   *
-   * <p>
-   * Objects of the following type(s) are allowed in the list
-   * {@link RegionAttributesType }
-   *
-   *
-   */
-  public List<RegionAttributesType> getRegionAttributes() {
-    if (regionAttributes == null) {
-      regionAttributes = new ArrayList<RegionAttributesType>();
-    }
-    return this.regionAttributes;
+  public RegionAttributesType getRegionAttributes() {
+    return regionAttributes;
+  }
+
+  public void setRegionAttributes(RegionAttributesType regionAttributes) {
+    this.regionAttributes = regionAttributes;
   }
 
   /**
@@ -229,7 +204,7 @@ public class RegionConfig implements CacheElement {
    */
   public List<RegionConfig.Index> getIndexes() {
     if (indexes == null) {
-      indexes = new ArrayList<RegionConfig.Index>();
+      indexes = new ArrayList<>();
     }
     return this.indexes;
   }
@@ -283,10 +258,6 @@ public class RegionConfig implements CacheElement {
    *
    * <p>
    * Objects of the following type(s) are allowed in the list
-   * {@link Element }
-   * {@link CacheElement }
-   *
-   *
    */
   public List<CacheElement> getCustomRegionElements() {
     if (regionElements == null) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionAttributeGetFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionAttributeGetFunction.java
deleted file mode 100644
index ebee2db..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionAttributeGetFunction.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.cli.domain;
-
-import org.apache.geode.cache.configuration.RegionAttributesType;
-
-@FunctionalInterface
-public interface RegionAttributeGetFunction {
-  Object getValue(RegionAttributesType attributesType);
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionAttributeSetFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionAttributeSetFunction.java
deleted file mode 100644
index 051e815..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionAttributeSetFunction.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.cli.domain;
-
-import org.apache.geode.cache.configuration.RegionAttributesType;
-
-@FunctionalInterface
-public interface RegionAttributeSetFunction {
-  void setAttributeValue(RegionAttributesType attributesType);
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionConfigFactory.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionConfigFactory.java
index 59a01a7..39df779 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionConfigFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/domain/RegionConfigFactory.java
@@ -14,14 +14,16 @@
  */
 package org.apache.geode.management.internal.cli.domain;
 
-import java.util.List;
 import java.util.Optional;
+import java.util.function.Consumer;
+import java.util.function.Function;
 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;
+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.management.internal.cli.functions.RegionFunctionArgs;
@@ -62,7 +64,7 @@ public class RegionConfigFactory {
     }
 
     if (args.getEntryIdleTimeCustomExpiry() != null) {
-      Object maybeEntryIdleAttr = getRegionAttributeValue(regionConfig, a -> a.getEntryIdleTime());
+      Object maybeEntryIdleAttr = getAttribute(regionConfig, a -> a.getEntryIdleTime());
       RegionAttributesType.EntryIdleTime entryIdleTime =
           maybeEntryIdleAttr != null ? (RegionAttributesType.EntryIdleTime) maybeEntryIdleAttr
               : new RegionAttributesType.EntryIdleTime();
@@ -134,7 +136,7 @@ public class RegionConfigFactory {
     }
 
     if (args.getEntryTTLCustomExpiry() != null) {
-      Object maybeEntryTTLAttr = getRegionAttributeValue(regionConfig, a -> a.getEntryTimeToLive());
+      Object maybeEntryTTLAttr = getAttribute(regionConfig, a -> a.getEntryTimeToLive());
       RegionAttributesType.EntryTimeToLive entryTimeToLive =
           maybeEntryTTLAttr != null ? (RegionAttributesType.EntryTimeToLive) maybeEntryTTLAttr
               : new RegionAttributesType.EntryTimeToLive();
@@ -295,6 +297,13 @@ public class RegionConfigFactory {
           a -> a.setDataPolicy(regionAttributes.getDataPolicy().toConfigType()));
     }
 
+    if (regionAttributes != null && regionAttributes.getScope() != null
+        && !regionAttributes.getDataPolicy().withPartitioning()) {
+      addAttribute(regionConfig,
+          a -> a.setScope(
+              RegionAttributesScope.fromValue(regionAttributes.getScope().toConfigTypeString())));
+    }
+
     return regionConfig;
   }
 
@@ -313,19 +322,20 @@ public class RegionConfigFactory {
     return regions[regions.length - 1];
   }
 
-  private void addAttribute(RegionConfig config, RegionAttributeSetFunction func) {
-    final List<RegionAttributesType> regionAttributes = config.getRegionAttributes();
-    if (regionAttributes.isEmpty()) {
-      regionAttributes.add(new RegionAttributesType());
+  private void addAttribute(RegionConfig config, Consumer<RegionAttributesType> consumer) {
+    if (config.getRegionAttributes() == null) {
+      config.setRegionAttributes(new RegionAttributesType());
     }
 
-    func.setAttributeValue(regionAttributes.get(0));
+    consumer.accept(config.getRegionAttributes());
   }
 
-  private Object getRegionAttributeValue(RegionConfig config, RegionAttributeGetFunction function) {
-    return config.getRegionAttributes().stream()
-        .findFirst()
-        .map(a -> function.getValue(a))
-        .orElse(null);
+  private Object getAttribute(RegionConfig config,
+      Function<RegionAttributesType, Object> function) {
+    if (config.getRegionAttributes() == null) {
+      return null;
+    }
+
+    return function.apply(config.getRegionAttributes());
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
index 353ee30..9dfa16d 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
@@ -37,7 +37,6 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.RegionExistsException;
 import org.apache.geode.cache.RegionFactory;
-import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.ResultSender;
 import org.apache.geode.cache.util.ObjectSizer;
@@ -45,14 +44,12 @@ import org.apache.geode.compression.Compressor;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.execute.InternalFunction;
-import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.commands.RegionCommandsUtils;
 import org.apache.geode.management.internal.cli.domain.ClassName;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.util.RegionPath;
-import org.apache.geode.management.internal.configuration.domain.XmlEntity;
 
 /**
  *
@@ -141,20 +138,9 @@ public class RegionCreateFunction implements InternalFunction {
     return new CliFunctionResult(memberNameOrId, CliFunctionResult.StatusState.ERROR);
   }
 
-  private XmlEntity getXmlEntityForRegion(Region<?, ?> region) {
-    Region<?, ?> curRegion = region;
-    while (curRegion != null && curRegion.getParentRegion() != null) {
-      curRegion = curRegion.getParentRegion();
-    }
-
-    return new XmlEntity(CacheXml.REGION, "name", curRegion.getName());
-  }
-
   private <K, V> Region<?, ?> createRegion(Cache cache, RegionFunctionArgs regionCreateArgs) {
     Region<K, V> createdRegion = null;
 
-    final RegionShortcut regionShortcut = regionCreateArgs.getRegionShortcut();
-
     // create the region factory using the arguments
     RegionAttributes<K, V> regionAttributes = regionCreateArgs.getRegionAttributes();
     boolean isPartitioned = regionAttributes.getPartitionAttributes() != null;
diff --git a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
index cf04eb9..2ed61b6 100644
--- a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
+++ b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
@@ -136,7 +136,7 @@ org/apache/geode/cache/configuration/RegionAttributesMirrorType,false,value:java
 org/apache/geode/cache/configuration/RegionAttributesScope,false,value:java/lang/String
 org/apache/geode/cache/configuration/RegionAttributesType$EvictionAttributes$LruHeapPercentage,false,action:org/apache/geode/cache/configuration/EnumActionDestroyOverflow
 org/apache/geode/cache/configuration/RegionAttributesType$EvictionAttributes$LruMemorySize,false,maximum:java/lang/String
-org/apache/geode/cache/configuration/RegionConfig,false,entries:java/util/List,indexes:java/util/List,name:java/lang/String,refid:java/lang/String,regionAttributes:java/util/List,regionElements:java/util/List,regions:java/util/List
+org/apache/geode/cache/configuration/RegionConfig,false,entries:java/util/List,indexes:java/util/List,name:java/lang/String,refid:java/lang/String,regionAttributes:org/apache/geode/cache/configuration/RegionAttributesType,regionElements:java/util/List,regions:java/util/List
 org/apache/geode/cache/configuration/RegionConfig$Index,false,expression:java/lang/String,fromClause:java/lang/String,imports:java/lang/String,keyIndex:java/lang/Boolean,name:java/lang/String,type:java/lang/String
 org/apache/geode/cache/execute/EmptyRegionFunctionException,true,1
 org/apache/geode/cache/execute/FunctionAdapter,true,-4891043890440825485
diff --git a/geode-core/src/test/java/org/apache/geode/cache/configuration/CacheConfigTest.java b/geode-core/src/test/java/org/apache/geode/cache/configuration/CacheConfigTest.java
index 056e544..b15a95d 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/configuration/CacheConfigTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/configuration/CacheConfigTest.java
@@ -166,4 +166,22 @@ public class CacheConfigTest {
     assertThat(regionAttributes.getRegionTimeToLive().getExpirationAttributes().getCustomExpiry())
         .isEqualTo(declarableWithString);
   }
+
+  @Test
+  public void regionConfig() {
+    cacheConfig = new CacheConfig("1.0");
+    RegionConfig regionConfig = new RegionConfig();
+    regionConfig.setName("test");
+    regionConfig.setRefid("REPLICATE");
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setCacheLoader(new DeclarableType("abc.Foo"));
+    regionConfig.setRegionAttributes(attributes);
+    cacheConfig.getRegions().add(regionConfig);
+
+    // make sure the xml marshed by this config can be validated with xsd
+    String xml = service.marshall(cacheConfig);
+
+    CacheConfig newCache = service.unMarshall(xml);
+    assertThat(cacheConfig).isEqualToComparingFieldByFieldRecursively(newCache);
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/domain/RegionConfigFactoryTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/domain/RegionConfigFactoryTest.java
index 31b5bd9..4eec7e5 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/domain/RegionConfigFactoryTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/domain/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.configuration.ClassNameType;
 import org.apache.geode.cache.configuration.DeclarableType;
 import org.apache.geode.cache.configuration.EnumActionDestroyOverflow;
 import org.apache.geode.cache.configuration.RegionAttributesType;
@@ -57,9 +56,9 @@ public class RegionConfigFactoryTest {
   }
 
   @Test
-  public void generatesWithNoAttributes() {
+  public void generatesNullWithNoAttributes() {
     RegionConfig config = subject.generate(args);
-    assertThat(config.getRegionAttributes()).isEmpty();
+    assertThat(config.getRegionAttributes()).isNull();
   }
 
   @Test
@@ -68,8 +67,8 @@ public class RegionConfigFactoryTest {
     args.setValueConstraint("value-const");
 
     RegionConfig config = subject.generate(args);
-    assertThat(getRegionAttributeValue(config, t -> t.getKeyConstraint())).isEqualTo("key-const");
-    assertThat(getRegionAttributeValue(config, t -> t.getValueConstraint()))
+    assertThat(config.getRegionAttributes().getKeyConstraint()).isEqualTo("key-const");
+    assertThat(config.getRegionAttributes().getValueConstraint())
         .isEqualTo("value-const");
   }
 
@@ -83,19 +82,16 @@ public class RegionConfigFactoryTest {
 
     RegionConfig config = subject.generate(args);
     RegionAttributesType.RegionTimeToLive regionTimeToLive =
-        (RegionAttributesType.RegionTimeToLive) getRegionAttributeValue(config,
-            t -> t.getRegionTimeToLive());
+        config.getRegionAttributes().getRegionTimeToLive();
     assertThat(regionTimeToLive.getExpirationAttributes().getTimeout()).isEqualTo("10");
 
     RegionAttributesType.EntryTimeToLive entryTimeToLive =
-        (RegionAttributesType.EntryTimeToLive) getRegionAttributeValue(config,
-            t -> t.getEntryTimeToLive());
+        config.getRegionAttributes().getEntryTimeToLive();
     assertThat(entryTimeToLive.getExpirationAttributes().getAction())
         .isEqualTo(ExpirationAction.LOCAL_DESTROY.toXmlString());
 
     RegionAttributesType.EntryIdleTime entryIdleTime =
-        (RegionAttributesType.EntryIdleTime) getRegionAttributeValue(config,
-            t -> t.getEntryIdleTime());
+        config.getRegionAttributes().getEntryIdleTime();
     DeclarableType customExpiry = entryIdleTime.getExpirationAttributes().getCustomExpiry();
     assertThat(customExpiry.getClassName()).isEqualTo("java.lang.String");
     assertThat(entryIdleTime.getExpirationAttributes().getAction())
@@ -110,8 +106,8 @@ public class RegionConfigFactoryTest {
     args.setDiskSynchronous(false);
 
     RegionConfig config = subject.generate(args);
-    assertThat(getRegionAttributeValue(config, t -> t.getDiskStoreName())).isEqualTo("disk-store");
-    assertThat(getRegionAttributeValue(config, t -> t.isDiskSynchronous())).isEqualTo(false);
+    assertThat(config.getRegionAttributes().getDiskStoreName()).isEqualTo("disk-store");
+    assertThat(config.getRegionAttributes().isDiskSynchronous()).isEqualTo(false);
   }
 
   @Test
@@ -122,8 +118,7 @@ public class RegionConfigFactoryTest {
 
     RegionConfig config = subject.generate(args);
     RegionAttributesType.PartitionAttributes partitionAttributes =
-        (RegionAttributesType.PartitionAttributes) getRegionAttributeValue(config,
-            t -> t.getPartitionAttributes());
+        config.getRegionAttributes().getPartitionAttributes();
     assertThat(partitionAttributes).isNotNull();
     assertThat(partitionAttributes.getColocatedWith()).isEqualTo("colo-with");
     assertThat(partitionAttributes.getLocalMaxMemory()).isEqualTo("100");
@@ -148,17 +143,17 @@ public class RegionConfigFactoryTest {
     args.setOffHeap(true);
     RegionConfig config = subject.generate(args);
 
-    assertThat(getRegionAttributeValue(config, t -> t.isStatisticsEnabled())).isEqualTo(false);
-    assertThat(getRegionAttributeValue(config, t -> t.isEnableSubscriptionConflation()))
+    assertThat(config.getRegionAttributes().isStatisticsEnabled()).isEqualTo(false);
+    assertThat(config.getRegionAttributes().isEnableSubscriptionConflation())
         .isEqualTo(true);
-    assertThat(getRegionAttributeValue(config, t -> t.isConcurrencyChecksEnabled()))
+    assertThat(config.getRegionAttributes().isConcurrencyChecksEnabled())
         .isEqualTo(true);
-    assertThat(getRegionAttributeValue(config, t -> t.isEnableSubscriptionConflation()))
+    assertThat(config.getRegionAttributes().isEnableSubscriptionConflation())
         .isEqualTo(true);
-    assertThat(getRegionAttributeValue(config, t -> t.isMulticastEnabled()))
+    assertThat(config.getRegionAttributes().isMulticastEnabled())
         .isEqualTo(false);
-    assertThat(getRegionAttributeValue(config, t -> t.isCloningEnabled())).isEqualTo(false);
-    assertThat(getRegionAttributeValue(config, t -> t.isOffHeap())).isEqualTo(true);
+    assertThat(config.getRegionAttributes().isCloningEnabled()).isEqualTo(false);
+    assertThat(config.getRegionAttributes().isOffHeap()).isEqualTo(true);
   }
 
   @Test
@@ -166,9 +161,9 @@ public class RegionConfigFactoryTest {
     args.setGatewaySenderIds(new String[] {"some-id", "some-other-id"});
     RegionConfig config = subject.generate(args);
 
-    assertThat((String) getRegionAttributeValue(config, t -> t.getGatewaySenderIds()))
+    assertThat(config.getRegionAttributes().getGatewaySenderIds())
         .contains("some-id");
-    assertThat((String) getRegionAttributeValue(config, t -> t.getGatewaySenderIds()))
+    assertThat(config.getRegionAttributes().getGatewaySenderIds())
         .contains("some-other-id");
   }
 
@@ -179,8 +174,7 @@ public class RegionConfigFactoryTest {
     RegionConfig config = subject.generate(args);
 
     RegionAttributesType.EvictionAttributes evictionAttributes =
-        (RegionAttributesType.EvictionAttributes) getRegionAttributeValue(config,
-            t -> t.getEvictionAttributes());
+        config.getRegionAttributes().getEvictionAttributes();
     assertThat(evictionAttributes).isNotNull();
     assertThat(evictionAttributes.getLruHeapPercentage().getAction())
         .isSameAs(EnumActionDestroyOverflow.LOCAL_DESTROY);
@@ -195,8 +189,7 @@ public class RegionConfigFactoryTest {
     RegionConfig config = subject.generate(args);
 
     RegionAttributesType.EvictionAttributes evictionAttributes =
-        (RegionAttributesType.EvictionAttributes) getRegionAttributeValue(config,
-            t -> t.getEvictionAttributes());
+        config.getRegionAttributes().getEvictionAttributes();
     assertThat(evictionAttributes).isNotNull();
     assertThat(evictionAttributes.getLruMemorySize().getAction())
         .isSameAs(EnumActionDestroyOverflow.LOCAL_DESTROY);
@@ -209,8 +202,7 @@ public class RegionConfigFactoryTest {
         null);
     RegionConfig config = subject.generate(args);
     RegionAttributesType.EvictionAttributes evictionAttributes =
-        (RegionAttributesType.EvictionAttributes) getRegionAttributeValue(config,
-            t -> t.getEvictionAttributes());
+        config.getRegionAttributes().getEvictionAttributes();
     assertThat(evictionAttributes).isNotNull();
     assertThat(evictionAttributes.getLruEntryCount().getAction())
         .isSameAs(EnumActionDestroyOverflow.OVERFLOW_TO_DISK);
@@ -222,9 +214,9 @@ public class RegionConfigFactoryTest {
     args.setAsyncEventQueueIds(new String[] {"id-1", "id-2"});
     RegionConfig config = subject.generate(args);
 
-    assertThat((String) getRegionAttributeValue(config, t -> t.getAsyncEventQueueIds()))
+    assertThat(config.getRegionAttributes().getAsyncEventQueueIds())
         .contains("id-1");
-    assertThat((String) getRegionAttributeValue(config, t -> t.getAsyncEventQueueIds()))
+    assertThat(config.getRegionAttributes().getAsyncEventQueueIds())
         .contains("id-2");
   }
 
@@ -235,19 +227,15 @@ public class RegionConfigFactoryTest {
     args.setCacheWriter(new ClassName("java.lang.String"));
     RegionConfig config = subject.generate(args);
 
-    List<DeclarableType> cacheListeners = config.getRegionAttributes().stream()
-        .filter(a -> !a.getCacheListeners().isEmpty())
-        .findFirst()
-        .map(a -> a.getCacheListeners())
-        .orElse(null);
+    List<DeclarableType> cacheListeners = config.getRegionAttributes().getCacheListeners();
 
     assertThat(cacheListeners).isNotNull();
     assertThat(cacheListeners.get(0).getClassName()).isEqualTo("java.lang.String");
     assertThat(
-        ((DeclarableType) getRegionAttributeValue(config, t -> t.getCacheLoader())).getClassName())
+        config.getRegionAttributes().getCacheLoader().getClassName())
             .isEqualTo("java.lang.String");
     assertThat(
-        ((DeclarableType) getRegionAttributeValue(config, t -> t.getCacheWriter())).getClassName())
+        config.getRegionAttributes().getCacheWriter().getClassName())
             .isEqualTo("java.lang.String");
   }
 
@@ -259,15 +247,9 @@ public class RegionConfigFactoryTest {
     RegionConfig config = subject.generate(args);
 
     assertThat(
-        ((ClassNameType) getRegionAttributeValue(config, t -> t.getCompressor())).getClassName())
+        config.getRegionAttributes().getCompressor().getClassName())
             .isEqualTo("java.lang.String");
-    assertThat(getRegionAttributeValue(config, t -> t.getConcurrencyLevel())).isEqualTo("1");
+    assertThat(config.getRegionAttributes().getConcurrencyLevel()).isEqualTo("1");
   }
 
-  private Object getRegionAttributeValue(RegionConfig config, RegionAttributeGetFunction function) {
-    return config.getRegionAttributes().stream()
-        .findFirst()
-        .map(a -> function.getValue(a))
-        .orElse(null);
-  }
 }