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 2017/10/10 15:36:02 UTC

[geode] branch develop updated: GEODE-3785: correctly update the schema version, namespace and lcoati… (#889)

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 0719e2b  GEODE-3785: correctly update the schema version, namespace and lcoati… (#889)
0719e2b is described below

commit 0719e2b23855dbcd438392b54957677aff895798
Author: jinmeiliao <ji...@pivotal.io>
AuthorDate: Tue Oct 10 08:35:49 2017 -0700

    GEODE-3785: correctly update the schema version, namespace and lcoati… (#889)
    
    * GEODE-3785: correctly update the schema version, namespace and lcoation when importing old schema
    
    * Refactor and update tests
    
    * fix test failure
---
 .../geode/internal/cache/xmlcache/CacheXml.java    |   2 +-
 .../cli/functions/RegionCreateFunction.java        |   4 +-
 .../configuration/domain/CacheElement.java         |  26 +-
 .../internal/configuration/utils/XmlUtils.java     | 143 +++-------
 .../ImportOldClusterConfigDUnitTest.java           | 104 +++++++
 .../configuration/utils/XmlUtilsJUnitTest.java     | 306 +++++++++++++++------
 .../management/internal/configuration/cluster8.xml |  25 ++
 .../configuration/domain/CacheElementJUnitTest.xml |  16 ++
 ...LocationMapMapOfStringListOfStringAttribute.xml |  10 -
 gradle/rat.gradle                                  |   1 -
 10 files changed, 425 insertions(+), 212 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXml.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXml.java
index 2dd6533..4cca8b0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXml.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXml.java
@@ -111,7 +111,7 @@ public abstract class CacheXml implements EntityResolver2, ErrorHandler {
    * 
    * @since GemFire 8.1
    */
-  protected static final String SCHEMA_8_1_LOCATION =
+  public static final String SCHEMA_8_1_LOCATION =
       "http://schema.pivotal.io/gemfire/cache/cache-8.1.xsd";
   /**
    * The location of the Geode 1.0 schema file.
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
index 4b205ce..69a8826 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
@@ -33,7 +33,7 @@ import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.RegionExistsException;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.cache.execute.FunctionAdapter;
+import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.ResultSender;
 import org.apache.geode.compression.Compressor;
@@ -53,7 +53,7 @@ import org.apache.geode.management.internal.configuration.domain.XmlEntity;
  *
  * @since GemFire 7.0
  */
-public class RegionCreateFunction extends FunctionAdapter implements InternalEntity {
+public class RegionCreateFunction implements Function, InternalEntity {
 
   private static final Logger logger = LogService.getLogger();
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/domain/CacheElement.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/domain/CacheElement.java
index 97f48a5..63f64d1 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/domain/CacheElement.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/domain/CacheElement.java
@@ -107,8 +107,12 @@ public class CacheElement {
    */
   public static LinkedHashMap<String, CacheElement> buildElementMap(final Document doc)
       throws IOException, XPathExpressionException, SAXException, ParserConfigurationException {
-    final Map<String, List<String>> schemaLocationMap =
-        XmlUtils.buildSchemaLocationMap(getAttribute(doc.getFirstChild(),
+    Node cacheNode = doc.getFirstChild();
+    if ("#comment".equals(cacheNode.getNodeName())) {
+      cacheNode = cacheNode.getNextSibling();
+    }
+    final Map<String, String> schemaLocationMap =
+        XmlUtils.buildSchemaLocationMap(getAttribute(cacheNode,
             W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, W3C_XML_SCHEMA_INSTANCE_NS_URI));
 
     final LinkedHashMap<String, CacheElement> elementMap = new LinkedHashMap<>();
@@ -131,23 +135,19 @@ public class CacheElement {
    * @throws IOException if unable to open {@link InputSource}.
    * @since GemFire 8.1
    */
-  private static InputSource resolveSchema(final Map<String, List<String>> schemaLocationMap,
+  private static InputSource resolveSchema(final Map<String, String> schemaLocationMap,
       String namespaceUri) throws IOException {
     final EntityResolver2 entityResolver = new CacheXmlParser();
 
     InputSource inputSource = null;
 
     // Try loading schema from locations until we find one.
-    final List<String> locations = schemaLocationMap.get(namespaceUri);
-    for (final String location : locations) {
-      try {
-        inputSource = entityResolver.resolveEntity(null, location);
-        if (null != inputSource) {
-          break;
-        }
-      } catch (final SAXException e) {
-        // ignore
-      }
+    final String location = schemaLocationMap.get(namespaceUri);
+
+    try {
+      inputSource = entityResolver.resolveEntity(null, location);
+    } catch (final SAXException e) {
+      // ignore
     }
 
     if (null == inputSource) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/XmlUtils.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/XmlUtils.java
index 63aa1ec..8d9f445 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/XmlUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/XmlUtils.java
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.management.internal.configuration.utils;
 
-import static javax.xml.XMLConstants.NULL_NS_URI;
 import static javax.xml.XMLConstants.W3C_XML_SCHEMA_INSTANCE_NS_URI;
 import static org.apache.geode.management.internal.configuration.utils.XmlConstants.W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION;
 import static org.apache.geode.management.internal.configuration.utils.XmlConstants.W3C_XML_SCHEMA_INSTANCE_PREFIX;
@@ -23,12 +22,10 @@ import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
 import java.io.StringWriter;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.StringTokenizer;
@@ -279,25 +276,9 @@ public class XmlUtils {
    * @return {@link Map} of schema namespace URIs to location URLs.
    * @since GemFire 8.1
    */
-  public static Map<String, List<String>> buildSchemaLocationMap(final String schemaLocation) {
-    return buildSchemaLocationMap(new HashMap<>(), schemaLocation);
-  }
-
-  /**
-   * Build schema location map of schemas used in given <code>schemaLocationAttribute</code> and
-   * adds them to the given <code>schemaLocationMap</code>.
-   * 
-   * @see <a href="http://www.w3.org/TR/xmlschema-0/#schemaLocation">XML Schema Part 0: Primer
-   *      Second Edition | 5.6 schemaLocation</a>
-   * 
-   * @param schemaLocationMap {@link Map} to add schema locations to.
-   * @param schemaLocation attribute value to build schema location map from.
-   * @return {@link Map} of schema namespace URIs to location URLs.
-   * @since GemFire 8.1
-   */
-  static Map<String, List<String>> buildSchemaLocationMap(
-      Map<String, List<String>> schemaLocationMap, final String schemaLocation) {
-    if (null == schemaLocation || schemaLocation.isEmpty()) {
+  public static Map<String, String> buildSchemaLocationMap(final String schemaLocation) {
+    Map<String, String> schemaLocationMap = new HashMap<>();
+    if (StringUtils.isBlank(schemaLocation)) {
       return schemaLocationMap;
     }
 
@@ -305,14 +286,7 @@ public class XmlUtils {
     while (st.hasMoreElements()) {
       final String ns = st.nextToken();
       final String loc = st.nextToken();
-      List<String> locs = schemaLocationMap.get(ns);
-      if (null == locs) {
-        locs = new ArrayList<>();
-        schemaLocationMap.put(ns, locs);
-      }
-      if (!locs.contains(loc)) {
-        locs.add(loc);
-      }
+      schemaLocationMap.put(ns, loc);
     }
 
     return schemaLocationMap;
@@ -490,7 +464,6 @@ public class XmlUtils {
    * @throws ParserConfigurationException
    * @since GemFire 8.1
    */
-  // UnitTest SharedConfigurationTest.testCreateAndUpgradeDocumentFromXml()
   public static Document upgradeSchema(Document document, final String namespaceUri,
       final String schemaLocation, String schemaVersion)
       throws XPathExpressionException, ParserConfigurationException {
@@ -506,60 +479,43 @@ public class XmlUtils {
 
     if (null != document.getDoctype()) {
       Node root = document.getDocumentElement();
-
       Document copiedDocument = getDocumentBuilder().newDocument();
       Node copiedRoot = copiedDocument.importNode(root, true);
       copiedDocument.appendChild(copiedRoot);
-
       document = copiedDocument;
     }
 
     final Element root = document.getDocumentElement();
+    // since root is the cache element, then this oldNamespace will be the cache's namespaceURI
+    String oldNamespaceUri = root.getNamespaceURI();
 
-    final Map<String, String> namespacePrefixMap = buildNamespacePrefixMap(root);
-
-    // Add CacheXml namespace if missing.
-    String cachePrefix = namespacePrefixMap.get(namespaceUri);
-    if (null == cachePrefix) {
-      // Default to null prefix.
-      cachePrefix = NULL_NS_URI;
-      // Move all into new namespace
-      changeNamespace(root, NULL_NS_URI, namespaceUri);
-      namespacePrefixMap.put(namespaceUri, cachePrefix);
+    // update the namespace
+    if (!namespaceUri.equals(oldNamespaceUri)) {
+      changeNamespace(root, oldNamespaceUri, namespaceUri);
     }
 
-    // Add schema instance namespace if missing.
-    String xsiPrefix = namespacePrefixMap.get(W3C_XML_SCHEMA_INSTANCE_NS_URI);
-    if (null == xsiPrefix) {
+    // update the version
+    root.setAttribute("version", schemaVersion);
+
+    // update the schemaLocation attribute
+    Node schemaLocationAttr = root.getAttributeNodeNS(W3C_XML_SCHEMA_INSTANCE_NS_URI,
+        W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION);
+    String xsiPrefix = findPrefix(root, W3C_XML_SCHEMA_INSTANCE_NS_URI);;
+    Map<String, String> uriToLocation = new HashMap<>();
+    if (schemaLocationAttr != null) {
+      uriToLocation = buildSchemaLocationMap(schemaLocationAttr.getNodeValue());
+    } else if (xsiPrefix == null) {
+      // this namespace is not defined yet, define it
       xsiPrefix = W3C_XML_SCHEMA_INSTANCE_PREFIX;
       root.setAttribute("xmlns:" + xsiPrefix, W3C_XML_SCHEMA_INSTANCE_NS_URI);
-      namespacePrefixMap.put(W3C_XML_SCHEMA_INSTANCE_NS_URI, xsiPrefix);
     }
 
-    // Create schemaLocation attribute if missing.
-    final String schemaLocationAttribute = getAttribute(root,
-        W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, W3C_XML_SCHEMA_INSTANCE_NS_URI);
-
-    // Update schemaLocation for namespace.
-    final Map<String, List<String>> schemaLocationMap =
-        buildSchemaLocationMap(schemaLocationAttribute);
-    List<String> schemaLocations = schemaLocationMap.get(namespaceUri);
-    if (null == schemaLocations) {
-      schemaLocations = new ArrayList<>();
-      schemaLocationMap.put(namespaceUri, schemaLocations);
-    }
-    schemaLocations.clear();
-    schemaLocations.add(schemaLocation);
-    String schemaLocationValue = getSchemaLocationValue(schemaLocationMap);
-    root.setAttributeNS(W3C_XML_SCHEMA_INSTANCE_NS_URI,
-        xsiPrefix + ":" + W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, schemaLocationValue);
+    uriToLocation.remove(oldNamespaceUri);
+    uriToLocation.put(namespaceUri, schemaLocation);
 
-    // Set schema version
-    if (cachePrefix == null || cachePrefix.isEmpty()) {
-      root.setAttribute("version", schemaVersion);
-    } else {
-      root.setAttributeNS(namespaceUri, cachePrefix + ":version", schemaVersion);
-    }
+    root.setAttributeNS(W3C_XML_SCHEMA_INSTANCE_NS_URI,
+        xsiPrefix + ":" + W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION,
+        getSchemaLocationValue(uriToLocation));
 
     return document;
   }
@@ -575,51 +531,34 @@ public class XmlUtils {
    * @param schemaLocationMap {@link Map} to get schema locations from.
    * @since GemFire 8.1
    */
-  private static String getSchemaLocationValue(final Map<String, List<String>> schemaLocationMap) {
+  private static String getSchemaLocationValue(final Map<String, String> schemaLocationMap) {
     final StringBuilder sb = new StringBuilder();
-    for (final Map.Entry<String, List<String>> entry : schemaLocationMap.entrySet()) {
-      for (final String schemaLocation : entry.getValue()) {
-        if (sb.length() > 0) {
-          sb.append(' ');
-        }
-        sb.append(entry.getKey()).append(' ').append(schemaLocation);
+    for (final Map.Entry<String, String> entry : schemaLocationMap.entrySet()) {
+      if (sb.length() > 0) {
+        sb.append(' ');
       }
+      sb.append(entry.getKey()).append(' ').append(entry.getValue());
     }
     return sb.toString();
   }
 
-  /**
-   * Build {@link Map} of namespace URIs to prefixes.
-   * 
-   * @param root {@link Element} to get namespaces and prefixes from.
-   * @return {@link Map} of namespace URIs to prefixes.
-   * @since GemFire 8.1
-   */
-  private static Map<String, String> buildNamespacePrefixMap(final Element root) {
-    final HashMap<String, String> namespacePrefixMap = new HashMap<>();
-
-    // Look for all of the attributes of cache that start with
-    // xmlns
+  static String findPrefix(final Element root, final String namespaceUri) {
+    // Look for all of the attributes of cache that start with xmlns
     NamedNodeMap attributes = root.getAttributes();
     for (int i = 0; i < attributes.getLength(); i++) {
       Node item = attributes.item(i);
       if (item.getNodeName().startsWith("xmlns")) {
-        // Anything after the colon is the prefix
-        // eg xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-        // has a prefix of xsi
-        String[] splitName = item.getNodeName().split(":");
-        String prefix;
-        if (splitName.length > 1) {
-          prefix = splitName[1];
-        } else {
-          prefix = "";
+        if (item.getNodeValue().equals(namespaceUri)) {
+          String[] splitName = item.getNodeName().split(":");
+          if (splitName.length > 1) {
+            return splitName[1];
+          } else {
+            return "";
+          }
         }
-        String uri = item.getTextContent();
-        namespacePrefixMap.put(uri, prefix);
       }
     }
-
-    return namespacePrefixMap;
+    return null;
   }
 
   /**
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ImportOldClusterConfigDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ImportOldClusterConfigDUnitTest.java
new file mode 100644
index 0000000..55a1cb7
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ImportOldClusterConfigDUnitTest.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.configuration;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.nio.file.Path;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.management.internal.configuration.utils.ZipUtils;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.GfshShellConnectionRule;
+
+@Category(DistributedTest.class)
+public class ImportOldClusterConfigDUnitTest {
+  @Rule
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private Path zipFile;
+  private MemberVM locator, server;
+
+  @Before
+  public void before() throws Exception {
+    // create the cc.zip that contains the 8.1 version cache.xml
+    File ccDir = tempFolder.newFolder("cluster_config");
+    File clusterDir = new File(ccDir, "cluster");
+    clusterDir.mkdir();
+
+    FileUtils.copyURLToFile(this.getClass().getResource("cluster8.xml"),
+        new File(clusterDir, "cluster.xml"));
+    zipFile = new File(tempFolder.getRoot(), "cc.zip").toPath();
+
+    ZipUtils.zipDirectory(ccDir.toPath(), zipFile);
+  }
+
+  @Test
+  public void importOldConfigThenCreateRegionCorruptsCachXml() throws Exception {
+    locator = lsRule.startLocatorVM(0);
+
+    gfsh.connectAndVerify(locator);
+
+    gfsh.executeAndVerifyCommand(
+        "import cluster-configuration --zip-file-name=" + zipFile.toString());
+
+    server = lsRule.startServerVM(1, locator.getPort());
+
+    server.invoke(ImportOldClusterConfigDUnitTest::regionOneExists);
+
+    gfsh.executeAndVerifyCommand("create region --name=two --type=REPLICATE");
+    server.invoke(ImportOldClusterConfigDUnitTest::regionOneExists);
+    server.invoke(ImportOldClusterConfigDUnitTest::regionTwoExists);
+
+    lsRule.stopVM(1);
+
+    server = lsRule.startServerVM(1, locator.getPort());
+    server.invoke(ImportOldClusterConfigDUnitTest::regionOneExists);
+    server.invoke(ImportOldClusterConfigDUnitTest::regionTwoExists);
+  }
+
+  private static void regionOneExists() {
+    regionExists("one");
+  }
+
+  private static void regionTwoExists() {
+    regionExists("two");
+  }
+
+  private static void regionExists(String regionName) {
+    Cache cache = LocatorServerStartupRule.serverStarter.getCache();
+    assertThat(cache).isNotNull();
+    Region<Object, Object> one = cache.getRegion(regionName);
+    assertThat(one).isNotNull();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.java
index c5bdf07..4993e5f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.java
@@ -14,30 +14,33 @@
  */
 package org.apache.geode.management.internal.configuration.utils;
 
-import static org.apache.geode.management.internal.configuration.utils.XmlConstants.*;
-import static javax.xml.XMLConstants.*;
-import static org.junit.Assert.*;
+import static javax.xml.XMLConstants.W3C_XML_SCHEMA_INSTANCE_NS_URI;
+import static org.apache.geode.internal.cache.xmlcache.CacheXml.GEMFIRE_NAMESPACE;
+import static org.apache.geode.internal.cache.xmlcache.CacheXml.GEODE_NAMESPACE;
+import static org.apache.geode.internal.cache.xmlcache.CacheXml.LATEST_SCHEMA_LOCATION;
+import static org.apache.geode.management.internal.configuration.utils.XmlConstants.W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.fail;
 
-import java.io.IOException;
 import java.io.InputStreamReader;
-import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import javax.xml.XMLConstants;
-import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.xpath.XPathExpressionException;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
 
+import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.management.internal.configuration.domain.XmlEntity;
 import org.apache.geode.management.internal.configuration.utils.XmlUtils.XPathContext;
 import org.apache.geode.test.junit.categories.UnitTest;
@@ -45,12 +48,17 @@ import org.apache.geode.test.junit.categories.UnitTest;
 /**
  * Unit tests for {@link XmlUtils}. See Also {@link XmlUtilsAddNewNodeJUnitTest} for tests related
  * to {@link XmlUtils#addNewNode(Document, XmlEntity)}
- *
+ * 
  * @since GemFire 8.1
  */
 @Category(UnitTest.class)
 public class XmlUtilsJUnitTest {
 
+  private static final String GEODE_SCHEMA_LOCATION =
+      GEODE_NAMESPACE + " " + LATEST_SCHEMA_LOCATION;
+  private static final String GEMFIRE_SCHEMA_LOCATION =
+      GEMFIRE_NAMESPACE + " " + CacheXml.SCHEMA_8_1_LOCATION;
+
   /**
    * Test method for {@link XmlUtils#buildSchemaLocationMap(String)}.
    */
@@ -60,84 +68,26 @@ public class XmlUtilsJUnitTest {
         .getResourceAsStream("XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml")));
     final String schemaLocationAttribute = XmlUtils.getAttribute(doc.getDocumentElement(),
         W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, W3C_XML_SCHEMA_INSTANCE_NS_URI);
-    final Map<String, List<String>> schemaLocationMap =
+    final Map<String, String> schemaLocationMap =
         XmlUtils.buildSchemaLocationMap(schemaLocationAttribute);
 
     assertNotNull(schemaLocationMap);
     assertEquals(2, schemaLocationMap.size());
 
-    final List<String> locations1 = schemaLocationMap.get("http://geode.apache.org/schema/cache");
+    final String locations1 = schemaLocationMap.get("http://geode.apache.org/schema/cache");
     assertNotNull(locations1);
-    assertEquals(1, locations1.size());
-    assertEquals("http://geode.apache.org/schema/cache/cache-1.0.xsd", locations1.get(0));
+    assertEquals("http://geode.apache.org/schema/cache/cache-1.0.xsd", locations1);
 
-    final List<String> locations2 = schemaLocationMap
+    final String locations2 = schemaLocationMap
         .get("urn:java:org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest");
     assertNotNull(locations2);
-    assertEquals(2, locations2.size());
-    assertEquals(
-        "classpath:/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.xsd",
-        locations2.get(0));
-    assertEquals("XmlUtilsJUnitTest.xsd", locations2.get(1));
+    assertEquals("XmlUtilsJUnitTest.xsd", locations2);
 
-    final List<String> locations3 = schemaLocationMap.get("urn:__does_not_exist__");
+    final String locations3 = schemaLocationMap.get("urn:__does_not_exist__");
     assertNull(locations3);
   }
 
-  /**
-   * Test method for {@link XmlUtils#buildSchemaLocationMap(Map, String)}.
-   */
-  @Test
-  public void testBuildSchemaLocationMapMapOfStringListOfStringAttribute() throws Exception {
-    Map<String, List<String>> schemaLocationMap = new HashMap<>();
-
-    final Document doc1 = XmlUtils.createDocumentFromReader(new InputStreamReader(getClass()
-        .getResourceAsStream("XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml")));
-    final String schemaLocationAttribute1 = XmlUtils.getAttribute(doc1.getDocumentElement(),
-        W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, W3C_XML_SCHEMA_INSTANCE_NS_URI);
-    schemaLocationMap =
-        XmlUtils.buildSchemaLocationMap(schemaLocationMap, schemaLocationAttribute1);
-
-    final Document doc2 =
-        XmlUtils.createDocumentFromReader(new InputStreamReader(getClass().getResourceAsStream(
-            "XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml")));
-    final String schemaLocationAttribute2 = XmlUtils.getAttribute(doc2.getDocumentElement(),
-        W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, W3C_XML_SCHEMA_INSTANCE_NS_URI);
-    schemaLocationMap =
-        XmlUtils.buildSchemaLocationMap(schemaLocationMap, schemaLocationAttribute2);
-
-    assertNotNull(schemaLocationMap);
-    assertEquals(3, schemaLocationMap.size());
-
-    final List<String> locations1 = schemaLocationMap.get("http://geode.apache.org/schema/cache");
-    assertNotNull(locations1);
-    assertEquals(2, locations1.size());
-    assertEquals("http://geode.apache.org/schema/cache/cache-1.0.xsd", locations1.get(0));
-    assertEquals("cache-1.0.xsd", locations1.get(1));
-
-    final List<String> locations2 = schemaLocationMap
-        .get("urn:java:org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest");
-    assertNotNull(locations2);
-    assertEquals(2, locations2.size());
-    assertEquals(
-        "classpath:/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.xsd",
-        locations2.get(0));
-    assertEquals("XmlUtilsJUnitTest.xsd", locations2.get(1));
-
-    final List<String> locations3 = schemaLocationMap.get("urn:__does_not_exist__");
-    assertNull(locations3);
 
-    final List<String> locations4 = schemaLocationMap.get(
-        "urn:java:org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest2");
-    assertNotNull(locations4);
-    assertEquals(1, locations4.size());
-    assertEquals("XmlUtilsJUnitTest2.xsd", locations4.get(0));
-  }
-
-  /**
-   * Test method for {@link XmlUtils#buildSchemaLocationMap(Map, String)}. Asserts map is empty if
-   * schemaLocation attribute is <code>null</code>.
-   */
   @Test
   public void testBuildSchemaLocationMapNullAttribute() throws Exception {
     final Document doc = XmlUtils.createDocumentFromReader(new InputStreamReader(getClass()
@@ -145,14 +95,13 @@ public class XmlUtilsJUnitTest {
     final String schemaLocationAttribute = XmlUtils.getAttribute(doc.getDocumentElement(),
         W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, W3C_XML_SCHEMA_INSTANCE_NS_URI);
     assertNull(schemaLocationAttribute);
-    final Map<String, List<String>> schemaLocationMap =
+    final Map<String, String> schemaLocationMap =
         XmlUtils.buildSchemaLocationMap(schemaLocationAttribute);
     assertEquals(0, schemaLocationMap.size());
   }
 
   /**
-   * Test method for {@link XmlUtils#buildSchemaLocationMap(Map, String)}. Asserts map is empty if
-   * schemaLocation attribute is empty.
+   * Asserts map is empty if schemaLocation attribute is empty.
    */
   @Test
   public void testBuildSchemaLocationMapEmptyAttribute() throws Exception {
@@ -161,13 +110,13 @@ public class XmlUtilsJUnitTest {
     final String schemaLocationAttribute = XmlUtils.getAttribute(doc.getDocumentElement(),
         W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, W3C_XML_SCHEMA_INSTANCE_NS_URI);
     assertNotNull(schemaLocationAttribute);
-    final Map<String, List<String>> schemaLocationMap =
+    final Map<String, String> schemaLocationMap =
         XmlUtils.buildSchemaLocationMap(schemaLocationAttribute);
     assertEquals(0, schemaLocationMap.size());
   }
 
   /**
-   * Test method for {@link XmlUtils#querySingleElement(Node, String, XPathContext)}.
+   *
    */
   @Test
   public void testQuerySingleElement() throws Exception {
@@ -215,7 +164,7 @@ public class XmlUtilsJUnitTest {
    * Test method for {@link XmlUtils#changeNamespace(Node, String, String)}.
    */
   @Test
-  public void testChangeNamespace() throws Exception {
+  public void testChangeNamespaceWithNoRootNamespace() throws Exception {
     Document doc = XmlUtils.getDocumentBuilder().newDocument();
     Element root = doc.createElement("root");
     root = (Element) doc.appendChild(root);
@@ -239,19 +188,210 @@ public class XmlUtilsJUnitTest {
   }
 
   @Test
+  public void testChangeNamespaceWithExistingRootNamespace() throws Exception {
+    Document doc = XmlUtils.getDocumentBuilder().newDocument();
+
+    final String ns0 = "urn:namespace0";
+    Element root = doc.createElementNS(ns0, "root");
+    root = (Element) doc.appendChild(root);
+
+    final Element child = doc.createElementNS(ns0, "child");
+    root.appendChild(child);
+    final String ns2 = "urn:namespace2";
+    final Element childWithNamespace = doc.createElementNS(ns2, "childWithNamespace");
+    root.appendChild(childWithNamespace);
+    root.appendChild(doc.createTextNode("some text"));
+
+    assertEquals(ns0, root.getNamespaceURI());
+    assertEquals(ns0, child.getNamespaceURI());
+    assertEquals(ns2, childWithNamespace.getNamespaceURI());
+
+    final String ns1 = "urn:namespace1";
+    root = (Element) XmlUtils.changeNamespace(root, ns0, ns1);
+
+    assertEquals(ns1, root.getNamespaceURI());
+    assertEquals(ns1, root.getElementsByTagName("child").item(0).getNamespaceURI());
+    assertEquals(ns2, root.getElementsByTagName("childWithNamespace").item(0).getNamespaceURI());
+  }
+
+  @Test
   public void testCreateAndUpgradeDocumentFromXml() throws Exception {
-    Document doc = XmlUtils.createAndUpgradeDocumentFromXml(
-        IOUtils.toString(this.getClass().getResourceAsStream("SharedConfigurationJUnitTest.xml")));
+    Document doc = XmlUtils.createAndUpgradeDocumentFromXml(IOUtils.toString(
+        this.getClass().getResourceAsStream("SharedConfigurationJUnitTest.xml"), "UTF-8"));
 
     String schemaLocation = XmlUtils.getAttribute(doc.getDocumentElement(),
         W3C_XML_SCHEMA_INSTANCE_ATTRIBUTE_SCHEMA_LOCATION, W3C_XML_SCHEMA_INSTANCE_NS_URI);
 
     assertNotNull(schemaLocation);
-    assertEquals(CacheXml.GEODE_NAMESPACE + " " + CacheXml.LATEST_SCHEMA_LOCATION, schemaLocation);
+    assertEquals(GEODE_NAMESPACE + " " + CacheXml.LATEST_SCHEMA_LOCATION, schemaLocation);
 
     assertEquals(CacheXml.VERSION_LATEST,
         XmlUtils.getAttribute(doc.getDocumentElement(), "version"));
   }
 
+  private static String CLUSTER8_XML =
+      "<cache xsi:schemaLocation=\"http://schema.pivotal.io/gemfire/cache http://schema.pivotal.io/gemfire/cache/cache-8.1.xsd\"\n"
+          + "       version=\"8.1\"\n" + "       xmlns=\"http://schema.pivotal.io/gemfire/cache\"\n"
+          + "       xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\">\n"
+          + "    <region name=\"one\">\n"
+          + "        <region-attributes scope=\"distributed-ack\" data-policy=\"replicate\"/>\n"
+          + "    </region>\n" + "</cache>";
+
+  private static String CLUSTER9_XML = "<cache xmlns=\"http://geode.apache.org/schema/cache\"\n"
+      + "       xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"\n"
+      + "       xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\"\n"
+      + "       version=\"1.0\"></cache>";
+
+  @Test
+  public void testUpgradeSchemaFromGemfireNamespace() throws Exception {
+    Document doc = XmlUtils.createDocumentFromXml(CLUSTER8_XML);
+    Element oldRoot = doc.getDocumentElement();
+    assertThat(oldRoot.getAttribute(CacheXml.VERSION)).isEqualTo("8.1");
+    assertThat(oldRoot.getNamespaceURI()).isEqualTo(CacheXml.GEMFIRE_NAMESPACE);
+    assertThat(oldRoot.getAttribute("xsi:schemaLocation")).isEqualTo(GEMFIRE_SCHEMA_LOCATION);
 
+    String version = "1.0";
+    doc = XmlUtils.upgradeSchema(doc, GEODE_NAMESPACE, LATEST_SCHEMA_LOCATION, version);
+
+    Element root = doc.getDocumentElement();
+
+    assertThat(root.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(root.getAttribute(CacheXml.VERSION)).isEqualTo(version);
+    assertThat(root.getAttribute("xsi:schemaLocation")).isEqualTo(GEODE_SCHEMA_LOCATION);
+
+    Node regionNode = root.getElementsByTagName("region").item(0);
+    assertThat(regionNode.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+  }
+
+  @Test
+  public void testUpgradeSchemaFromOtherInvaidNS() throws Exception {
+    String xml = "<cache version=\"8.1\" xmlns=\"http://test.org/cache\"></cache>";
+    Document doc = XmlUtils.createDocumentFromXml(xml);
+
+    String version = "1.0";
+    doc = XmlUtils.upgradeSchema(doc, GEODE_NAMESPACE, LATEST_SCHEMA_LOCATION, version);
+
+    Element root = doc.getDocumentElement();
+    assertThat(root.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(root.getAttribute(CacheXml.VERSION)).isEqualTo(version);
+    assertThat(root.getAttribute("xsi:schemaLocation")).isEqualTo(GEODE_SCHEMA_LOCATION);
+  }
+
+  @Test
+  public void testUpgradeSchemaFromGemfireNamespaceWithNoLocation() throws Exception {
+    String xml = "<cache version=\"8.1\" xmlns=\"http://schema.pivotal.io/gemfire/cache\"></cache>";
+    Document doc = XmlUtils.createDocumentFromXml(xml);
+    Element oldRoot = doc.getDocumentElement();
+    assertThat(oldRoot.getAttribute(CacheXml.VERSION)).isEqualTo("8.1");
+    assertThat(oldRoot.getNamespaceURI()).isEqualTo(CacheXml.GEMFIRE_NAMESPACE);
+
+    String version = "1.0";
+    doc = XmlUtils.upgradeSchema(doc, GEODE_NAMESPACE, LATEST_SCHEMA_LOCATION, version);
+
+    Element root = doc.getDocumentElement();
+    assertThat(root.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(root.getAttribute(CacheXml.VERSION)).isEqualTo(version);
+    assertThat(root.getAttribute("xsi:schemaLocation")).isEqualTo(GEODE_SCHEMA_LOCATION);
+  }
+
+  @Test
+  public void testUpgradeSchemaFromGemfireWithCustomPrefix() throws Exception {
+    String xml = "<a:cache xmlns:a=\"http://schema.pivotal.io/gemfire/cache\">\n"
+        + "    <a:region name=\"one\">\n"
+        + "        <a:region-attributes scope=\"distributed-ack\" data-policy=\"replicate\"/>\n"
+        + "    </a:region>\n" + "</a:cache>";
+    Document doc = XmlUtils.createDocumentFromXml(xml);
+    Element oldRoot = doc.getDocumentElement();
+    assertThat(oldRoot.getNamespaceURI()).isEqualTo(CacheXml.GEMFIRE_NAMESPACE);
+
+    String version = "1.0";
+    doc = XmlUtils.upgradeSchema(doc, GEODE_NAMESPACE, LATEST_SCHEMA_LOCATION, version);
+
+    Element root = doc.getDocumentElement();
+
+    assertThat(root.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(root.getAttribute(CacheXml.VERSION)).isEqualTo(version);
+    assertThat(root.getAttribute("xsi:schemaLocation")).isEqualTo(GEODE_SCHEMA_LOCATION);
+
+    Node regionNode = root.getElementsByTagNameNS(GEODE_NAMESPACE, "region").item(0);
+    assertThat(regionNode.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+  }
+
+  @Test
+  public void testUpgradeVersionFromGeodeWithCustomPrefix() throws Exception {
+    String xml = "<a:cache xmlns:a=\"http://geode.apache.org/schema/cache\">\n"
+        + "    <a:region name=\"one\">\n"
+        + "        <a:region-attributes scope=\"distributed-ack\" data-policy=\"replicate\"/>\n"
+        + "    </a:region>\n" + "</a:cache>";
+    Document doc = XmlUtils.createDocumentFromXml(xml);
+    String schemaLocation2 = "http://geode.apache.org/schema/cache/cache-2.0.xsd";
+    String version = "2.0";
+    doc = XmlUtils.upgradeSchema(doc, GEODE_NAMESPACE, schemaLocation2, version);
+
+    Element root = doc.getDocumentElement();
+
+    assertThat(root.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(root.getAttribute(CacheXml.VERSION)).isEqualTo(version);
+    assertThat(root.getAttribute("xsi:schemaLocation"))
+        .isEqualTo(GEODE_NAMESPACE + " " + schemaLocation2);
+
+    Node regionNode = root.getElementsByTagNameNS(GEODE_NAMESPACE, "region").item(0);
+    assertThat(regionNode.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+  }
+
+  @Test
+  public void testUpgradeSchemaFromGeodeNamespace() throws Exception {
+    Document doc = XmlUtils.createDocumentFromXml(CLUSTER9_XML);
+    Element oldRoot = doc.getDocumentElement();
+    assertThat(oldRoot.getAttribute(CacheXml.VERSION)).isEqualTo("1.0");
+    assertThat(oldRoot.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(oldRoot.getAttribute("xsi:schemaLocation")).isEqualTo(GEODE_SCHEMA_LOCATION);
+
+    String version = "1.0";
+    doc = XmlUtils.upgradeSchema(doc, GEODE_NAMESPACE, LATEST_SCHEMA_LOCATION, version);
+    Element root = doc.getDocumentElement();
+
+    assertThat(root.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(root.getAttribute(CacheXml.VERSION)).isEqualTo(version);
+    assertThat(root.getAttribute("xsi:schemaLocation")).isEqualTo(GEODE_SCHEMA_LOCATION);
+  }
+
+  @Test
+  public void testUpgradeSchemaFromGeodeNamespaceToAnotherVersion() throws Exception {
+    Document doc = XmlUtils.createDocumentFromXml(CLUSTER9_XML);
+    Element oldRoot = doc.getDocumentElement();
+    assertThat(oldRoot.getAttribute(CacheXml.VERSION)).isEqualTo("1.0");
+    assertThat(oldRoot.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(oldRoot.getAttribute("xsi:schemaLocation")).isEqualTo(GEODE_SCHEMA_LOCATION);
+
+    String schemaLocation2 = "http://geode.apache.org/schema/cache/cache-2.0.xsd";
+    doc = XmlUtils.upgradeSchema(doc, GEODE_NAMESPACE, schemaLocation2, "2.0");
+    Element root = doc.getDocumentElement();
+
+    assertThat(root.getNamespaceURI()).isEqualTo(GEODE_NAMESPACE);
+    assertThat(root.getAttribute(CacheXml.VERSION)).isEqualTo("2.0");
+    assertThat(root.getAttribute("xsi:schemaLocation"))
+        .isEqualTo(GEODE_NAMESPACE + " " + schemaLocation2);
+  }
+
+  @Test
+  public void testUpgradeSchemaWithMultipNS() throws Exception {
+    String xml = "<cache xmlns=\"http://cache\"\n"
+        + "       xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"\n"
+        + "       xmlns:aop=\"http://aop\"\n" + "       version=\"8.1\"\n"
+        + "       xsi:schemaLocation=\"http://cache http://test.org/cache.xsd "
+        + "        http://aop http://test.org/aop.xsd\">\n" + "</cache>";
+    Document doc = XmlUtils.createDocumentFromXml(xml);
+
+    String version = "1.0";
+    String namespace = "http://geode.apache.org/schema/cache";
+    doc = XmlUtils.upgradeSchema(doc, GEODE_NAMESPACE, LATEST_SCHEMA_LOCATION, "1.0");
+
+    Element root = doc.getDocumentElement();
+
+    String expectedSchemaLocation = "http://aop http://test.org/aop.xsd " + GEODE_SCHEMA_LOCATION;
+    assertThat(root.getNamespaceURI()).isEqualTo(namespace);
+    assertThat(root.getAttribute(CacheXml.VERSION)).isEqualTo(version);
+    assertThat(root.getAttribute("xsi:schemaLocation")).isEqualTo(expectedSchemaLocation);
+  }
 }
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/cluster8.xml b/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/cluster8.xml
new file mode 100644
index 0000000..81c0dd3
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/cluster8.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<cache xsi:schemaLocation="http://schema.pivotal.io/gemfire/cache http://schema.pivotal.io/gemfire/cache/cache-8.1.xsd"
+       version="8.1"
+       xmlns="http://schema.pivotal.io/gemfire/cache"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <region name="one">
+        <region-attributes scope="distributed-ack" data-policy="replicate"/>
+    </region>
+</cache>
\ No newline at end of file
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/domain/CacheElementJUnitTest.xml b/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/domain/CacheElementJUnitTest.xml
index d995b91..e1ecbcf 100644
--- a/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/domain/CacheElementJUnitTest.xml
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/domain/CacheElementJUnitTest.xml
@@ -1,4 +1,20 @@
 <?xml version="1.0"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
 <cache
     xmlns="http://geode.apache.org/schema/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml b/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
deleted file mode 100644
index 8edf31a..0000000
--- a/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
+++ /dev/null
@@ -1,10 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<cache
-    xmlns="http://geode.apache.org/schema/cache"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-    xmlns:test="urn:java:org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest"
-    xsi:schemaLocation="http://geode.apache.org/schema/cache cache-1.0.xsd
-                        urn:java:org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest classpath:/org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest.xsd
-                        urn:java:org/apache/geode/management/internal/configuration/utils/XmlUtilsJUnitTest2 XmlUtilsJUnitTest2.xsd"
-    version="1.0">
-</cache>
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 729a283..b0012af 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -114,7 +114,6 @@ rat {
     'geode-assembly/cargo_logs/**',
 
     // these are test files that don't expect the first element to be a comment
-    'geode-core/src/test/resources/org/apache/geode/management/internal/configuration/domain/CacheElementJUnitTest.xml',
     'geode-core/src/test/resources/org/apache/geode/management/internal/configuration/utils/*.xml',
 
     '**/META-INF/**',

-- 
To stop receiving notification emails like this one, please contact
['"commits@geode.apache.org" <co...@geode.apache.org>'].