You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by sa...@apache.org on 2018/04/26 22:29:51 UTC

[geode] branch develop updated: GEODE-4794: ConfigurePDXCommand Fails When Using Defaults (#1852)

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

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


The following commit(s) were added to refs/heads/develop by this push:
     new ee06900  GEODE-4794: ConfigurePDXCommand Fails When Using Defaults (#1852)
ee06900 is described below

commit ee0690097577ac9dbe241b0bb94ca6e8e4ed89fb
Author: Sai Boorlagadda <sa...@gmail.com>
AuthorDate: Thu Apr 26 15:29:47 2018 -0700

    GEODE-4794: ConfigurePDXCommand Fails When Using Defaults (#1852)
---
 .../geode/cache/configuration/StringType.java      |   6 +
 .../internal/cli/commands/ConfigurePDXCommand.java | 131 ++++++-------
 .../management/internal/cli/i18n/CliStrings.java   |   4 +-
 .../ConfigurePDXCommandIntegrationTest.java        |  17 +-
 .../cli/commands/ConfigurePDXCommandTest.java      | 213 ++++-----------------
 5 files changed, 106 insertions(+), 265 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/StringType.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/StringType.java
index 4fa5e19..f95dd0e 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/configuration/StringType.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/StringType.java
@@ -57,6 +57,12 @@ public class StringType {
   @XmlValue
   protected String value;
 
+  public StringType() {}
+
+  public StringType(String value) {
+    this.value = value;
+  }
+
   /**
    * Gets the value of the value property.
    *
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommand.java
index 4541789..ffa2aa4 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommand.java
@@ -15,18 +15,17 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
-import java.io.PrintWriter;
-import java.io.StringWriter;
 import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
 
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
+import org.apache.geode.cache.configuration.ParameterType;
+import org.apache.geode.cache.configuration.PdxType;
+import org.apache.geode.cache.configuration.StringType;
 import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
-import org.apache.geode.internal.cache.CacheConfig;
-import org.apache.geode.internal.cache.xmlcache.CacheCreation;
-import org.apache.geode.internal.cache.xmlcache.CacheXml;
-import org.apache.geode.internal.cache.xmlcache.CacheXmlGenerator;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.AbstractCliAroundInterceptor;
@@ -34,7 +33,6 @@ import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.InfoResultData;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.configuration.domain.XmlEntity;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.pdx.ReflectionBasedAutoSerializer;
 import org.apache.geode.security.ResourcePermission;
@@ -46,28 +44,6 @@ public class ConfigurePDXCommand extends InternalGfshCommand {
     return new ReflectionBasedAutoSerializer(checkPortability, patterns);
   }
 
-  /**
-   * @param forParsing if true then this creation is used for parsing xml; if false then it is used
-   *        for generating xml.
-   * @since GemFire 5.7
-   */
-  protected CacheCreation getCacheCreation(boolean forParsing) {
-    return new CacheCreation(forParsing);
-  }
-
-  /**
-   * Creates the XmlEntity associated to the PDX configuration.
-   */
-  protected XmlEntity createXmlEntity(CacheCreation cache) {
-    final StringWriter stringWriter = new StringWriter();
-    final PrintWriter printWriter = new PrintWriter(stringWriter);
-    CacheXmlGenerator.generate(cache, printWriter, true, false, false);
-    printWriter.close();
-    String xmlDefinition = stringWriter.toString();
-
-    return XmlEntity.builder().withType(CacheXml.PDX).withConfig(xmlDefinition).build();
-  }
-
   @CliCommand(value = CliStrings.CONFIGURE_PDX, help = CliStrings.CONFIGURE_PDX__HELP)
   @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION,
       interceptor = "org.apache.geode.management.internal.cli.commands.ConfigurePDXCommand$Interceptor")
@@ -75,10 +51,12 @@ public class ConfigurePDXCommand extends InternalGfshCommand {
       operation = ResourcePermission.Operation.MANAGE)
   public Result configurePDX(
       @CliOption(key = CliStrings.CONFIGURE_PDX__READ__SERIALIZED,
+          unspecifiedDefaultValue = "false",
           help = CliStrings.CONFIGURE_PDX__READ__SERIALIZED__HELP) Boolean readSerialized,
       @CliOption(key = CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS,
+          unspecifiedDefaultValue = "false",
           help = CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS__HELP) Boolean ignoreUnreadFields,
-      @CliOption(key = CliStrings.CONFIGURE_PDX__DISKSTORE, specifiedDefaultValue = "",
+      @CliOption(key = CliStrings.CONFIGURE_PDX__DISKSTORE, specifiedDefaultValue = "DEFAULT",
           help = CliStrings.CONFIGURE_PDX__DISKSTORE__HELP) String diskStore,
       @CliOption(key = CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES,
           help = CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES__HELP) String[] nonPortableClassesPatterns,
@@ -86,69 +64,74 @@ public class ConfigurePDXCommand extends InternalGfshCommand {
           help = CliStrings.CONFIGURE_PDX__PORTABLE__AUTO__SERIALIZER__CLASSES__HELP) String[] portableClassesPatterns) {
 
     Result result;
-    ReflectionBasedAutoSerializer autoSerializer;
-    CacheCreation cache = getCacheCreation(true);
-    InfoResultData ird = ResultBuilder.createInfoResultData();
+    ReflectionBasedAutoSerializer autoSerializer = null;
 
-    if (!getAllNormalMembers().isEmpty()) {
-      ird.addLine(CliStrings.CONFIGURE_PDX__NORMAL__MEMBERS__WARNING);
+    if (getConfigurationService() == null) {
+      return ResultBuilder
+          .createUserErrorResult("Configure pdx failed because cluster configuration is disabled.");
     }
 
-    // Set persistent and the disk-store
-    if (diskStore != null) {
-      cache.setPdxPersistent(true);
-      ird.addLine(CliStrings.CONFIGURE_PDX__PERSISTENT + " = " + cache.getPdxPersistent());
-
-      if (!diskStore.equals("")) {
-        cache.setPdxDiskStore(diskStore);
-        ird.addLine(CliStrings.CONFIGURE_PDX__DISKSTORE + " = " + cache.getPdxDiskStore());
-      } else {
-        ird.addLine(CliStrings.CONFIGURE_PDX__DISKSTORE + " = " + "DEFAULT");
-      }
-    } else {
-      cache.setPdxPersistent(CacheConfig.DEFAULT_PDX_PERSISTENT);
-      ird.addLine(CliStrings.CONFIGURE_PDX__PERSISTENT + " = " + cache.getPdxPersistent());
-    }
+    InfoResultData infoResultData = ResultBuilder.createInfoResultData();
 
-    // Set read-serialized
-    if (readSerialized != null) {
-      cache.setPdxReadSerialized(readSerialized);
-    } else {
-      cache.setPdxReadSerialized(CacheConfig.DEFAULT_PDX_READ_SERIALIZED);
+    if (!getAllNormalMembers().isEmpty()) {
+      infoResultData.addLine(CliStrings.CONFIGURE_PDX__NORMAL__MEMBERS__WARNING);
     }
 
-    ird.addLine(CliStrings.CONFIGURE_PDX__READ__SERIALIZED + " = " + cache.getPdxReadSerialized());
-
-    // Set ignoreUnreadFields
-    if (ignoreUnreadFields != null) {
-      cache.setPdxIgnoreUnreadFields(ignoreUnreadFields);
+    // Set persistent and the disk-store
+    if (diskStore != null) {
+      infoResultData.addLine(CliStrings.CONFIGURE_PDX__PERSISTENT + " = true");
+      infoResultData.addLine(CliStrings.CONFIGURE_PDX__DISKSTORE + " = " + diskStore);
     } else {
-      cache.setPdxIgnoreUnreadFields(CacheConfig.DEFAULT_PDX_IGNORE_UNREAD_FIELDS);
+      infoResultData.addLine(CliStrings.CONFIGURE_PDX__PERSISTENT + " = false");
     }
 
-    ird.addLine(
-        CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS + " = " + cache.getPdxIgnoreUnreadFields());
+    infoResultData.addLine(CliStrings.CONFIGURE_PDX__READ__SERIALIZED + " = " + readSerialized);
+    infoResultData
+        .addLine(CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS + " = " + ignoreUnreadFields);
 
     // Auto Serializer Configuration
     if (portableClassesPatterns != null) {
       autoSerializer = createReflectionBasedAutoSerializer(true, portableClassesPatterns);
-      cache.setPdxSerializer(autoSerializer);
-      ird.addLine("PDX Serializer = " + cache.getPdxSerializer().getClass().getName());
-      ird.addLine("Portable Classes = " + Arrays.toString(portableClassesPatterns));
+      infoResultData.addLine("PDX Serializer = " + autoSerializer.getClass().getName());
+      infoResultData.addLine("Portable Classes = " + Arrays.toString(portableClassesPatterns));
     }
 
     if (nonPortableClassesPatterns != null) {
       autoSerializer = createReflectionBasedAutoSerializer(false, nonPortableClassesPatterns);
-      cache.setPdxSerializer(autoSerializer);
-      ird.addLine("PDX Serializer = " + cache.getPdxSerializer().getClass().getName());
-      ird.addLine("Non Portable Classes = " + Arrays.toString(nonPortableClassesPatterns));
+      infoResultData.addLine("PDX Serializer = " + autoSerializer.getClass().getName());
+      infoResultData
+          .addLine("Non Portable Classes = " + Arrays.toString(nonPortableClassesPatterns));
     }
 
-    XmlEntity xmlEntity = createXmlEntity(cache);
-    result = ResultBuilder.buildResult(ird);
-    persistClusterConfiguration(result,
-        () -> ((InternalClusterConfigurationService) getConfigurationService())
-            .addXmlEntity(xmlEntity, null));
+    result = ResultBuilder.buildResult(infoResultData);
+    ReflectionBasedAutoSerializer finalAutoSerializer = autoSerializer;
+    getConfigurationService().updateCacheConfig(InternalClusterConfigurationService.CLUSTER_CONFIG,
+        config -> {
+          if (config.getPdx() == null) {
+            config.setPdx(new PdxType());
+          }
+          config.getPdx().setReadSerialized(readSerialized);
+          config.getPdx().setIgnoreUnreadFields(ignoreUnreadFields);
+          config.getPdx().setDiskStoreName(diskStore);
+          config.getPdx().setPersistent(diskStore != null);
+
+          if (portableClassesPatterns != null || nonPortableClassesPatterns != null) {
+            PdxType.PdxSerializer pdxSerializer = new PdxType.PdxSerializer();
+            pdxSerializer.setClassName(ReflectionBasedAutoSerializer.class.getName());
+
+            List<ParameterType> parameters =
+                finalAutoSerializer.getConfig().entrySet().stream().map(entry -> {
+                  ParameterType parameterType = new ParameterType();
+                  parameterType.setName((String) entry.getKey());
+                  parameterType.setString(new StringType((String) entry.getValue()));
+                  return parameterType;
+                }).collect(Collectors.toList());
+            pdxSerializer.getParameter().addAll(parameters);
+
+            config.getPdx().setPdxSerializer(pdxSerializer);
+          }
+          return config;
+        });
     return result;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index 0ed0e8e..0a2bba2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -3098,7 +3098,7 @@ public class CliStrings {
       "Configures Geode's Portable Data eXchange for all the cache(s) in the cluster. This command would not take effect on the running members in the system.\n This command persists the pdx configuration in the locator with cluster configuration service. \n This command should be issued before starting any data members.";
   public static final String CONFIGURE_PDX__READ__SERIALIZED = "read-serialized";
   public static final String CONFIGURE_PDX__READ__SERIALIZED__HELP =
-      "Set to true to have PDX deserialization produce a PdxInstance instead of an instance of the domain class";
+      "Set to true to have PDX deserialization produce a PdxInstance instead of an instance of the domain class. The default value for this options is \"false\"";
   public static final String CONFIGURE_PDX__IGNORE__UNREAD_FIELDS = "ignore-unread-fields";
   public static final String CONFIGURE_PDX__IGNORE__UNREAD_FIELDS__HELP =
       "Control whether pdx ignores fields that were unread during deserialization. The default is to preserve unread fields be including their data during serialization. But if you configure the cache to ignore unread fields then their data will be lost during serialization."
@@ -3108,7 +3108,7 @@ public class CliStrings {
       "Control whether the type metadata for PDX objects is persisted to disk. The default for this setting is false. If you are using persistent regions with PDX then you must set this to true. If you are using a GatewaySender or AsyncEventQueue with PDX then you should set this to true";
   public static final String CONFIGURE_PDX__DISKSTORE = "disk-store";
   public static final String CONFIGURE_PDX__DISKSTORE__HELP =
-      "Named disk store where the PDX type data will be stored";
+      "Named disk store where the PDX type data will be stored. The default value for this options is \"DEFAULT\"";
 
   public static final String CONFIGURE_PDX__CHECK__PORTABILITY = "check-portability";
   public static final String CONFIGURE_PDX__CHECK__PORTABILITY__HELP =
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandIntegrationTest.java
index 0b9389c..2319e6f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandIntegrationTest.java
@@ -17,7 +17,6 @@ package org.apache.geode.management.internal.cli.commands;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -35,7 +34,8 @@ public class ConfigurePDXCommandIntegrationTest {
   public GfshCommandRule gfsh = new GfshCommandRule().withTimeout(1);
 
   @Rule
-  public LocatorStarterRule locator = new LocatorStarterRule().withAutoStart().withJMXManager();
+  public LocatorStarterRule locator =
+      new LocatorStarterRule().withWorkingDir().withAutoStart().withJMXManager();;
 
   @Before
   public void before() throws Exception {
@@ -50,14 +50,13 @@ public class ConfigurePDXCommandIntegrationTest {
   }
 
   @Test
-  @Ignore("See https://issues.apache.org/jira/browse/GEODE-4794")
   public void commandShouldSucceedWhenUsingDefaults() {
     gfsh.executeAndAssertThat(BASE_COMMAND_STRING).statusIsSuccess().hasNoFailToPersistError();
 
     String sharedConfigXml = locator.getLocator().getSharedConfiguration()
         .getConfiguration("cluster").getCacheXmlContent();
     assertThat(sharedConfigXml).contains(
-        "<pdx ignore-unread-fields=\"false\" persistent=\"false\" read-serialized=\"false\"></pdx>");
+        "<pdx read-serialized=\"false\" ignore-unread-fields=\"false\" persistent=\"false\"/>");
   }
 
   @Test
@@ -69,7 +68,7 @@ public class ConfigurePDXCommandIntegrationTest {
     String sharedConfigXml = locator.getLocator().getSharedConfiguration()
         .getConfiguration("cluster").getCacheXmlContent();
     assertThat(sharedConfigXml).contains(
-        "<pdx disk-store-name=\"myDiskStore\" ignore-unread-fields=\"true\" persistent=\"true\" read-serialized=\"true\">");
+        "<pdx read-serialized=\"true\" ignore-unread-fields=\"true\" persistent=\"true\" disk-store-name=\"myDiskStore\">");
     assertThat(sharedConfigXml).contains("<pdx-serializer>",
         "<class-name>org.apache.geode.pdx.ReflectionBasedAutoSerializer</class-name>",
         "<parameter name=\"classes\">",
@@ -86,7 +85,8 @@ public class ConfigurePDXCommandIntegrationTest {
 
     String sharedConfigXml = locator.getLocator().getSharedConfiguration()
         .getConfiguration("cluster").getCacheXmlContent();
-    assertThat(sharedConfigXml).contains("<pdx>");
+    assertThat(sharedConfigXml).contains(
+        "<pdx read-serialized=\"false\" ignore-unread-fields=\"false\" persistent=\"false\">");
     assertThat(sharedConfigXml).contains("<pdx-serializer>",
         "<class-name>org.apache.geode.pdx.ReflectionBasedAutoSerializer</class-name>",
         "<parameter name=\"classes\">",
@@ -104,7 +104,7 @@ public class ConfigurePDXCommandIntegrationTest {
     String sharedConfigXml = locator.getLocator().getSharedConfiguration()
         .getConfiguration("cluster").getCacheXmlContent();
     assertThat(sharedConfigXml).contains(
-        "<pdx disk-store-name=\"myDiskStore\" ignore-unread-fields=\"true\" persistent=\"true\" read-serialized=\"true\">");
+        "<pdx read-serialized=\"true\" ignore-unread-fields=\"true\" persistent=\"true\" disk-store-name=\"myDiskStore\">");
     assertThat(sharedConfigXml).contains("<parameter name=\"check-portability\">")
         .contains("<string>true</string>").contains("</parameter>");
     assertThat(sharedConfigXml).contains("<pdx-serializer>",
@@ -123,7 +123,8 @@ public class ConfigurePDXCommandIntegrationTest {
 
     String sharedConfigXml = locator.getLocator().getSharedConfiguration()
         .getConfiguration("cluster").getCacheXmlContent();
-    assertThat(sharedConfigXml).contains("<pdx>");
+    assertThat(sharedConfigXml).contains(
+        "<pdx read-serialized=\"false\" ignore-unread-fields=\"false\" persistent=\"false\">");
     assertThat(sharedConfigXml).contains("<parameter name=\"check-portability\">")
         .contains("<string>true</string>").contains("</parameter>");
     assertThat(sharedConfigXml).contains("<pdx-serializer>",
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandTest.java
index 85a73f5..dcb8efb 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigurePDXCommandTest.java
@@ -14,9 +14,6 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
-import static org.apache.geode.internal.cache.CacheConfig.DEFAULT_PDX_IGNORE_UNREAD_FIELDS;
-import static org.apache.geode.internal.cache.CacheConfig.DEFAULT_PDX_PERSISTENT;
-import static org.apache.geode.internal.cache.CacheConfig.DEFAULT_PDX_READ_SERIALIZED;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
@@ -32,20 +29,14 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.commons.lang.StringUtils;
 import org.junit.Before;
-import org.junit.ClassRule;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
-import org.apache.geode.internal.cache.CacheConfig;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.xmlcache.CacheCreation;
-import org.apache.geode.management.internal.cli.result.CommandResult;
-import org.apache.geode.management.internal.configuration.domain.XmlEntity;
-import org.apache.geode.test.junit.assertions.CommandResultAssert;
 import org.apache.geode.test.junit.categories.UnitTest;
 import org.apache.geode.test.junit.rules.GfshParserRule;
 
@@ -53,31 +44,32 @@ import org.apache.geode.test.junit.rules.GfshParserRule;
 public class ConfigurePDXCommandTest {
   private static final String BASE_COMMAND_STRING = "configure pdx ";
 
-  @ClassRule
-  public static CustomGfshParserRule gfshParserRule = new CustomGfshParserRule();
-
   private InternalCache cache;
-  private XmlEntity xmlEntity;
-  private CacheCreation cacheCreation;
   private ConfigurePDXCommand command;
   private InternalClusterConfigurationService clusterConfigurationService;
 
+  @Rule
+  public GfshParserRule gfshParserRule = new GfshParserRule();
+
   @Before
   public void setUp() throws Exception {
     cache = mock(InternalCache.class);
-    xmlEntity = mock(XmlEntity.class);
     command = spy(ConfigurePDXCommand.class);
-    cacheCreation = spy(CacheCreation.class);
     clusterConfigurationService = mock(InternalClusterConfigurationService.class);
 
     doReturn(cache).when(command).getCache();
-    doReturn(xmlEntity).when(command).createXmlEntity(any());
-    doReturn(cacheCreation).when(command).getCacheCreation(anyBoolean());
     doReturn(Collections.emptySet()).when(command).getAllNormalMembers();
     doReturn(clusterConfigurationService).when(command).getConfigurationService();
   }
 
   @Test
+  public void errorOutIfCCNotRunning() {
+    doReturn(null).when(command).getConfigurationService();
+    gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING).statusIsError()
+        .containsOutput("Configure pdx failed because cluster configuration is disabled.");
+  }
+
+  @Test
   public void parsingShouldSucceedWithoutArguments() {
     assertThat(gfshParserRule.parse(BASE_COMMAND_STRING)).isNotNull();
   }
@@ -93,25 +85,8 @@ public class ConfigurePDXCommandTest {
 
   @Test
   public void executionShouldHandleInternalFailures() {
-    doThrow(new RuntimeException("Can't create CacheCreation.")).when(command)
-        .getCacheCreation(anyBoolean());
-    gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING).statusIsError()
-        .containsOutput("Could not process command due to error.")
-        .containsOutput("Can't create CacheCreation.");
-    doReturn(cacheCreation).when(command).getCacheCreation(anyBoolean());
-
-    doThrow(new RuntimeException("Can't find members.")).when(command).getAllNormalMembers();
-    gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING).statusIsError()
-        .containsOutput("Could not process command due to error.")
-        .containsOutput("Can't find members.");
     doReturn(Collections.emptySet()).when(command).getAllNormalMembers();
 
-    doThrow(new RuntimeException("Can't create XmlEntity.")).when(command).createXmlEntity(any());
-    gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING).statusIsError()
-        .containsOutput("Could not process command due to error.")
-        .containsOutput("Can't create XmlEntity.");
-    doReturn(xmlEntity).when(command).createXmlEntity(any());
-
     doThrow(new RuntimeException("Can't create ReflectionBasedAutoSerializer.")).when(command)
         .createReflectionBasedAutoSerializer(anyBoolean(), any());
     gfshParserRule
@@ -146,32 +121,28 @@ public class ConfigurePDXCommandTest {
     DistributedMember mockMember = mock(DistributedMember.class);
     when(mockMember.getId()).thenReturn("member0");
     members.add(mockMember);
-    doReturn(xmlEntity).when(command).createXmlEntity(any());
     doReturn(members).when(command).getAllNormalMembers();
 
     gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING).statusIsSuccess()
-        .hasDefaultsConfigured(command, cacheCreation).containsOutput(
+        .containsOutput(
             "The command would only take effect on new data members joining the distributed system. It won't affect the existing data members");
   }
 
   @Test
   public void executionShouldWorkCorrectlyWhenDefaultsAreUsed() {
-    // Factory Default
     gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING).statusIsSuccess()
-        .hasNoFailToPersistError().hasDefaultsConfigured(command, cacheCreation);
+        .hasNoFailToPersistError().containsOutput("persistent = false")
+        .containsOutput("read-serialized = false").containsOutput("ignore-unread-fields = false");
   }
 
   @Test
   public void executionShouldCorrectlyConfigurePersistenceWhenDefaultDiskStoreIsUsed() {
     // Default Disk Store
     gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING + "--disk-store")
-        .statusIsSuccess().hasNoFailToPersistError()
-        .hasReadSerializedConfigured(DEFAULT_PDX_READ_SERIALIZED, cacheCreation)
-        .hasIgnoreUnreadFieldsConfigured(DEFAULT_PDX_IGNORE_UNREAD_FIELDS, cacheCreation)
-        .hasPersistenceConfigured(true, "DEFAULT", cacheCreation);
+        .statusIsSuccess().hasNoFailToPersistError().containsOutput("persistent = true")
+        .containsOutput("disk-store = DEFAULT").containsOutput("read-serialized = false")
+        .containsOutput("ignore-unread-fields = false");
 
-    verify(cacheCreation, times(0)).setPdxSerializer(any());
-    verify(command, times(1)).persistClusterConfiguration(any(), any());
     verify(command, times(0)).createReflectionBasedAutoSerializer(anyBoolean(), any());
   }
 
@@ -179,13 +150,10 @@ public class ConfigurePDXCommandTest {
   public void executionShouldCorrectlyConfigurePersistenceWhenCustomDiskStoreIsUsed() {
     // Custom Disk Store
     gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING + "--disk-store=myDiskStore")
-        .statusIsSuccess().hasNoFailToPersistError()
-        .hasReadSerializedConfigured(DEFAULT_PDX_READ_SERIALIZED, cacheCreation)
-        .hasIgnoreUnreadFieldsConfigured(DEFAULT_PDX_IGNORE_UNREAD_FIELDS, cacheCreation)
-        .hasPersistenceConfigured(true, "myDiskStore", cacheCreation);
+        .statusIsSuccess().hasNoFailToPersistError().containsOutput("persistent = true")
+        .containsOutput("disk-store = myDiskStore").containsOutput("read-serialized = false")
+        .containsOutput("ignore-unread-fields = false");
 
-    verify(cacheCreation, times(0)).setPdxSerializer(any());
-    verify(command, times(1)).persistClusterConfiguration(any(), any());
     verify(command, times(0)).createReflectionBasedAutoSerializer(anyBoolean(), any());
   }
 
@@ -193,13 +161,9 @@ public class ConfigurePDXCommandTest {
   public void executionShouldCorrectlyConfigureReadSerializedWhenFlagIsSetAsTrue() {
     // Custom Configuration as True
     gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING + "--read-serialized=true")
-        .statusIsSuccess().hasNoFailToPersistError()
-        .hasReadSerializedConfigured(true, cacheCreation)
-        .hasIgnoreUnreadFieldsConfigured(DEFAULT_PDX_IGNORE_UNREAD_FIELDS, cacheCreation)
-        .hasPersistenceConfigured(DEFAULT_PDX_PERSISTENT, null, cacheCreation);
+        .statusIsSuccess().hasNoFailToPersistError().containsOutput("persistent = false")
+        .containsOutput("read-serialized = true").containsOutput("ignore-unread-fields = false");
 
-    verify(cacheCreation, times(0)).setPdxSerializer(any());
-    verify(command, times(1)).persistClusterConfiguration(any(), any());
     verify(command, times(0)).createReflectionBasedAutoSerializer(anyBoolean(), any());
   }
 
@@ -207,13 +171,9 @@ public class ConfigurePDXCommandTest {
   public void executionShouldCorrectlyConfigureReadSerializedWhenFlagIsSetAsFalse() {
     // Custom Configuration as False
     gfshParserRule.executeAndAssertThat(command, BASE_COMMAND_STRING + "--read-serialized=false")
-        .statusIsSuccess().hasNoFailToPersistError()
-        .hasReadSerializedConfigured(false, cacheCreation)
-        .hasIgnoreUnreadFieldsConfigured(DEFAULT_PDX_IGNORE_UNREAD_FIELDS, cacheCreation)
-        .hasPersistenceConfigured(DEFAULT_PDX_PERSISTENT, null, cacheCreation);
+        .statusIsSuccess().hasNoFailToPersistError().containsOutput("persistent = false")
+        .containsOutput("read-serialized = false").containsOutput("ignore-unread-fields = false");
 
-    verify(cacheCreation, times(0)).setPdxSerializer(any());
-    verify(command, times(1)).persistClusterConfiguration(any(), any());
     verify(command, times(0)).createReflectionBasedAutoSerializer(anyBoolean(), any());
   }
 
@@ -222,13 +182,9 @@ public class ConfigurePDXCommandTest {
     // Custom Configuration as True
     gfshParserRule
         .executeAndAssertThat(command, BASE_COMMAND_STRING + "--ignore-unread-fields=true")
-        .statusIsSuccess().hasNoFailToPersistError()
-        .hasReadSerializedConfigured(DEFAULT_PDX_READ_SERIALIZED, cacheCreation)
-        .hasIgnoreUnreadFieldsConfigured(true, cacheCreation)
-        .hasPersistenceConfigured(DEFAULT_PDX_PERSISTENT, null, cacheCreation);
+        .statusIsSuccess().hasNoFailToPersistError().containsOutput("persistent = false")
+        .containsOutput("read-serialized = false").containsOutput("ignore-unread-fields = true");
 
-    verify(cacheCreation, times(0)).setPdxSerializer(any());
-    verify(command, times(1)).persistClusterConfiguration(any(), any());
     verify(command, times(0)).createReflectionBasedAutoSerializer(anyBoolean(), any());
   }
 
@@ -237,13 +193,9 @@ public class ConfigurePDXCommandTest {
     // Custom Configuration as False
     gfshParserRule
         .executeAndAssertThat(command, BASE_COMMAND_STRING + "--ignore-unread-fields=false")
-        .statusIsSuccess().hasNoFailToPersistError()
-        .hasReadSerializedConfigured(DEFAULT_PDX_READ_SERIALIZED, cacheCreation)
-        .hasIgnoreUnreadFieldsConfigured(false, cacheCreation)
-        .hasPersistenceConfigured(DEFAULT_PDX_PERSISTENT, null, cacheCreation);
+        .statusIsSuccess().hasNoFailToPersistError().containsOutput("persistent = false")
+        .containsOutput("read-serialized = false").containsOutput("ignore-unread-fields = false");
 
-    verify(cacheCreation, times(0)).setPdxSerializer(any());
-    verify(command, times(1)).persistClusterConfiguration(any(), any());
     verify(command, times(0)).createReflectionBasedAutoSerializer(anyBoolean(), any());
   }
 
@@ -255,15 +207,11 @@ public class ConfigurePDXCommandTest {
     gfshParserRule
         .executeAndAssertThat(command,
             BASE_COMMAND_STRING + "--portable-auto-serializable-classes=" + patterns[0])
-        .statusIsSuccess().hasNoFailToPersistError()
-        .hasReadSerializedConfigured(DEFAULT_PDX_READ_SERIALIZED, cacheCreation)
-        .hasIgnoreUnreadFieldsConfigured(DEFAULT_PDX_IGNORE_UNREAD_FIELDS, cacheCreation)
-        .hasPersistenceConfigured(DEFAULT_PDX_PERSISTENT, null, cacheCreation)
+        .statusIsSuccess().hasNoFailToPersistError().containsOutput("persistent = false")
+        .containsOutput("read-serialized = false").containsOutput("ignore-unread-fields = false")
         .containsOutput("Portable Classes = [com.company.DomainObject.*#identity=id]")
         .containsOutput("PDX Serializer = org.apache.geode.pdx.ReflectionBasedAutoSerializer");
 
-    verify(cacheCreation, times(1)).setPdxSerializer(any());
-    verify(command, times(1)).persistClusterConfiguration(any(), any());
     verify(command, times(1)).createReflectionBasedAutoSerializer(true, patterns);
   }
 
@@ -275,108 +223,11 @@ public class ConfigurePDXCommandTest {
     gfshParserRule
         .executeAndAssertThat(command,
             BASE_COMMAND_STRING + "--auto-serializable-classes=" + patterns[0])
-        .statusIsSuccess().hasNoFailToPersistError()
-        .hasReadSerializedConfigured(CacheConfig.DEFAULT_PDX_READ_SERIALIZED, cacheCreation)
-        .hasIgnoreUnreadFieldsConfigured(CacheConfig.DEFAULT_PDX_IGNORE_UNREAD_FIELDS,
-            cacheCreation)
-        .hasPersistenceConfigured(CacheConfig.DEFAULT_PDX_PERSISTENT, null, cacheCreation)
+        .statusIsSuccess().hasNoFailToPersistError().containsOutput("persistent = false")
+        .containsOutput("read-serialized = false").containsOutput("ignore-unread-fields = false")
         .containsOutput("Non Portable Classes = [com.company.DomainObject.*#identity=id]")
         .containsOutput("PDX Serializer = org.apache.geode.pdx.ReflectionBasedAutoSerializer");
 
-    verify(cacheCreation, times(1)).setPdxSerializer(any());
-    verify(command, times(1)).persistClusterConfiguration(any(), any());
     verify(command, times(1)).createReflectionBasedAutoSerializer(false, patterns);
   }
-
-  static class CustomGfshParserRule extends GfshParserRule {
-    @Override
-    public <T> PDXCommandResultAssert executeAndAssertThat(T instance, String command) {
-      CommandResultAssert resultAssert = super.executeAndAssertThat(instance, command);;
-
-      return new PDXCommandResultAssert(resultAssert.getCommandResult());
-    }
-  }
-
-  static class PDXCommandResultAssert extends CommandResultAssert {
-    public PDXCommandResultAssert(CommandResult commandResult) {
-      super(commandResult);
-    }
-
-    @Override
-    public PDXCommandResultAssert statusIsError() {
-      super.statusIsError();
-
-      return this;
-    }
-
-    @Override
-    public PDXCommandResultAssert statusIsSuccess() {
-      super.statusIsSuccess();
-
-      return this;
-    }
-
-    @Override
-    public PDXCommandResultAssert containsOutput(String... expectedOutputs) {
-      super.containsOutput(expectedOutputs);
-
-      return this;
-    }
-
-    @Override
-    public PDXCommandResultAssert hasNoFailToPersistError() {
-      super.hasNoFailToPersistError();
-
-      return this;
-    }
-
-    public PDXCommandResultAssert hasPersistenceConfigured(boolean persistenceEnabled,
-        String diskStoreName, CacheCreation cache) {
-      assertThat(actual.getOutput()).contains("persistent = " + persistenceEnabled);
-
-      if (StringUtils.isNotEmpty(diskStoreName)) {
-        assertThat(actual.getOutput()).contains("disk-store = " + diskStoreName);
-      }
-
-      if (persistenceEnabled) {
-        verify(cache, times(1)).setPdxPersistent(true);
-      } else {
-        verify(cache, times(0)).setPdxPersistent(true);
-      }
-
-      return this;
-    }
-
-
-    public PDXCommandResultAssert hasReadSerializedConfigured(boolean readSerializedEnabled,
-        CacheCreation cache) {
-      assertThat(actual.getOutput()).contains("read-serialized = " + readSerializedEnabled);
-      verify(cache, times(1)).setPdxReadSerialized(readSerializedEnabled);
-
-      return this;
-    }
-
-    public PDXCommandResultAssert hasIgnoreUnreadFieldsConfigured(boolean ignoreUnreadFieldsEnabled,
-        CacheCreation cache) {
-      assertThat(actual.getOutput())
-          .contains("ignore-unread-fields = " + ignoreUnreadFieldsEnabled);
-      verify(cache, times(1)).setPdxIgnoreUnreadFields(ignoreUnreadFieldsEnabled);
-
-      return this;
-    }
-
-    public PDXCommandResultAssert hasDefaultsConfigured(ConfigurePDXCommand command,
-        CacheCreation cacheCreation) {
-      hasNoFailToPersistError();
-      hasReadSerializedConfigured(DEFAULT_PDX_READ_SERIALIZED, cacheCreation);
-      hasIgnoreUnreadFieldsConfigured(DEFAULT_PDX_IGNORE_UNREAD_FIELDS, cacheCreation);
-      hasPersistenceConfigured(DEFAULT_PDX_PERSISTENT, null, cacheCreation);
-
-      verify(cacheCreation, times(0)).setPdxSerializer(any());
-      verify(command, times(1)).persistClusterConfiguration(any(), any());
-      verify(command, times(0)).createReflectionBasedAutoSerializer(anyBoolean(), any());
-
-      return this;
-    }
-  }
 }

-- 
To stop receiving notification emails like this one, please contact
sai_boorlagadda@apache.org.