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/23 23:43:55 UTC

[geode] branch develop updated: GEODE-4858: use FastPathScanner to register all the bind classes when c… (#1838)

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 7972a9e  GEODE-4858: use FastPathScanner to register all the bind classes when c… (#1838)
7972a9e is described below

commit 7972a9e7be975dd198db9dba534134368a48d755
Author: jinmeiliao <ji...@pivotal.io>
AuthorDate: Mon Apr 23 16:43:50 2018 -0700

    GEODE-4858: use FastPathScanner to register all the bind classes when c… (#1838)
    
    * do not register bind class on demand. Register all custom element at service start time.
---
 .../internal/configuration/ConnectorService.java   |  6 +-
 .../configuration/ConnectorServiceTest.java        |  4 +-
 .../geode/cache/configuration/CacheConfig.java     |  4 +-
 .../geode/cache/configuration/XSDRootElement.java  | 32 ++++++++
 .../distributed/ClusterConfigurationService.java   | 20 -----
 .../InternalClusterConfigurationService.java       | 27 ++++---
 .../apache/geode/internal/config/JAXBService.java  | 93 +++++++---------------
 .../internal/cli/util/ClasspathScanLoadHelper.java |  9 +++
 .../internal/CacheConfigIntegrationTest.java       |  2 +-
 .../InternalClusterConfigurationServiceTest.java   | 12 +--
 .../geode/internal/config/JAXBServiceTest.java     | 18 ++---
 11 files changed, 108 insertions(+), 119 deletions(-)

diff --git a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/configuration/ConnectorService.java b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/configuration/ConnectorService.java
index 9d5f3bb..521dac8 100644
--- a/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/configuration/ConnectorService.java
+++ b/geode-connectors/src/main/java/org/apache/geode/connectors/jdbc/internal/configuration/ConnectorService.java
@@ -37,6 +37,7 @@ import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.XmlType;
 
 import org.apache.geode.cache.configuration.CacheElement;
+import org.apache.geode.cache.configuration.XSDRootElement;
 import org.apache.geode.connectors.jdbc.JdbcConnectorException;
 import org.apache.geode.connectors.jdbc.internal.TableMetaDataView;
 import org.apache.geode.pdx.internal.PdxType;
@@ -105,10 +106,9 @@ import org.apache.geode.pdx.internal.TypeRegistry;
 @XmlAccessorType(XmlAccessType.FIELD)
 @XmlType(name = "", propOrder = {"connection", "regionMapping"})
 @XmlRootElement(name = "connector-service", namespace = "http://geode.apache.org/schema/jdbc")
+@XSDRootElement(namespace = "http://geode.apache.org/schema/jdbc",
+    schemaLocation = "http://geode.apache.org/schema/jdbc/jdbc-1.0.xsd")
 public class ConnectorService implements CacheElement {
-  public static String SCHEMA =
-      "http://geode.apache.org/schema/jdbc http://geode.apache.org/schema/jdbc/jdbc-1.0.xsd";
-
   @XmlElement(namespace = "http://geode.apache.org/schema/jdbc")
   protected List<ConnectorService.Connection> connection;
   @XmlElement(name = "region-mapping", namespace = "http://geode.apache.org/schema/jdbc")
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/configuration/ConnectorServiceTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/configuration/ConnectorServiceTest.java
index 4f0afa2..34e7658 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/configuration/ConnectorServiceTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/configuration/ConnectorServiceTest.java
@@ -26,6 +26,7 @@ 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.internal.ClassPathLoader;
 import org.apache.geode.internal.config.JAXBService;
 import org.apache.geode.test.junit.categories.UnitTest;
@@ -38,8 +39,7 @@ public class ConnectorServiceTest {
 
   @Before
   public void setUp() throws Exception {
-    jaxbService = new JAXBService();
-    jaxbService.registerBindClassWithSchema(ConnectorService.class, ConnectorService.SCHEMA);
+    jaxbService = new JAXBService(CacheConfig.class, ConnectorService.class);
     // find the local jdbc-1.0.xsd
     URL local_xsd = ClassPathLoader.getLatest()
         .getResource("META-INF/schemas/geode.apache.org/schema/jdbc/jdbc-1.0.xsd");
diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java
index eb4d58a..49ac6e8 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java
@@ -265,9 +265,10 @@ import org.apache.geode.annotations.Experimental;
         "diskStore", "pdx", "regionAttributes", "jndiBindings", "region", "functionService",
         "resourceManager", "serializationRegistration", "backup", "initializer", "cacheElements"})
 @XmlRootElement(name = "cache", namespace = "http://geode.apache.org/schema/cache")
+@XSDRootElement(namespace = "http://geode.apache.org/schema/cache",
+    schemaLocation = "http://geode.apache.org/schema/cache/cache-1.0.xsd")
 @Experimental
 public class CacheConfig {
-
   @XmlElement(name = "cache-transaction-manager",
       namespace = "http://geode.apache.org/schema/cache")
   protected CacheTransactionManagerType cacheTransactionManager;
@@ -1021,7 +1022,6 @@ public class CacheConfig {
     this.version = value;
   }
 
-
   /**
    * <p>
    * Java class for anonymous complex type.
diff --git a/geode-core/src/main/java/org/apache/geode/cache/configuration/XSDRootElement.java b/geode-core/src/main/java/org/apache/geode/cache/configuration/XSDRootElement.java
new file mode 100644
index 0000000..477e1a6
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/cache/configuration/XSDRootElement.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cache.configuration;
+
+import static java.lang.annotation.ElementType.TYPE;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+@Retention(RUNTIME)
+@Target({TYPE})
+public @interface XSDRootElement {
+  String namespace() default "";
+
+  String schemaLocation() default "";
+}
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 2681145..8cf9b84 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
@@ -32,20 +32,6 @@ 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
@@ -64,7 +50,6 @@ public interface ClusterConfigurationService {
 
   default <T extends CacheElement> T getCustomCacheElement(String group, String id,
       Class<T> classT) {
-    registerBindClass(classT);
     CacheConfig cacheConfig = getCacheConfig(group);
     if (cacheConfig == null) {
       return null;
@@ -73,7 +58,6 @@ public interface ClusterConfigurationService {
   }
 
   default void saveCustomCacheElement(String group, CacheElement element) {
-    registerBindClass(element.getClass());
     updateCacheConfig(group, cacheConfig -> {
       CacheElement foundElement =
           findCustomCacheElement(cacheConfig, element.getId(), element.getClass());
@@ -87,7 +71,6 @@ public interface ClusterConfigurationService {
 
   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) {
@@ -100,13 +83,11 @@ public interface ClusterConfigurationService {
 
   default <T extends CacheElement> T getCustomRegionElement(String group, String regionPath,
       String id, Class<T> 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) {
@@ -127,7 +108,6 @@ public interface ClusterConfigurationService {
 
   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) {
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 459d6e3..7af4e0f 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
@@ -74,6 +74,7 @@ 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.XSDRootElement;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.ClusterConfigurationService;
 import org.apache.geode.distributed.DistributedLockService;
@@ -90,6 +91,7 @@ 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;
+import org.apache.geode.management.internal.cli.util.ClasspathScanLoadHelper;
 import org.apache.geode.management.internal.configuration.callbacks.ConfigurationChangeListener;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.management.internal.configuration.domain.SharedConfigurationStatus;
@@ -140,14 +142,16 @@ public class InternalClusterConfigurationService implements ClusterConfiguration
   private JAXBService jaxbService;
 
   @TestingOnly
-  InternalClusterConfigurationService() {
+  InternalClusterConfigurationService(Class<?>... xsdClasses) {
     configDirPath = null;
     configDiskDirPath = null;
     cache = null;
-    jaxbService = new JAXBService();
+    jaxbService = new JAXBService(xsdClasses);
+    jaxbService.validateWithLocalCacheXSD();
   }
 
-  public InternalClusterConfigurationService(InternalCache cache) throws IOException {
+  public InternalClusterConfigurationService(InternalCache cache, Class<?>... xsdClasses)
+      throws IOException {
     this.cache = cache;
     Properties properties = cache.getDistributedSystem().getProperties();
     // resolve the cluster config dir
@@ -172,7 +176,16 @@ public class InternalClusterConfigurationService implements ClusterConfiguration
     this.configDiskDirPath = FilenameUtils.concat(clusterConfigRootDir, configDiskDirName);
     this.sharedConfigLockingService = getSharedConfigLockService(cache.getDistributedSystem());
     this.status.set(SharedConfigurationStatus.NOT_STARTED);
-    jaxbService = new JAXBService();
+    if (xsdClasses != null && xsdClasses.length > 0) {
+      this.jaxbService = new JAXBService(xsdClasses);
+    }
+    // else, scan the classpath to find all the classes annotated with XSDRootElement
+    else {
+      Set<Class<?>> scannedClasses = ClasspathScanLoadHelper
+          .scanClasspathForAnnotation(XSDRootElement.class, "org.apache.geode");
+      this.jaxbService = new JAXBService(scannedClasses.toArray(new Class[scannedClasses.size()]));
+    }
+    jaxbService.validateWithLocalCacheXSD();
   }
 
   /**
@@ -824,12 +837,6 @@ public class InternalClusterConfigurationService implements ClusterConfiguration
   }
 
   @Override
-  public void registerBindClassWithSchema(Class clazz, String namespaceAndLocation) {
-    jaxbService.registerBindClassWithSchema(clazz, namespaceAndLocation);
-  }
-
-
-  @Override
   public CacheConfig getCacheConfig(String group) {
     if (group == null) {
       group = CLUSTER_CONFIG;
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
index 1c85d53..1422ec9 100644
--- 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
@@ -20,10 +20,8 @@ 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.Arrays;
 import java.util.Objects;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 import javax.xml.XMLConstants;
@@ -33,26 +31,36 @@ import javax.xml.bind.Unmarshaller;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
+import org.apache.commons.lang.StringUtils;
 import org.xml.sax.SAXException;
 
-import org.apache.geode.cache.configuration.CacheConfig;
+import org.apache.geode.cache.configuration.XSDRootElement;
 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 JAXBService(Class<?>... xsdRootClasses) {
+    try {
+      JAXBContext jaxbContext = JAXBContext.newInstance(xsdRootClasses);
+      marshaller = jaxbContext.createMarshaller();
+      unmarshaller = jaxbContext.createUnmarshaller();
+      marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+
+      String schemas = Arrays.stream(xsdRootClasses).map(c -> {
+        XSDRootElement element = c.getAnnotation(XSDRootElement.class);
+        if (element != null && StringUtils.isNotEmpty(element.namespace())
+            && StringUtils.isNotEmpty(element.schemaLocation())) {
+          return (element.namespace() + " " + element.schemaLocation());
+        }
+        return null;
+      }).filter(Objects::nonNull).collect(Collectors.joining(" "));
+
+      marshaller.setProperty(Marshaller.JAXB_SCHEMA_LOCATION, schemas);
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage(), e);
+    }
   }
 
   public void validateWith(URL url) {
@@ -66,52 +74,11 @@ public class JAXBService {
     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 void validateWithLocalCacheXSD() {
+    // 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 String marshall(Object object) {
@@ -131,8 +98,4 @@ public class JAXBService {
       throw new RuntimeException(e.getMessage(), e);
     }
   }
-
-  public void registerBindClasses(Class clazz) {
-    registerBindClassWithSchema(clazz, null);
-  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ClasspathScanLoadHelper.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ClasspathScanLoadHelper.java
index 35b0422..08e1170 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ClasspathScanLoadHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ClasspathScanLoadHelper.java
@@ -22,6 +22,7 @@ import java.util.Set;
 
 import io.github.lukehutch.fastclasspathscanner.FastClasspathScanner;
 
+
 /**
  * Utility class to scan class-path & load classes.
  *
@@ -38,6 +39,14 @@ public class ClasspathScanLoadHelper {
         .collect(toSet());
   }
 
+  public static Set<Class<?>> scanClasspathForAnnotation(Class<?> annotation,
+      String... packagesToScan) {
+    Set<Class<?>> classesWithAnnotation = new HashSet<>();
+    new FastClasspathScanner(packagesToScan)
+        .matchClassesWithAnnotation(annotation, classesWithAnnotation::add).scan();
+    return classesWithAnnotation;
+  }
+
   private static boolean isInstantiable(Class<?> klass) {
     int modifiers = klass.getModifiers();
 
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 72cb4f8..2193442 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
@@ -45,7 +45,7 @@ public class CacheConfigIntegrationTest {
     xmlFile = temporaryFolder.newFile("cache.xml");
     CacheConfig cacheConfig = new CacheConfig();
     cacheConfig.setVersion("1.0");
-    JAXBService service = new JAXBService();
+    JAXBService service = new JAXBService(CacheConfig.class);
     String xml = service.marshall(cacheConfig);
     FileUtils.writeStringToFile(xmlFile, xml, "UTF-8");
 
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 e18bda4..8403a97 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
@@ -52,8 +52,9 @@ public class InternalClusterConfigurationServiceTest {
 
   @Before
   public void setUp() throws Exception {
-    service = spy(InternalClusterConfigurationService.class);
-    service2 = spy(InternalClusterConfigurationService.class);
+    service = spy(new InternalClusterConfigurationService(CacheConfig.class, ElementOne.class,
+        ElementTwo.class));
+    service2 = spy(new InternalClusterConfigurationService(CacheConfig.class));
     configuration = new Configuration("cluster");
     doReturn(configuration).when(service).getConfiguration(any());
     doReturn(configuration).when(service2).getConfiguration(any());
@@ -115,6 +116,7 @@ public class InternalClusterConfigurationServiceTest {
   }
 
   @Test
+  // in case a locator in the cluster doesn't have the plugin installed
   public void xmlWithCustomElementsCanBeUnMarshalledByAnotherService() {
     service.saveCustomCacheElement("cluster", new ElementOne("one"));
     service.saveCustomCacheElement("cluster", new ElementTwo("two"));
@@ -122,11 +124,11 @@ public class InternalClusterConfigurationServiceTest {
     String prettyXml = configuration.getCacheXmlContent();
     System.out.println(prettyXml);
 
-    // the xml is sent to another locator, and can interpreted ocrrectly
+    // the xml is sent to another locator with no such plugin installed, it can be parsed
+    // but the element couldn't be recognized by the locator without the plugin
     service2.updateCacheConfig("cluster", cc -> cc);
-
     ElementOne elementOne = service2.getCustomCacheElement("cluster", "one", ElementOne.class);
-    assertThat(elementOne.getId()).isEqualTo("one");
+    assertThat(elementOne).isNull();
 
     String uglyXml = configuration.getCacheXmlContent();
     System.out.println(uglyXml);
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
index 0593f19..9f0cc7a 100644
--- 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
@@ -19,7 +19,6 @@ 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;
 
@@ -48,8 +47,11 @@ public class JAXBServiceTest {
 
   @Before
   public void setUp() throws Exception {
-    service = spy(new JAXBService());
-    service2 = spy(new JAXBService());
+    service = new JAXBService(CacheConfig.class, ElementOne.class, ElementTwo.class);
+    service.validateWithLocalCacheXSD();
+
+    service2 = new JAXBService(CacheConfig.class);
+    service2.validateWithLocalCacheXSD();
   }
 
   @Test
@@ -83,7 +85,6 @@ public class JAXBServiceTest {
     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
@@ -96,15 +97,13 @@ public class JAXBServiceTest {
     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 = service2.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);
@@ -120,18 +119,15 @@ public class JAXBServiceTest {
   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(0)).isNotInstanceOf(ElementOne.class);
     assertThat(elements.get(1)).isNotInstanceOf(ElementTwo.class);
 
     String uglyXml = service2.marshall(cacheConfig);

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