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/04/11 15:09:56 UTC

[geode] branch develop updated: GEODE-4858: pulling JaxbService out of InternalClusterConfigurationSe… (#1754)

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

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


The following commit(s) were added to refs/heads/develop by this push:
     new 634cd85  GEODE-4858: pulling JaxbService out of InternalClusterConfigurationSe… (#1754)
634cd85 is described below

commit 634cd853356c5b367cb9357b169441a549d3358d
Author: jinmeiliao <ji...@pivotal.io>
AuthorDate: Wed Apr 11 08:09:49 2018 -0700

    GEODE-4858: pulling JaxbService out of InternalClusterConfigurationSe… (#1754)
    
    * GEODE-4858: pulling JaxbService out of InternalClusterConfigurationService
    
    * add capability for JaxService to validate with another xsd
    * cache element use the default namespace ""
    * add capability for modules to register schema location
    * use package-info to do namespace mapping
---
 .../geode/cache/configuration/package-info.java    |  25 +++
 .../distributed/ClusterConfigurationService.java   |  37 +++-
 .../InternalClusterConfigurationService.java       |  91 ++-------
 .../apache/geode/internal/config/JAXBService.java  | 138 +++++++++++++
 .../internal/CacheConfigIntegrationTest.java       |   3 +-
 .../ClusterConfigurationServiceDUnitTest.java      |  63 ++++++
 .../InternalClusterConfigurationServiceTest.java   | 191 +----------------
 .../geode/internal/config/JAXBServiceTest.java     | 226 +++++++++++++++++++++
 .../DescribeJndiBindingCommandDUnitTest.java       |   2 +-
 .../commands/DestroyJndiBindingCommandTest.java    |   5 +
 10 files changed, 512 insertions(+), 269 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/package-info.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/package-info.java
new file mode 100644
index 0000000..c97d892
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+@XmlSchema(namespace = "http://geode.apache.org/schema/cache",
+    xmlns = {@XmlNs(prefix = "", namespaceURI = "http://geode.apache.org/schema/cache")},
+    elementFormDefault = javax.xml.bind.annotation.XmlNsForm.QUALIFIED)
+
+package org.apache.geode.cache.configuration;
+
+import javax.xml.bind.annotation.XmlNs;
+import javax.xml.bind.annotation.XmlSchema;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/ClusterConfigurationService.java b/geode-core/src/main/java/org/apache/geode/distributed/ClusterConfigurationService.java
index 120dab0..65a3066 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/ClusterConfigurationService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/ClusterConfigurationService.java
@@ -30,34 +30,45 @@ import org.apache.geode.management.internal.cli.exceptions.EntityNotFoundExcepti
 
 @Experimental
 public interface ClusterConfigurationService {
+  /**
+   * if you want the output xml have schemaLocation correctly set for your namespace, use this
+   * instead of registerBindClass(Class)./
+   *
+   * @param clazz e.g. CacheConfig.class
+   * @param nameSpaceAndLocation e.g. "http://geode.apache.org/schema/cache
+   *        http://geode.apache.org/schema/cache/cache-1.0.xsd"
+   */
+  void registerBindClassWithSchema(Class clazz, String nameSpaceAndLocation);
+
+  default void registerBindClass(Class clazz) {
+    registerBindClassWithSchema(clazz, null);
+  }
 
   /**
    * retrieves the configuration object of a member group
    *
    * @param group the member group name, if null, then "cluster" is assumed
-   * @param additionalBindClass custom element classes if needed
    * @return the configuration object
    */
-  CacheConfig getCacheConfig(String group, Class<? extends CacheElement>... additionalBindClass);
+  CacheConfig getCacheConfig(String group);
 
   /**
    * update the cluster configuration of a member group
    *
    * @param group the member group name, if null, then "cluster" is assumed
    * @param mutator the change you want to apply to the configuration
-   * @param additionalBindClass custom element classes if needed
    */
-  void updateCacheConfig(String group, UnaryOperator<CacheConfig> mutator,
-      Class<? extends CacheElement>... additionalBindClass);
+  void updateCacheConfig(String group, UnaryOperator<CacheConfig> mutator);
 
 
   default <T extends CacheElement> T getCustomCacheElement(String group, String id,
       Class<T> classT) {
-    CacheConfig cacheConfig = getCacheConfig(group, classT);
+    CacheConfig cacheConfig = getCacheConfig(group);
     return findCustomCacheElement(cacheConfig, id, classT);
   }
 
   default void saveCustomCacheElement(String group, CacheElement element) {
+    registerBindClass(element.getClass());
     updateCacheConfig(group, cacheConfig -> {
       CacheElement foundElement =
           findCustomCacheElement(cacheConfig, element.getId(), element.getClass());
@@ -66,11 +77,12 @@ public interface ClusterConfigurationService {
       }
       cacheConfig.getCustomCacheElements().add(element);
       return cacheConfig;
-    }, element.getClass());
+    });
   }
 
   default void deleteCustomCacheElement(String group, String id,
       Class<? extends CacheElement> classT) {
+    registerBindClass(classT);
     updateCacheConfig(group, config -> {
       CacheElement cacheElement = findCustomCacheElement(config, id, classT);
       if (cacheElement == null) {
@@ -78,16 +90,18 @@ public interface ClusterConfigurationService {
       }
       config.getCustomCacheElements().remove(cacheElement);
       return config;
-    }, classT);
+    });
   }
 
   default <T extends CacheElement> T getCustomRegionElement(String group, String regionPath,
       String id, Class<T> classT) {
-    CacheConfig cacheConfig = getCacheConfig(group, classT);
+    registerBindClass(classT);
+    CacheConfig cacheConfig = getCacheConfig(group);
     return findCustomRegionElement(cacheConfig, regionPath, id, classT);
   }
 
   default void saveCustomRegionElement(String group, String regionPath, CacheElement element) {
+    registerBindClass(element.getClass());
     updateCacheConfig(group, cacheConfig -> {
       RegionConfig regionConfig = findRegionConfiguration(cacheConfig, regionPath);
       if (regionConfig == null) {
@@ -103,11 +117,12 @@ public interface ClusterConfigurationService {
       }
       regionConfig.getCustomRegionElements().add(element);
       return cacheConfig;
-    }, element.getClass());
+    });
   }
 
   default void deleteCustomRegionElement(String group, String regionPath, String id,
       Class<? extends CacheElement> classT) {
+    registerBindClass(classT);
     updateCacheConfig(group, cacheConfig -> {
       RegionConfig regionConfig = findRegionConfiguration(cacheConfig, regionPath);
       if (regionConfig == null) {
@@ -120,7 +135,7 @@ public interface ClusterConfigurationService {
       }
       regionConfig.getCustomRegionElements().remove(element);
       return cacheConfig;
-    }, classT);
+    });
   }
 
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalClusterConfigurationService.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalClusterConfigurationService.java
index 7c6a194..ed4e467 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalClusterConfigurationService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalClusterConfigurationService.java
@@ -26,7 +26,6 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintWriter;
-import java.io.StringReader;
 import java.io.StringWriter;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -35,7 +34,6 @@ import java.nio.file.StandardCopyOption;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
@@ -49,9 +47,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.UnaryOperator;
 import java.util.stream.Collectors;
 
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
 import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.transform.TransformerException;
 import javax.xml.transform.TransformerFactoryConfigurationError;
@@ -79,7 +74,6 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.TimeoutException;
 import org.apache.geode.cache.configuration.CacheConfig;
-import org.apache.geode.cache.configuration.CacheElement;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.ClusterConfigurationService;
 import org.apache.geode.distributed.DistributedLockService;
@@ -92,6 +86,7 @@ import org.apache.geode.internal.cache.persistence.PersistentMemberID;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
 import org.apache.geode.internal.cache.xmlcache.CacheXmlGenerator;
+import org.apache.geode.internal.config.JAXBService;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.internal.beans.FileUploader;
 import org.apache.geode.management.internal.cli.CliUtil;
@@ -142,14 +137,14 @@ public class InternalClusterConfigurationService implements ClusterConfiguration
 
   private final InternalCache cache;
   private DistributedLockService sharedConfigLockingService;
+  private JAXBService jaxbService;
 
   @TestingOnly
   InternalClusterConfigurationService() {
     configDirPath = null;
     configDiskDirPath = null;
     cache = null;
-    bindClasses.add(CacheConfig.class);
-    initJaxbContext();
+    jaxbService = new JAXBService();
   }
 
   public InternalClusterConfigurationService(InternalCache cache) throws IOException {
@@ -177,8 +172,7 @@ public class InternalClusterConfigurationService implements ClusterConfiguration
     this.configDiskDirPath = FilenameUtils.concat(clusterConfigRootDir, configDiskDirName);
     this.sharedConfigLockingService = getSharedConfigLockService(cache.getDistributedSystem());
     this.status.set(SharedConfigurationStatus.NOT_STARTED);
-    bindClasses.add(CacheConfig.class);
-    initJaxbContext();
+    jaxbService = new JAXBService();
   }
 
   /**
@@ -822,47 +816,6 @@ public class InternalClusterConfigurationService implements ClusterConfiguration
     return configDir;
   }
 
-  Collection<Class> bindClasses = new ArrayList<>();
-  private Marshaller marshaller;
-  private Unmarshaller unmarshaller;
-
-  void initJaxbContext() {
-    try {
-      JAXBContext jaxbContext =
-          JAXBContext.newInstance(bindClasses.toArray(new Class[bindClasses.size()]));
-      marshaller = jaxbContext.createMarshaller();
-      // currently we are generating the xml from jabx object so we don't need schema validation.
-      // but in the future, we will need to add the various xsd for schema validation. and these
-      // xsd needs to be local (no network access)
-      marshaller.setProperty(Marshaller.JAXB_SCHEMA_LOCATION,
-          "http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd");
-      marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-      unmarshaller = jaxbContext.createUnmarshaller();
-    } catch (Exception e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
-  }
-
-  private Marshaller getMarshaller(Class<? extends CacheElement>... additionBindClass) {
-    if (bindClasses.containsAll(Arrays.asList(additionBindClass))) {
-      return marshaller;
-    }
-
-    bindClasses.addAll(Arrays.asList(additionBindClass));
-    initJaxbContext();
-    return marshaller;
-  }
-
-  private Unmarshaller getUnmarshaller(Class<? extends CacheElement>... additionBindClass) {
-    if (bindClasses.containsAll(Arrays.asList(additionBindClass))) {
-      return unmarshaller;
-    }
-
-    bindClasses.addAll(Arrays.asList(additionBindClass));
-    initJaxbContext();
-    return unmarshaller;
-  }
-
   private String generateInitialXmlContent() {
     StringWriter sw = new StringWriter();
     PrintWriter pw = new PrintWriter(sw);
@@ -871,8 +824,13 @@ public class InternalClusterConfigurationService implements ClusterConfiguration
   }
 
   @Override
-  public CacheConfig getCacheConfig(String group,
-      Class<? extends CacheElement>... additionalBindClass) {
+  public void registerBindClassWithSchema(Class clazz, String namespaceAndLocation) {
+    jaxbService.registerBindClassWithSchema(clazz, namespaceAndLocation);
+  }
+
+
+  @Override
+  public CacheConfig getCacheConfig(String group) {
     if (group == null) {
       group = CLUSTER_CONFIG;
     }
@@ -880,47 +838,28 @@ public class InternalClusterConfigurationService implements ClusterConfiguration
     if (xmlContent == null || xmlContent.isEmpty()) {
       xmlContent = generateInitialXmlContent();
     }
-    return unMarshall(xmlContent, additionalBindClass);
+    return jaxbService.unMarshall(xmlContent);
   }
 
   @Override
-  public void updateCacheConfig(String group, UnaryOperator<CacheConfig> mutator,
-      Class<? extends CacheElement>... additionalBindClass) {
+  public void updateCacheConfig(String group, UnaryOperator<CacheConfig> mutator) {
     if (group == null) {
       group = CLUSTER_CONFIG;
     }
     lockSharedConfiguration();
     try {
-      CacheConfig cacheConfig = getCacheConfig(group, additionalBindClass);
+      CacheConfig cacheConfig = getCacheConfig(group);
       cacheConfig = mutator.apply(cacheConfig);
       if (cacheConfig == null) {
         // mutator returns a null config, indicating no change needs to be persisted
         return;
       }
       Configuration configuration = getConfiguration(group);
-      configuration.setCacheXmlContent(marshall(cacheConfig, additionalBindClass));
+      configuration.setCacheXmlContent(jaxbService.marshall(cacheConfig));
       getConfigurationRegion().put(group, configuration);
     } finally {
       unlockSharedConfiguration();
     }
   }
 
-  String marshall(CacheConfig config, Class<? extends CacheElement>... additionalClass) {
-    StringWriter sw = new StringWriter();
-    try {
-      getMarshaller(additionalClass).marshal(config, sw);
-    } catch (Exception e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return sw.toString();
-  }
-
-  CacheConfig unMarshall(String xml, Class<? extends CacheElement>... additionalClass) {
-    try {
-      return (CacheConfig) getUnmarshaller(additionalClass).unmarshal(new StringReader(xml));
-    } catch (Exception e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
-  }
-
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/config/JAXBService.java b/geode-core/src/main/java/org/apache/geode/internal/config/JAXBService.java
new file mode 100644
index 0000000..1c85d53
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/config/JAXBService.java
@@ -0,0 +1,138 @@
+/*
+ * 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.internal.config;
+
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+
+import org.xml.sax.SAXException;
+
+import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.internal.ClassPathLoader;
+
+public class JAXBService {
+  public static String CACHE_SCHEMA =
+      "http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd";
+  Map<Class, String> classAndSchema = new HashMap<>();
+
+  Marshaller marshaller;
+  Unmarshaller unmarshaller;
+
+  // the default service will handle the cache.xsd validation and set's the cache schema location
+  public JAXBService() {
+    registerBindClassWithSchema(CacheConfig.class, CACHE_SCHEMA);
+    // find the local Cache-1.0.xsd
+    URL local_cache_xsd = ClassPathLoader.getLatest()
+        .getResource("META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd");
+    validateWith(local_cache_xsd);
+  }
+
+  public void validateWith(URL url) {
+    SchemaFactory factory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+    Schema schema = null;
+    try {
+      schema = factory.newSchema(url);
+    } catch (SAXException e) {
+      throw new RuntimeException(e.getMessage(), e);
+    }
+    marshaller.setSchema(schema);
+  }
+
+  /**
+   * if you want the output xml have schemaLocation correctly set for your namespace, use this
+   * instead of registerBindClass(Class)
+   *
+   * @param c e.g CacheConfig.class
+   * @param nameSpaceAndSchemaLocation e.g "http://geode.apache.org/schema/cache
+   *        http://geode.apache.org/schema/cache/cache-1.0.xsd"
+   */
+  public void registerBindClassWithSchema(Class c, String nameSpaceAndSchemaLocation) {
+    // if this class is not in the map yet
+    if (!classAndSchema.keySet().contains(c)) {
+      classAndSchema.put(c, nameSpaceAndSchemaLocation);
+      try {
+        Set<Class> bindClasses = classAndSchema.keySet();
+        JAXBContext jaxbContext =
+            JAXBContext.newInstance(bindClasses.toArray(new Class[bindClasses.size()]));
+        marshaller = jaxbContext.createMarshaller();
+        unmarshaller = jaxbContext.createUnmarshaller();
+        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+        updateSchema();
+      } catch (Exception e) {
+        throw new RuntimeException(e.getMessage(), e);
+      }
+      return;
+    }
+
+    // if this class is in the map already and we are simply adding/updating schema
+    String oldSchema = classAndSchema.get(c);
+    if (nameSpaceAndSchemaLocation == null) {
+      return;
+    }
+
+    if (!nameSpaceAndSchemaLocation.equals(oldSchema)) {
+      classAndSchema.put(c, nameSpaceAndSchemaLocation);
+      updateSchema();
+    }
+  }
+
+  void updateSchema() {
+    try {
+      String schemas = classAndSchema.values().stream().filter(Objects::nonNull)
+          .collect(Collectors.joining(" "));
+      marshaller.setProperty(Marshaller.JAXB_SCHEMA_LOCATION, schemas);
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage(), e);
+    }
+  }
+
+  public String marshall(Object object) {
+    StringWriter sw = new StringWriter();
+    try {
+      marshaller.marshal(object, sw);
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage(), e);
+    }
+    return sw.toString();
+  }
+
+  public <T> T unMarshall(String xml) {
+    try {
+      return (T) unmarshaller.unmarshal(new StringReader(xml));
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage(), e);
+    }
+  }
+
+  public void registerBindClasses(Class clazz) {
+    registerBindClassWithSchema(clazz, null);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/CacheConfigIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/CacheConfigIntegrationTest.java
index 9c80499..72cb4f8 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/CacheConfigIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/CacheConfigIntegrationTest.java
@@ -26,6 +26,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.internal.config.JAXBService;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.apache.geode.test.junit.rules.ServerStarterRule;
 
@@ -44,7 +45,7 @@ public class CacheConfigIntegrationTest {
     xmlFile = temporaryFolder.newFile("cache.xml");
     CacheConfig cacheConfig = new CacheConfig();
     cacheConfig.setVersion("1.0");
-    InternalClusterConfigurationService service = new InternalClusterConfigurationService();
+    JAXBService service = new JAXBService();
     String xml = service.marshall(cacheConfig);
     FileUtils.writeStringToFile(xmlFile, xml, "UTF-8");
 
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterConfigurationServiceDUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterConfigurationServiceDUnitTest.java
new file mode 100644
index 0000000..c2809d2
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterConfigurationServiceDUnitTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.distributed.internal;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.distributed.ClusterConfigurationService;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+
+
+@Category(DistributedTest.class)
+public class ClusterConfigurationServiceDUnitTest {
+
+
+  @Rule
+  public ClusterStartupRule cluster = new ClusterStartupRule();
+
+  @Test
+  public void xmlCreatedByCCServiceCanBeLoadedByServer() {
+    MemberVM locator = cluster.startLocatorVM(0);
+
+    locator.invoke(() -> {
+      ClusterConfigurationService ccService =
+          ClusterStartupRule.getLocator().getSharedConfiguration();
+      ccService.updateCacheConfig("cluster", cc -> {
+        RegionConfig regionConfig = new RegionConfig();
+        regionConfig.setName("regionB");
+        regionConfig.setRefid("REPLICATE");
+        cc.getRegion().add(regionConfig);
+        return cc;
+      });
+    });
+
+    MemberVM server = cluster.startServerVM(1, locator.getPort());
+
+    server.invoke(() -> {
+      assertThat(ClusterStartupRule.getCache().getRegion("/regionB")).isNotNull();
+    });
+
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalClusterConfigurationServiceTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalClusterConfigurationServiceTest.java
index 6b338e0..e48d1a4 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalClusterConfigurationServiceTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalClusterConfigurationServiceTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.geode.distributed.internal;
 
+import static org.apache.geode.internal.config.JAXBServiceTest.setBasicValues;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.mockito.ArgumentMatchers.any;
@@ -25,20 +26,16 @@ import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.XmlType;
-
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.configuration.CacheConfig;
-import org.apache.geode.cache.configuration.CacheElement;
 import org.apache.geode.cache.configuration.JndiBindingsType;
 import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.internal.config.JAXBServiceTest;
+import org.apache.geode.internal.config.JAXBServiceTest.ElementOne;
+import org.apache.geode.internal.config.JAXBServiceTest.ElementTwo;
 import org.apache.geode.management.internal.cli.exceptions.EntityNotFoundException;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.test.junit.categories.UnitTest;
@@ -46,107 +43,17 @@ import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class InternalClusterConfigurationServiceTest {
-  private String xml;
-  private CacheConfig unmarshalled;
-  private InternalClusterConfigurationService service, service2;
+  private InternalClusterConfigurationService service;
   private Configuration configuration;
 
   @Before
   public void setUp() throws Exception {
     service = spy(InternalClusterConfigurationService.class);
-    service2 = spy(InternalClusterConfigurationService.class);
     configuration = new Configuration("cluster");
     doReturn(configuration).when(service).getConfiguration(any());
-    doReturn(configuration).when(service2).getConfiguration(any());
     doReturn(mock(Region.class)).when(service).getConfigurationRegion();
-    doReturn(mock(Region.class)).when(service2).getConfigurationRegion();
     doReturn(true).when(service).lockSharedConfiguration();
-    doReturn(true).when(service2).lockSharedConfiguration();
     doNothing().when(service).unlockSharedConfiguration();
-    doNothing().when(service2).unlockSharedConfiguration();
-  }
-
-  @Test
-  public void testCacheMarshall() {
-    CacheConfig cacheConfig = new CacheConfig();
-    setBasicValues(cacheConfig);
-
-    xml = service.marshall(cacheConfig);
-    System.out.println(xml);
-    assertThat(xml).contains("</cache>");
-
-    assertThat(xml).contains("xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"");
-    assertThat(xml).contains(
-        "xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\"");
-
-    unmarshalled = service.unMarshall(xml);
-  }
-
-  @Test
-  public void testCacheOneMarshall() throws Exception {
-    CacheConfig cache = new CacheConfig();
-    setBasicValues(cache);
-    cache.getCustomCacheElements().add(new ElementOne("test"));
-
-    xml = service.marshall(cache, ElementOne.class);
-    System.out.println(xml);
-    assertThat(xml).contains("custom-one>");
-  }
-
-  @Test
-  public void testMixMarshall() throws Exception {
-    CacheConfig cache = new CacheConfig();
-    setBasicValues(cache);
-    cache.getCustomCacheElements().add(new ElementOne("testOne"));
-
-    xml = service.marshall(cache, ElementOne.class);
-    System.out.println(xml);
-    assertThat(xml).contains("custom-one>");
-
-    // xml generated with CacheConfigOne marshaller can be unmarshalled by CacheConfigTwo
-    unmarshalled = service.unMarshall(xml);
-    unmarshalled.getCustomCacheElements().add(new ElementTwo("testTwo"));
-
-    // xml generated wtih CacheConfigTwo has both elements in there.
-    xml = service.marshall(unmarshalled, ElementTwo.class);
-    System.out.println(xml);
-    assertThat(xml).contains("custom-one>");
-    assertThat(xml).contains("custom-two>");
-    assertThat(xml).containsPattern("xmlns:ns\\d=\"http://geode.apache.org/schema/cache\"");
-    assertThat(xml).containsPattern("xmlns:ns\\d=\"http://geode.apache.org/schema/CustomOne\"");
-    assertThat(xml).containsPattern("xmlns:ns\\d=\"http://geode.apache.org/schema/CustomTwo\"");
-  }
-
-  @Test
-  public void xmlWithCustomElementsCanBeUnMarshalledByAnotherService() {
-    service.saveCustomCacheElement("cluster", new ElementOne("one"));
-    service.saveCustomCacheElement("cluster", new ElementTwo("two"));
-
-    String prettyXml = configuration.getCacheXmlContent();
-    System.out.println(prettyXml);
-
-    // the xml is sent to another locator, and can interpreted ocrrectly
-    service2.updateCacheConfig("cluster", cc -> {
-      return cc;
-    });
-
-    ElementOne elementOne = service2.getCustomCacheElement("cluster", "one", ElementOne.class);
-    assertThat(elementOne.getId()).isEqualTo("one");
-
-    String uglyXml = configuration.getCacheXmlContent();
-    System.out.println(uglyXml);
-    assertThat(uglyXml).isNotEqualTo(prettyXml);
-
-    // the xml can be unmarshalled correctly by the first locator
-    CacheConfig cacheConfig = service.getCacheConfig("cluster");
-    service.updateCacheConfig("cluster", cc -> {
-      return cc;
-    });
-    assertThat(cacheConfig.getCustomCacheElements()).hasSize(2);
-    assertThat(cacheConfig.getCustomCacheElements().get(0)).isInstanceOf(ElementOne.class);
-    assertThat(cacheConfig.getCustomCacheElements().get(1)).isInstanceOf(ElementTwo.class);
-
-    assertThat(configuration.getCacheXmlContent()).isEqualTo(prettyXml);
   }
 
   @Test
@@ -164,22 +71,6 @@ public class InternalClusterConfigurationServiceTest {
         .contains("<region name=\"regionA\" refid=\"REPLICATE\"/>");
   }
 
-  private void setBasicValues(CacheConfig cache) {
-    cache.setCopyOnRead(true);
-    CacheConfig.GatewayReceiver receiver = new CacheConfig.GatewayReceiver();
-    receiver.setBindAddress("localhost");
-    receiver.setEndPort("8080");
-    receiver.setManualStart(false);
-    receiver.setStartPort("6000");
-    cache.setGatewayReceiver(receiver);
-    cache.setVersion("1.0");
-
-    RegionConfig region = new RegionConfig();
-    region.setName("testRegion");
-    region.setRefid("REPLICATE");
-    cache.getRegion().add(region);
-  }
-
   @Test
   public void jndiBindings() {
     service.updateCacheConfig("cluster", cacheConfig -> {
@@ -207,7 +98,7 @@ public class InternalClusterConfigurationServiceTest {
     System.out.println(configuration.getCacheXmlContent());
     assertThat(configuration.getCacheXmlContent()).contains("custom-one>");
 
-    ElementTwo customTwo = new ElementTwo("testTwo");
+    JAXBServiceTest.ElementTwo customTwo = new ElementTwo("testTwo");
     service.saveCustomCacheElement("cluster", customTwo);
     System.out.println(configuration.getCacheXmlContent());
     assertThat(configuration.getCacheXmlContent()).contains("custom-one>");
@@ -220,7 +111,7 @@ public class InternalClusterConfigurationServiceTest {
     service.saveCustomCacheElement("cluster", customOne);
     System.out.println(configuration.getCacheXmlContent());
     assertThat(configuration.getCacheXmlContent()).contains("custom-one>");
-    assertThat(configuration.getCacheXmlContent()).contains("<id>testOne</id>");
+    assertThat(configuration.getCacheXmlContent()).containsPattern("<ns\\d:id>testOne</ns\\d:id>");
     assertThat(configuration.getCacheXmlContent()).doesNotContain("<value>");
 
     customOne = service.getCustomCacheElement("cluster", "testOne", ElementOne.class);
@@ -228,8 +119,9 @@ public class InternalClusterConfigurationServiceTest {
     service.saveCustomCacheElement("cluster", customOne);
     System.out.println(configuration.getCacheXmlContent());
     assertThat(configuration.getCacheXmlContent()).contains("custom-one>");
-    assertThat(configuration.getCacheXmlContent()).contains("<id>testOne</id>");
-    assertThat(configuration.getCacheXmlContent()).contains("<value>valueOne</value>");
+    assertThat(configuration.getCacheXmlContent()).containsPattern("<ns\\d:id>testOne</ns\\d:id>");
+    assertThat(configuration.getCacheXmlContent())
+        .containsPattern("<ns\\d:value>valueOne</ns\\d:value>");
   }
 
   @Test
@@ -263,7 +155,7 @@ public class InternalClusterConfigurationServiceTest {
     System.out.println(configuration.getCacheXmlContent());
     assertThat(configuration.getCacheXmlContent()).contains("region name=\"testRegion\"");
     assertThat(configuration.getCacheXmlContent())
-        .containsPattern("\\w*:custom-one\\W*\\w*:region");
+        .containsPattern("</ns\\d:custom-one>\n" + "    </region>");
 
     ElementOne retrieved =
         service.getCustomRegionElement("cluster", "testRegion", "elementOne", ElementOne.class);
@@ -274,65 +166,4 @@ public class InternalClusterConfigurationServiceTest {
     System.out.println(configuration.getCacheXmlContent());
     assertThat(configuration.getCacheXmlContent()).doesNotContain("custom-one>");
   }
-
-
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlType(name = "", propOrder = {"id", "value"})
-  @XmlRootElement(name = "custom-one", namespace = "http://geode.apache.org/schema/CustomOne")
-  public static class ElementOne implements CacheElement {
-    private String id;
-    private String value;
-
-    public ElementOne() {}
-
-    public String getValue() {
-      return value;
-    }
-
-    public void setValue(String value) {
-      this.value = value;
-    }
-
-    public ElementOne(String id) {
-      this.id = id;
-    }
-
-    public String getId() {
-      return id;
-    }
-
-    public void setId(String value) {
-      this.id = value;
-    }
-  }
-
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlType(name = "", propOrder = {"id", "value"})
-  @XmlRootElement(name = "custom-two", namespace = "http://geode.apache.org/schema/CustomTwo")
-  public static class ElementTwo implements CacheElement {
-    private String id;
-    private String value;
-
-    public ElementTwo() {}
-
-    public String getValue() {
-      return value;
-    }
-
-    public void setValue(String value) {
-      this.value = value;
-    }
-
-    public ElementTwo(String id) {
-      this.id = id;
-    }
-
-    public String getId() {
-      return id;
-    }
-
-    public void setId(String value) {
-      this.id = value;
-    }
-  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/config/JAXBServiceTest.java b/geode-core/src/test/java/org/apache/geode/internal/config/JAXBServiceTest.java
new file mode 100644
index 0000000..0593f19
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/config/JAXBServiceTest.java
@@ -0,0 +1,226 @@
+/*
+ * 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.internal.config;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.spy;
+
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.cache.configuration.CacheElement;
+import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+
+@Category(UnitTest.class)
+public class JAXBServiceTest {
+
+  private String xml;
+  private CacheConfig unmarshalled;
+  private JAXBService service, service2;
+
+  @Before
+  public void setUp() throws Exception {
+    service = spy(new JAXBService());
+    service2 = spy(new JAXBService());
+  }
+
+  @Test
+  public void testCacheMarshall() {
+    CacheConfig cacheConfig = new CacheConfig();
+    setBasicValues(cacheConfig);
+
+    xml = service.marshall(cacheConfig);
+    System.out.println(xml);
+    // cache has the default namespace
+    assertThat(xml).contains("xmlns=\"http://geode.apache.org/schema/cache\"");
+    assertThat(xml).contains(
+        "xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\"");
+    assertThat(xml).contains("xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"");
+    assertThat(xml).contains("</cache>");
+
+    unmarshalled = service.unMarshall(xml);
+  }
+
+  @Test
+  public void invalidXmlShouldFail() throws Exception {
+    CacheConfig cacheConfig = new CacheConfig();
+    // missing version attribute
+    assertThatThrownBy(() -> service.marshall(cacheConfig))
+        .hasStackTraceContaining("Attribute 'version' must appear on element 'cache'");
+  }
+
+  @Test
+  public void testCacheOneMarshall() throws Exception {
+    CacheConfig cache = new CacheConfig();
+    setBasicValues(cache);
+    cache.getCustomCacheElements().add(new ElementOne("test"));
+
+    service.registerBindClasses(ElementOne.class);
+    xml = service.marshall(cache);
+    System.out.println(xml);
+    // cache has the default namespace
+    assertThat(xml).contains("xmlns=\"http://geode.apache.org/schema/cache\"");
+    assertThat(xml).contains("custom-one>");
+  }
+
+  @Test
+  public void testMixMarshall() throws Exception {
+    CacheConfig cache = new CacheConfig();
+    setBasicValues(cache);
+    cache.getCustomCacheElements().add(new ElementOne("testOne"));
+    service.registerBindClasses(ElementOne.class);
+
+    xml = service.marshall(cache);
+    System.out.println(xml);
+    assertThat(xml).contains("custom-one>");
+
+    unmarshalled = service.unMarshall(xml);
+    unmarshalled.getCustomCacheElements().add(new ElementTwo("testTwo"));
+    service.registerBindClasses(ElementTwo.class);
+
+    // xml generated wtih CacheConfigTwo has both elements in there.
+    xml = service.marshall(unmarshalled);
+    System.out.println(xml);
+    assertThat(xml).contains("custom-one>");
+    assertThat(xml).contains("custom-two>");
+    assertThat(xml).containsPattern("xmlns=\"http://geode.apache.org/schema/cache\"");
+    assertThat(xml).containsPattern("xmlns:ns\\d=\"http://geode.apache.org/schema/CustomOne\"");
+    assertThat(xml).containsPattern("xmlns:ns\\d=\"http://geode.apache.org/schema/CustomTwo\"");
+  }
+
+  @Test
+  public void xmlWithCustomElementsCanBeUnMarshalledByAnotherService() {
+    CacheConfig cache = new CacheConfig();
+    setBasicValues(cache);
+    service.registerBindClasses(ElementOne.class);
+    service.registerBindClasses(ElementTwo.class);
+    cache.getCustomCacheElements().add(new ElementOne("test"));
+    cache.getCustomCacheElements().add(new ElementTwo("test"));
+
+    String prettyXml = service.marshall(cache);
+    System.out.println(prettyXml);
+
+    service2.registerBindClasses(ElementOne.class);
+    CacheConfig cacheConfig = service2.unMarshall(prettyXml);
+    List elements = cacheConfig.getCustomCacheElements();
+    assertThat(elements.get(0)).isInstanceOf(ElementOne.class);
+    assertThat(elements.get(1)).isNotInstanceOf(ElementTwo.class);
+
+    String uglyXml = service2.marshall(cacheConfig);
+    System.out.println(uglyXml);
+    assertThat(uglyXml).isNotEqualTo(prettyXml);
+
+    // the xml can be unmarshalled correctly by the first service
+    String newXml = service.marshall(service.unMarshall(uglyXml));
+    assertThat(newXml).isEqualTo(prettyXml);
+  }
+
+  public static void setBasicValues(CacheConfig cache) {
+    cache.setCopyOnRead(true);
+    CacheConfig.GatewayReceiver receiver = new CacheConfig.GatewayReceiver();
+    receiver.setBindAddress("localhost");
+    receiver.setEndPort("8080");
+    receiver.setManualStart(false);
+    receiver.setStartPort("6000");
+    cache.setGatewayReceiver(receiver);
+    cache.setVersion("1.0");
+
+    RegionConfig region = new RegionConfig();
+    region.setName("testRegion");
+    region.setRefid("REPLICATE");
+    cache.getRegion().add(region);
+  }
+
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlType(name = "", propOrder = {"id", "value"})
+  @XmlRootElement(name = "custom-one", namespace = "http://geode.apache.org/schema/CustomOne")
+  public static class ElementOne implements CacheElement {
+    @XmlElement(name = "id", namespace = "http://geode.apache.org/schema/CustomOne")
+    private String id;
+    @XmlElement(name = "value", namespace = "http://geode.apache.org/schema/CustomOne")
+    private String value;
+
+    public ElementOne() {}
+
+    public String getValue() {
+      return value;
+    }
+
+    public void setValue(String value) {
+      this.value = value;
+    }
+
+    public ElementOne(String id) {
+      this.id = id;
+    }
+
+    public String getId() {
+      return id;
+    }
+
+    public void setId(String value) {
+      this.id = value;
+    }
+  }
+
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlType(name = "", propOrder = {"id", "value"})
+  @XmlRootElement(name = "custom-two", namespace = "http://geode.apache.org/schema/CustomTwo")
+  public static class ElementTwo implements CacheElement {
+    @XmlElement(name = "id", namespace = "http://geode.apache.org/schema/CustomTwo")
+    private String id;
+    @XmlElement(name = "value", namespace = "http://geode.apache.org/schema/CustomTwo")
+    private String value;
+
+    public ElementTwo() {}
+
+    public String getValue() {
+      return value;
+    }
+
+    public void setValue(String value) {
+      this.value = value;
+    }
+
+    public ElementTwo(String id) {
+      this.id = id;
+    }
+
+    public String getId() {
+      return id;
+    }
+
+    public void setId(String value) {
+      this.id = value;
+    }
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DescribeJndiBindingCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DescribeJndiBindingCommandDUnitTest.java
index 92c046c..565878f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DescribeJndiBindingCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DescribeJndiBindingCommandDUnitTest.java
@@ -53,7 +53,7 @@ public class DescribeJndiBindingCommandDUnitTest {
   @Test
   public void describeJndiBindingForSimpleDataSource() {
     gfsh.executeAndAssertThat(
-        "create jndi-binding --name=jndi-simple --type=SIMPLE --jdbc-driver-class=org.apache.derby.jdbc.EmbeddedDriver --connection-url=\"jdbc:derby:newDB;create=true\" --username=joe --datasource-config-properties={'name':'prop1','value':'value1'}")
+        "create jndi-binding --name=jndi-simple --type=SIMPLE --jdbc-driver-class=org.apache.derby.jdbc.EmbeddedDriver --connection-url=\"jdbc:derby:newDB;create=true\" --username=joe --datasource-config-properties={'name':'prop1','value':'value1','type':'java.lang.String'}")
         .statusIsSuccess().tableHasColumnOnlyWithValues("Member", "server-1");
 
     gfsh.executeAndAssertThat("describe jndi-binding --name=jndi-simple").statusIsSuccess()
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
index 62c93c2..bcc8d25 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.management.internal.cli.commands;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -69,6 +70,8 @@ public class DestroyJndiBindingCommandTest {
     doReturn(cache).when(command).getCache();
     cacheConfig = mock(CacheConfig.class);
     ccService = mock(InternalClusterConfigurationService.class);
+
+
     doReturn(ccService).when(command).getConfigurationService();
     when(ccService.getCacheConfig(any())).thenReturn(cacheConfig);
     doCallRealMethod().when(ccService).updateCacheConfig(any(), any());
@@ -120,6 +123,7 @@ public class DestroyJndiBindingCommandTest {
 
   @Test
   public void whenNoMembersFoundAndClusterConfigRunningThenUpdateClusterConfig() {
+    doNothing().when(ccService).updateCacheConfig(any(), any());
     doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
     when(ccService.findIdentifiable(any(), any()))
         .thenReturn(mock(JndiBindingsType.JndiBinding.class));
@@ -177,6 +181,7 @@ public class DestroyJndiBindingCommandTest {
 
     doReturn(members).when(command).findMembers(any(), any());
     doReturn(results).when(command).executeAndGetFunctionResult(any(), any(), any());
+    doNothing().when(ccService).updateCacheConfig(any(), any());
     when(ccService.findIdentifiable(any(), any()))
         .thenReturn(mock(JndiBindingsType.JndiBinding.class));
 

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