You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2020/12/11 04:27:58 UTC

[lucene-solr] branch master updated: SOLR-14827: Refactor schema loading to not use XPath (#1963)

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

noble pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new a95ce0d  SOLR-14827: Refactor schema loading to not use XPath (#1963)
a95ce0d is described below

commit a95ce0d4224539094dc602ba8afa1ff796009a2b
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Fri Dec 11 15:27:47 2020 +1100

    SOLR-14827: Refactor schema loading to not use XPath (#1963)
---
 solr/CHANGES.txt                                   |   2 +
 .../apache/solr/cloud/CloudConfigSetService.java   |  34 +++-
 .../apache/solr/cloud/ZkSolrResourceLoader.java    |   7 +-
 .../org/apache/solr/core/ConfigSetService.java     |  30 +++-
 .../java/org/apache/solr/core/XmlConfigFile.java   |  23 +--
 .../apache/solr/schema/FieldTypePluginLoader.java  |  83 ++++------
 .../java/org/apache/solr/schema/IndexSchema.java   | 175 ++++++++-------------
 .../org/apache/solr/schema/IndexSchemaFactory.java |  68 ++++++--
 .../org/apache/solr/schema/ManagedIndexSchema.java |   7 +-
 .../solr/schema/ManagedIndexSchemaFactory.java     |  17 +-
 .../java/org/apache/solr/schema/SchemaManager.java |   6 +-
 .../apache/solr/schema/ZkIndexSchemaReader.java    |  21 +--
 .../java/org/apache/solr/util/DOMConfigNode.java   |  89 +++++++++++
 .../java/org/apache/solr/util/DataConfigNode.java  | 132 ++++++++++++++++
 .../solr/util/plugin/AbstractPluginLoader.java     |  19 +--
 .../apache/solr/util/plugin/MapPluginLoader.java   |   6 +-
 .../solr/util/plugin/NamedListPluginLoader.java    |   4 +-
 .../bad-schema-daterangefield-instance-options.xml |   1 -
 .../solr/schema/TestManagedSchemaThreadSafety.java |   8 +-
 .../org/apache/solr/cluster/api/SimpleMap.java     |   5 +
 .../java/org/apache/solr/common/ConfigNode.java    | 104 ++++++++++++
 .../java/org/apache/solr/common/util/DOMUtil.java  | 107 ++++++++++---
 .../apache/solr/common/util/PropertiesUtil.java    |  10 +-
 23 files changed, 693 insertions(+), 265 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 13d81f3..c704930 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -221,6 +221,8 @@ Improvements
 * SOLR-14965: metrics: Adds two metrics to the SolrCloud Overseer: solr_metrics_overseer_stateUpdateQueueSize
   and solr_metrics_overseer_collectionWorkQueueSize with corresponding entries in the the Prometheus exporter's
   default/stock configuration.  (Saatchi Bhalla, Megan Carey, Andrzej BiaƂecki, David Smiley)
+  
+* SOLR-14827: Refactor schema loading to not use XPath (noble)
 
 * SOLR-14987: Reuse HttpSolrClient per node vs. one per Solr core when using CloudSolrStream (Timothy Potter)
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
index 644b49e..88fbe781 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
@@ -17,8 +17,13 @@
 package org.apache.solr.cloud;
 
 import java.lang.invoke.MethodHandles;
+import java.lang.ref.WeakReference;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.cloud.api.collections.CreateCollectionCmd;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkConfigManager;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -39,7 +44,7 @@ import org.slf4j.LoggerFactory;
  */
 public class CloudConfigSetService extends ConfigSetService {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+  private Map<String, ConfigCacheEntry> cache = new ConcurrentHashMap<>();
   private final ZkController zkController;
 
   public CloudConfigSetService(SolrResourceLoader loader, boolean shareSchema, ZkController zkController) {
@@ -47,6 +52,20 @@ public class CloudConfigSetService extends ConfigSetService {
     this.zkController = zkController;
   }
 
+  public void storeConfig(String resource, ConfigNode config, int znodeVersion) {
+    cache.put(resource, new ConfigCacheEntry(config, znodeVersion));
+  }
+
+  public ConfigNode getConfig(String resource, int znodeVersion) {
+    ConfigCacheEntry e = cache.get(resource);
+    if (e == null) return null;
+    ConfigNode configNode = e.configNode.get();
+    if (configNode == null) cache.remove(resource);
+    if (e.znodeVersion == znodeVersion) return configNode;
+    if (e.znodeVersion < znodeVersion) cache.remove(resource);
+    return null;
+  }
+
   @Override
   public SolrResourceLoader createCoreResourceLoader(CoreDescriptor cd) {
     final String colName = cd.getCollectionName();
@@ -111,4 +130,17 @@ public class CloudConfigSetService extends ConfigSetService {
   public String configSetName(CoreDescriptor cd) {
     return "configset " + cd.getConfigSet();
   }
+
+  private static class ConfigCacheEntry {
+    final WeakReference<ConfigNode> configNode;
+    final int znodeVersion;
+
+    private ConfigCacheEntry(ConfigNode configNode, int znodeVersion) {
+      this.configNode = new WeakReference<>(configNode);
+      this.znodeVersion = znodeVersion;
+    }
+  }
+  public SolrCloudManager getSolrCloudManager() {
+    return zkController.getSolrCloudManager();
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
index 0e17b5c..b52d6bb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
@@ -81,7 +81,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
         if (zkController.pathExists(file)) {
           Stat stat = new Stat();
           byte[] bytes = zkController.getZkClient().getData(file, null, stat, true);
-          return new ZkByteArrayInputStream(bytes, stat);
+          return new ZkByteArrayInputStream(bytes, file, stat);
         } else {
           //Path does not exists. We only retry for session expired exceptions.
           break;
@@ -126,11 +126,12 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
 
   public static class ZkByteArrayInputStream extends ByteArrayInputStream{
 
+    public final String fileName;
     private final Stat stat;
-    public ZkByteArrayInputStream(byte[] buf, Stat stat) {
+    public ZkByteArrayInputStream(byte[] buf, String fileName, Stat stat) {
       super(buf);
+      this.fileName = fileName;
       this.stat = stat;
-
     }
 
     public Stat getStat(){
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
index 1d45bc5..cdea9d9 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
@@ -18,6 +18,7 @@ package org.apache.solr.core;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -28,12 +29,23 @@ import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.solr.cloud.CloudConfigSetService;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.IndexSchemaFactory;
+import org.apache.solr.util.DOMConfigNode;
+import org.apache.solr.util.DataConfigNode;
+import org.apache.solr.util.SystemIdResolver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.ParserConfigurationException;
+
+import static org.apache.solr.schema.IndexSchema.SCHEMA;
+import static org.apache.solr.schema.IndexSchema.SLASH;
 
 /**
  * Service class used by the CoreContainer to load ConfigSets for use in SolrCore
@@ -135,14 +147,14 @@ public abstract class ConfigSetService {
         // note: luceneMatchVersion influences the schema
         String cacheKey = configSet + "/" + guessSchemaName + "/" + modVersion + "/" + solrConfig.luceneMatchVersion;
         return schemaCache.get(cacheKey,
-            (key) -> indexSchemaFactory.create(cdSchemaName, solrConfig));
+            (key) -> indexSchemaFactory.create(cdSchemaName, solrConfig, ConfigSetService.this));
       } else {
         log.warn("Unable to get schema modification version, configSet={} schema={}", configSet, guessSchemaName);
         // see explanation above; "guessSchema" is a guess
       }
     }
 
-    return indexSchemaFactory.create(cdSchemaName, solrConfig);
+    return indexSchemaFactory.create(cdSchemaName, solrConfig, this);
   }
 
   /**
@@ -186,6 +198,20 @@ public abstract class ConfigSetService {
    */
   public abstract String configSetName(CoreDescriptor cd);
 
+  public interface ConfigResource {
+
+    ConfigNode get() throws Exception;
+
+  }
+  public static ConfigNode getParsedSchema(InputStream is, SolrResourceLoader loader, String name) throws IOException, SAXException, ParserConfigurationException {
+    XmlConfigFile schemaConf = null;
+    InputSource inputSource = new InputSource(is);
+    inputSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
+    schemaConf = new XmlConfigFile(loader, SCHEMA, inputSource, SLASH + SCHEMA + SLASH, null);
+    return new DataConfigNode(new DOMConfigNode(schemaConf.getDocument().getDocumentElement()));
+
+  }
+
   /**
    * The Solr standalone version of ConfigSetService.
    *
diff --git a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
index 6d8cb92..5971eb5 100644
--- a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
+++ b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
@@ -20,11 +20,6 @@ import javax.xml.namespace.QName;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.transform.Transformer;
-import javax.xml.transform.TransformerException;
-import javax.xml.transform.TransformerFactory;
-import javax.xml.transform.dom.DOMResult;
-import javax.xml.transform.dom.DOMSource;
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
@@ -144,7 +139,7 @@ public class XmlConfigFile { // formerly simply "Config"
       db.setErrorHandler(xmllog);
       try {
         doc = db.parse(is);
-        origDoc = copyDoc(doc);
+        origDoc = doc;
       } finally {
         // some XML parsers are broken and don't close the byte stream (but they should according to spec)
         IOUtils.closeQuietly(is.getByteStream());
@@ -152,7 +147,7 @@ public class XmlConfigFile { // formerly simply "Config"
       if (substituteProps != null) {
         DOMUtil.substituteProperties(doc, getSubstituteProperties());
       }
-    } catch (ParserConfigurationException | SAXException | TransformerException e)  {
+    } catch (ParserConfigurationException | SAXException e)  {
       SolrException.log(log, "Exception during parsing file: " + name, e);
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
@@ -178,15 +173,7 @@ public class XmlConfigFile { // formerly simply "Config"
     return this.substituteProperties;
   }
 
-  private static Document copyDoc(Document doc) throws TransformerException {
-    TransformerFactory tfactory = TransformerFactory.newInstance();
-    Transformer tx = tfactory.newTransformer();
-    DOMSource source = new DOMSource(doc);
-    DOMResult result = new DOMResult();
-    tx.transform(source, result);
-    return (Document) result.getNode();
-  }
-  
+
   /**
    * @since solr 1.3
    */
@@ -236,10 +223,6 @@ public class XmlConfigFile { // formerly simply "Config"
     return getNode(path, doc, errifMissing);
   }
 
-  public Node getUnsubstitutedNode(String path, boolean errIfMissing) {
-    return getNode(path, origDoc, errIfMissing);
-  }
-
   public Node getNode(String path, Document doc, boolean errIfMissing) {
     XPath xpath = xpathFactory.newXPath();
     String xstr = normalize(path);
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
index b4f51ec..345a2cf 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
@@ -16,13 +16,10 @@
  */
 package org.apache.solr.schema;
 
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-import javax.xml.xpath.XPathFactory;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
@@ -33,6 +30,7 @@ import org.apache.lucene.analysis.TokenizerFactory;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.util.Version;
 import org.apache.solr.analysis.TokenizerChain;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.SolrClassLoader;
 import org.apache.solr.common.util.DOMUtil;
@@ -40,9 +38,6 @@ import org.apache.solr.core.SolrConfig;
 import org.apache.solr.util.plugin.AbstractPluginLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.NamedNodeMap;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 
@@ -54,8 +49,6 @@ public final class FieldTypePluginLoader
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final XPath xpath = XPathFactory.newInstance().newXPath();
-
   /**
    * @param schema The schema that will be used to initialize the FieldTypes
    * @param fieldTypes All FieldTypes that are instantiated by 
@@ -81,27 +74,25 @@ public final class FieldTypePluginLoader
   protected FieldType create( SolrClassLoader loader,
                               String name, 
                               String className,
-                              Node node ) throws Exception {
+                              ConfigNode node ) throws Exception {
 
     FieldType ft = loader.newInstance(className, FieldType.class);
     ft.setTypeName(name);
     
-    String expression = "./analyzer[@type='query']";
-    Node anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+    ConfigNode anode = node.child(it -> "query".equals(it.attributes().get("type")) , "analyzer");
     Analyzer queryAnalyzer = readAnalyzer(anode);
 
-    expression = "./analyzer[@type='multiterm']";
-    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+    anode = node.child(it -> "multiterm".equals(it.attributes().get("type") ), "analyzer");
     Analyzer multiAnalyzer = readAnalyzer(anode);
 
     // An analyzer without a type specified, or with type="index"
-    expression = "./analyzer[not(@type)] | ./analyzer[@type='index']";
-    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+    anode = node.child(it ->
+        (it.attributes().get("type") == null || "index".equals(it.attributes().get("type"))), "analyzer");
     Analyzer analyzer = readAnalyzer(anode);
 
     // a custom similarity[Factory]
-    expression = "./similarity";
-    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+    anode = node.child("similarity") ;
+
     SimilarityFactory simFactory = IndexSchema.readSimilarity(loader, anode);
     if (null != simFactory) {
       ft.setSimilarity(simFactory);
@@ -153,9 +144,9 @@ public final class FieldTypePluginLoader
   }
   
   @Override
-  protected void init(FieldType plugin, Node node) throws Exception {
+  protected void init(FieldType plugin, ConfigNode node) throws Exception {
 
-    Map<String, String> params = DOMUtil.toMapExcept(node.getAttributes(), NAME);
+    Map<String, String> params = DOMUtil.toMapExcept(node, NAME);
     plugin.setArgs(schema, params);
   }
 
@@ -187,35 +178,29 @@ public final class FieldTypePluginLoader
   // <analyzer><tokenizer class="...."/><tokenizer class="...." arg="....">
   //
   //
-  private Analyzer readAnalyzer(Node node) throws XPathExpressionException {
+  private Analyzer readAnalyzer(ConfigNode node)  {
                                 
     final SolrClassLoader loader = schema.getSolrClassLoader();
 
     // parent node used to be passed in as "fieldtype"
-    // if (!fieldtype.hasChildNodes()) return null;
-    // Node node = DOMUtil.getChild(fieldtype,"analyzer");
-    
+
     if (node == null) return null;
-    NamedNodeMap attrs = node.getAttributes();
-    String analyzerName = DOMUtil.getAttr(attrs,"class");
+    String analyzerName = DOMUtil.getAttr(node,"class", null);
 
     // check for all of these up front, so we can error if used in 
     // conjunction with an explicit analyzer class.
-    NodeList charFilterNodes = (NodeList)xpath.evaluate
-      ("./charFilter",  node, XPathConstants.NODESET);
-    NodeList tokenizerNodes = (NodeList)xpath.evaluate
-      ("./tokenizer", node, XPathConstants.NODESET);
-    NodeList tokenFilterNodes = (NodeList)xpath.evaluate
-      ("./filter", node, XPathConstants.NODESET);
-      
+    List<ConfigNode> charFilterNodes = node.children("charFilter");
+    List<ConfigNode> tokenizerNodes = node.children("tokenizer");
+    List<ConfigNode> tokenFilterNodes = node.children("filter");
+
     if (analyzerName != null) {
 
       // explicitly check for child analysis factories instead of
       // just any child nodes, because the user might have their
       // own custom nodes (ie: <description> or something like that)
-      if (0 != charFilterNodes.getLength() ||
-          0 != tokenizerNodes.getLength() ||
-          0 != tokenFilterNodes.getLength()) {
+      if (0 != charFilterNodes.size() ||
+          0 != tokenizerNodes.size() ||
+          0 != tokenFilterNodes.size()) {
         throw new SolrException
         ( SolrException.ErrorCode.SERVER_ERROR,
           "Configuration Error: Analyzer class='" + analyzerName +
@@ -227,7 +212,7 @@ public final class FieldTypePluginLoader
         final Class<? extends Analyzer> clazz = loader.findClass(analyzerName, Analyzer.class);
         Analyzer analyzer = clazz.getConstructor().newInstance();
 
-        final String matchVersionStr = DOMUtil.getAttr(attrs, LUCENE_MATCH_VERSION_PARAM);
+        final String matchVersionStr = DOMUtil.getAttr(node, LUCENE_MATCH_VERSION_PARAM,null);
         final Version luceneMatchVersion = (matchVersionStr == null) ?
           schema.getDefaultLuceneMatchVersion() :
           SolrConfig.parseLuceneVersionString(matchVersionStr);
@@ -254,9 +239,9 @@ public final class FieldTypePluginLoader
       ("[schema.xml] analyzer/charFilter", CharFilterFactory.class, false, false) {
 
       @Override
-      @SuppressWarnings({"rawtypes"})
-      protected CharFilterFactory create(SolrClassLoader loader, String name, String className, Node node) throws Exception {
-        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+      @SuppressWarnings("rawtypes")
+      protected CharFilterFactory create(SolrClassLoader loader, String name, String className, ConfigNode node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMapExcept(node);
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, CharFilterFactory.class.getSimpleName()).toString());
         CharFilterFactory factory;
@@ -279,7 +264,7 @@ public final class FieldTypePluginLoader
       }
 
       @Override
-      protected void init(CharFilterFactory plugin, Node node) throws Exception {
+      protected void init(CharFilterFactory plugin, ConfigNode node) throws Exception {
         if( plugin != null ) {
           charFilters.add( plugin );
         }
@@ -305,9 +290,9 @@ public final class FieldTypePluginLoader
       ("[schema.xml] analyzer/tokenizer", TokenizerFactory.class, false, false) {
       
       @Override
-      @SuppressWarnings({"rawtypes"})
-      protected TokenizerFactory create(SolrClassLoader loader, String name, String className, Node node) throws Exception {
-        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+      @SuppressWarnings("rawtypes")
+      protected TokenizerFactory create(SolrClassLoader loader, String name, String className, ConfigNode node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMap(node);
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, TokenizerFactory.class.getSimpleName()).toString());
         TokenizerFactory factory;
@@ -330,7 +315,7 @@ public final class FieldTypePluginLoader
       }
       
       @Override
-      protected void init(TokenizerFactory plugin, Node node) throws Exception {
+      protected void init(TokenizerFactory plugin, ConfigNode node) throws Exception {
         if( !tokenizers.isEmpty() ) {
           throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
               "The schema defines multiple tokenizers for: "+node );
@@ -360,9 +345,9 @@ public final class FieldTypePluginLoader
       new AbstractPluginLoader<TokenFilterFactory>("[schema.xml] analyzer/filter", TokenFilterFactory.class, false, false)
     {
       @Override
-      @SuppressWarnings({"rawtypes"})
-      protected TokenFilterFactory create(SolrClassLoader loader, String name, String className, Node node) throws Exception {
-        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+      @SuppressWarnings("rawtypes")
+      protected TokenFilterFactory create(SolrClassLoader loader, String name, String className, ConfigNode node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMap(node);
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, TokenFilterFactory.class.getSimpleName()).toString());
         TokenFilterFactory factory;
@@ -385,7 +370,7 @@ public final class FieldTypePluginLoader
       }
       
       @Override
-      protected void init(TokenFilterFactory plugin, Node node) throws Exception {
+      protected void init(TokenFilterFactory plugin, ConfigNode node) throws Exception {
         if( plugin != null ) {
           filters.add( plugin );
         }
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index 305cd50..c2f3579 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -16,9 +16,6 @@
  */
 package org.apache.solr.schema;
 
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
 import java.io.IOException;
 import java.io.Writer;
 import java.lang.invoke.MethodHandles;
@@ -44,12 +41,14 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.queries.payloads.PayloadDecoder;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Version;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.MapSerializable;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
@@ -60,29 +59,23 @@ import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Cache;
-import org.apache.solr.common.util.DOMUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.core.XmlConfigFile;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.SchemaXmlWriter;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.similarities.SchemaSimilarityFactory;
 import org.apache.solr.uninverting.UninvertingReader;
 import org.apache.solr.util.ConcurrentLRUCache;
+import org.apache.solr.common.util.DOMUtil;
 import org.apache.solr.util.PayloadUtils;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NamedNodeMap;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.InputSource;
 
 import static java.util.Arrays.asList;
 import static java.util.Collections.singletonList;
@@ -122,13 +115,9 @@ public class IndexSchema {
   public static final String UNIQUE_KEY = "uniqueKey";
   public static final String VERSION = "version";
 
-  private static final String AT = "@";
   private static final String DESTINATION_DYNAMIC_BASE = "destDynamicBase";
-  private static final String SOLR_CORE_NAME = "solr.core.name";
   private static final String SOURCE_DYNAMIC_BASE = "sourceDynamicBase";
   private static final String SOURCE_EXPLICIT_FIELDS = "sourceExplicitFields";
-  private static final String TEXT_FUNCTION = "text()";
-  private static final String XPATH_OR = " | ";
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected String resourceName;
@@ -148,6 +137,9 @@ public class IndexSchema {
 
   public DynamicField[] getDynamicFields() { return dynamicFields; }
 
+  private static final Set<String> FIELDTYPE_KEYS = ImmutableSet.of("fieldtype", "fieldType");
+  private static final Set<String> FIELD_KEYS = ImmutableSet.of("dynamicField", "field");
+
   @SuppressWarnings({"unchecked", "rawtypes"})
   protected Cache<String, SchemaField> dynamicFieldCache = new ConcurrentLRUCache(10000, 8000, 9000,100, false,false, null);
 
@@ -163,27 +155,31 @@ public class IndexSchema {
   public DynamicCopy[] getDynamicCopyFields() { return dynamicCopyFields; }
 
   private Map<FieldType, PayloadDecoder> decoders = new HashMap<>();  // cache to avoid scanning token filters repeatedly, unnecessarily
-
   /**
    * keys are all fields copied to, count is num of copyField
    * directives that target them.
    */
   protected Map<SchemaField, Integer> copyFieldTargetCounts = new HashMap<>();
+  private ConfigNode rootNode;
+
 
   /**
    * Constructs a schema using the specified resource name and stream.
    * By default, this follows the normal config path directory searching rules.
    * @see SolrResourceLoader#openResource
    */
-  public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResourceLoader resourceLoader, Properties substitutableProperties) {
+  public IndexSchema(String name, ConfigSetService.ConfigResource schemaResource, Version luceneVersion, SolrResourceLoader resourceLoader, Properties substitutableProperties) {
     this(luceneVersion, resourceLoader, substitutableProperties);
 
     this.resourceName = Objects.requireNonNull(name);
+    ConfigNode.SUBSTITUTES.set(substitutableProperties::getProperty);
     try {
-      readSchema(is);
+      readSchema(schemaResource);
       loader.inform(loader);
     } catch (IOException e) {
       throw new RuntimeException(e);
+    } finally {
+      ConfigNode.SUBSTITUTES.remove();
     }
   }
 
@@ -474,23 +470,17 @@ public class IndexSchema {
     }
   }
 
-  protected void readSchema(InputSource is) {
+  protected void readSchema(ConfigSetService.ConfigResource is) {
     assert null != is : "schema InputSource should never be null";
     try {
-      // pass the config resource loader to avoid building an empty one for no reason:
-      // in the current case though, the stream is valid so we wont load the resource by name
-      XmlConfigFile schemaConf = new XmlConfigFile(loader, SCHEMA, is, SLASH+SCHEMA+SLASH, substitutableProperties);
-      Document document = schemaConf.getDocument();
-      final XPath xpath = schemaConf.getXPath();
-      String expression = stepsToPath(SCHEMA, AT + NAME);
-      Node nd = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+      rootNode = is.get();
+      name = rootNode.attributes().get("name");
       StringBuilder sb = new StringBuilder();
       // Another case where the initialization from the test harness is different than the "real world"
-      if (nd==null) {
+      if (name==null) {
         sb.append("schema has no name!");
         log.warn("{}", sb);
       } else {
-        name = nd.getNodeValue();
         sb.append("Schema ");
         sb.append(NAME);
         sb.append("=");
@@ -498,22 +488,21 @@ public class IndexSchema {
         log.info("{}", sb);
       }
 
-      //                      /schema/@version
-      expression = stepsToPath(SCHEMA, AT + VERSION);
-      version = schemaConf.getFloat(expression, 1.0f);
+      version =  Float.parseFloat(rootNode.attributes().get("version","1.0f"));
 
       // load the Field Types
       final FieldTypePluginLoader typeLoader = new FieldTypePluginLoader(this, fieldTypes, schemaAware);
-      expression = getFieldTypeXPathExpressions();
-      NodeList nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
-      typeLoader.load(solrClassLoader, nodes);
+
+      List<ConfigNode> fTypes = rootNode.children(null, FIELDTYPE_KEYS);
+      ConfigNode types = rootNode.child(TYPES);
+      if(types != null) fTypes.addAll(types.children(null, FIELDTYPE_KEYS));
+      typeLoader.load(solrClassLoader, fTypes);
 
       // load the fields
-      Map<String,Boolean> explicitRequiredProp = loadFields(document, xpath);
+      Map<String,Boolean> explicitRequiredProp = loadFields(rootNode);
+
 
-      expression = stepsToPath(SCHEMA, SIMILARITY); //   /schema/similarity
-      Node node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
-      similarityFactory = readSimilarity(solrClassLoader, node);
+      similarityFactory = readSimilarity(solrClassLoader, rootNode.child(SIMILARITY));
       if (similarityFactory == null) {
         final Class<?> simClass = SchemaSimilarityFactory.class;
         // use the loader to ensure proper SolrCoreAware handling
@@ -536,27 +525,22 @@ public class IndexSchema {
         }
       }
 
-      //                      /schema/defaultSearchField/text()
-      expression = stepsToPath(SCHEMA, "defaultSearchField", TEXT_FUNCTION);
-      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+      ConfigNode node = rootNode.child("defaultSearchField");
       if (node != null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Setting defaultSearchField in schema not supported since Solr 7");
       }
 
-      //                      /schema/solrQueryParser/@defaultOperator
-      expression = stepsToPath(SCHEMA, "solrQueryParser", AT + "defaultOperator");
-      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+      node = rootNode.child(it -> it.attributes().get("defaultOperator") != null, "solrQueryParser");
       if (node != null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Setting default operator in schema (solrQueryParser/@defaultOperator) not supported");
       }
 
-      //                      /schema/uniqueKey/text()
-      expression = stepsToPath(SCHEMA, UNIQUE_KEY, TEXT_FUNCTION);
-      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+      node = rootNode.child(UNIQUE_KEY);
+
       if (node==null) {
         log.warn("no {} specified in schema.", UNIQUE_KEY);
       } else {
-        uniqueKeyField=getIndexedField(node.getNodeValue().trim());
+        uniqueKeyField=getIndexedField(node.textValue().trim());
         uniqueKeyFieldName=uniqueKeyField.getName();
         uniqueKeyFieldType=uniqueKeyField.getType();
         
@@ -604,11 +588,9 @@ public class IndexSchema {
       }
 
       /////////////// parse out copyField commands ///////////////
-      // Map<String,ArrayList<SchemaField>> cfields = new HashMap<String,ArrayList<SchemaField>>();
-      // expression = "/schema/copyField";
 
       dynamicCopyFields = new DynamicCopy[] {};
-      loadCopyFields(document, xpath);
+      loadCopyFields(rootNode);
 
       postReadInform();
 
@@ -639,61 +621,55 @@ public class IndexSchema {
    * 
    * @return a map from field name to explicit required value  
    */ 
-  protected synchronized Map<String,Boolean> loadFields(Document document, XPath xpath) throws XPathExpressionException {
+  protected synchronized Map<String,Boolean> loadFields(ConfigNode n) {
     // Hang on to the fields that say if they are required -- this lets us set a reasonable default for the unique key
     Map<String,Boolean> explicitRequiredProp = new HashMap<>();
     
     ArrayList<DynamicField> dFields = new ArrayList<>();
 
-    //                  /schema/field | /schema/dynamicField | /schema/fields/field | /schema/fields/dynamicField
-    String expression = stepsToPath(SCHEMA, FIELD)
-        + XPATH_OR + stepsToPath(SCHEMA, DYNAMIC_FIELD)
-        + XPATH_OR + stepsToPath(SCHEMA, FIELDS, FIELD)
-        + XPATH_OR + stepsToPath(SCHEMA, FIELDS, DYNAMIC_FIELD);
-
-    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
-
-    for (int i=0; i<nodes.getLength(); i++) {
-      Node node = nodes.item(i);
-
-      NamedNodeMap attrs = node.getAttributes();
+    List<ConfigNode> nodes = n.children(null,  FIELD_KEYS);
+    ConfigNode child = n.child(FIELDS);
+    if(child != null) {
+      nodes.addAll(child.children(null, FIELD_KEYS));
+    }
 
-      String name = DOMUtil.getAttr(attrs, NAME, "field definition");
+    for (ConfigNode node : nodes) {
+      String name = DOMUtil.getAttr(node, NAME, "field definition");
       log.trace("reading field def {}", name);
-      String type = DOMUtil.getAttr(attrs, TYPE, "field " + name);
+      String type = DOMUtil.getAttr(node, TYPE, "field " + name);
 
       FieldType ft = fieldTypes.get(type);
-      if (ft==null) {
+      if (ft == null) {
         throw new SolrException
             (ErrorCode.BAD_REQUEST, "Unknown " + FIELD_TYPE + " '" + type + "' specified on field " + name);
       }
 
-      Map<String,String> args = DOMUtil.toMapExcept(attrs, NAME, TYPE);
+      Map<String, String> args = DOMUtil.toMapExcept(node, NAME, TYPE);
       if (null != args.get(REQUIRED)) {
         explicitRequiredProp.put(name, Boolean.valueOf(args.get(REQUIRED)));
       }
 
-      SchemaField f = SchemaField.create(name,ft,args);
+      SchemaField f = SchemaField.create(name, ft, args);
 
-      if (node.getNodeName().equals(FIELD)) {
-        SchemaField old = fields.put(f.getName(),f);
-        if( old != null ) {
+      if (node.name().equals(FIELD)) {
+        SchemaField old = fields.put(f.getName(), f);
+        if (old != null) {
           String msg = "[schema.xml] Duplicate field definition for '"
-            + f.getName() + "' [[["+old.toString()+"]]] and [[["+f.toString()+"]]]";
-          throw new SolrException(ErrorCode.SERVER_ERROR, msg );
+              + f.getName() + "' [[[" + old.toString() + "]]] and [[[" + f.toString() + "]]]";
+          throw new SolrException(ErrorCode.SERVER_ERROR, msg);
         }
         log.debug("field defined: {}", f);
-        if( f.getDefaultValue() != null ) {
+        if (f.getDefaultValue() != null) {
           if (log.isDebugEnabled()) {
             log.debug("{} contains default value {}", name, f.getDefaultValue());
           }
-          fieldsWithDefaultValue.add( f );
+          fieldsWithDefaultValue.add(f);
         }
         if (f.isRequired()) {
           log.debug("{} is required in this schema", name);
           requiredFields.add(f);
         }
-      } else if (node.getNodeName().equals(DYNAMIC_FIELD)) {
+      } else if (node.name().equals(DYNAMIC_FIELD)) {
         if (isValidDynamicField(dFields, f)) {
           addDynamicFieldNoDupCheck(dFields, f);
         }
@@ -733,17 +709,19 @@ public class IndexSchema {
   /**
    * Loads the copy fields
    */
-  protected synchronized void loadCopyFields(Document document, XPath xpath) throws XPathExpressionException {
-    String expression = "//" + COPY_FIELD;
-    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
-
-    for (int i=0; i<nodes.getLength(); i++) {
-      Node node = nodes.item(i);
-      NamedNodeMap attrs = node.getAttributes();
+  protected synchronized void loadCopyFields(ConfigNode n) {
+    List<ConfigNode> nodes = n.children(COPY_FIELD);
+    ConfigNode f = n.child(FIELDS);
+    if (f != null) {
+      List<ConfigNode> c = f.children(COPY_FIELD);
+      if (nodes.isEmpty()) nodes = c;
+      else nodes.addAll(c);
+    }
+    for (ConfigNode node : nodes) {
 
-      String source = DOMUtil.getAttr(attrs, SOURCE, COPY_FIELD + " definition");
-      String dest   = DOMUtil.getAttr(attrs, DESTINATION,  COPY_FIELD + " definition");
-      String maxChars = DOMUtil.getAttr(attrs, MAX_CHARS);
+      String source = DOMUtil.getAttr(node, SOURCE, COPY_FIELD + " definition");
+      String dest   = DOMUtil.getAttr(node, DESTINATION,  COPY_FIELD + " definition");
+      String maxChars = DOMUtil.getAttr(node, MAX_CHARS, null);
 
       int maxCharsInt = CopyField.UNLIMITED;
       if (maxChars != null) {
@@ -773,17 +751,6 @@ public class IndexSchema {
     }
   }
 
-  /**
-   * Converts a sequence of path steps into a rooted path, by inserting slashes in front of each step.
-   * @param steps The steps to join with slashes to form a path
-   * @return a rooted path: a leading slash followed by the given steps joined with slashes
-   */
-  private String stepsToPath(String... steps) {
-    StringBuilder builder = new StringBuilder();
-    for (String step : steps) { builder.append(SLASH).append(step); }
-    return builder.toString();
-  }
-
   /** Returns true if the given name has exactly one asterisk either at the start or end of the name */
   protected static boolean isValidFieldGlob(String name) {
     if (name.startsWith("*") || name.endsWith("*")) {
@@ -990,12 +957,12 @@ public class IndexSchema {
     dynamicCopyFields = temp;
   }
 
-  static SimilarityFactory readSimilarity(SolrClassLoader loader, Node node) {
+  static SimilarityFactory readSimilarity(SolrClassLoader loader, ConfigNode node) {
     if (node==null) {
       return null;
     } else {
       SimilarityFactory similarityFactory;
-      final String classArg = ((Element) node).getAttribute(SimilarityFactory.CLASS_NAME);
+      final String classArg = node.attributes().get(SimilarityFactory.CLASS_NAME);
       final Object obj = loader.newInstance(classArg, Object.class, "search.similarities.");
       if (obj instanceof SimilarityFactory) {
         // configure a factory, get a similarity back
@@ -1948,14 +1915,6 @@ public class IndexSchema {
     throw new SolrException(ErrorCode.SERVER_ERROR, msg);
   }
 
-  protected String getFieldTypeXPathExpressions() {
-    //               /schema/fieldtype | /schema/fieldType | /schema/types/fieldtype | /schema/types/fieldType
-    String expression = stepsToPath(SCHEMA, FIELD_TYPE.toLowerCase(Locale.ROOT)) // backcompat(?)
-            + XPATH_OR + stepsToPath(SCHEMA, FIELD_TYPE)
-            + XPATH_OR + stepsToPath(SCHEMA, TYPES, FIELD_TYPE.toLowerCase(Locale.ROOT))
-            + XPATH_OR + stepsToPath(SCHEMA, TYPES, FIELD_TYPE);
-    return expression;
-  }
 
   /**
    * Helper method that returns <code>true</code> if the {@link #ROOT_FIELD_NAME} uses the exact
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java b/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
index 095efd4..c49cb12 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
@@ -16,27 +16,35 @@
  */
 package org.apache.solr.schema;
 
-import java.io.InputStream;
-import java.lang.invoke.MethodHandles;
 
+import org.apache.solr.cloud.CloudConfigSetService;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.util.SystemIdResolver;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.xml.sax.InputSource;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 /** Base class for factories for IndexSchema implementations */
 public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  /** Instantiates the configured schema factory, then calls create on it. */
   public static IndexSchema buildIndexSchema(String resourceName, SolrConfig config) {
-    return newIndexSchemaFactory(config).create(resourceName, config);
+    return buildIndexSchema(resourceName, config, null);
+  }
+  /** Instantiates the configured schema factory, then calls create on it. */
+  public static IndexSchema buildIndexSchema(String resourceName, SolrConfig config, ConfigSetService configSetService) {
+    return newIndexSchemaFactory(config).create(resourceName, config, configSetService);
   }
 
   /** Instantiates us from {@link SolrConfig}. */
@@ -54,7 +62,7 @@ public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
 
   /**
    * Returns the resource (file) name that will be used for the schema itself.  The answer may be a guess.
-   * Do not pass the result of this to {@link #create(String, SolrConfig)}.
+   * Do not pass the result of this to {@link #create(String, SolrConfig, ConfigSetService)}.
    * The input is the name coming from the {@link org.apache.solr.core.CoreDescriptor}
    * which acts as a default or asked-for name.
    */
@@ -65,25 +73,57 @@ public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
   /**
    * Returns an index schema created from a local resource.  The input is usually from the core descriptor.
    */
-  public IndexSchema create(String resourceName, SolrConfig config) {
+  public IndexSchema create(String resourceName, SolrConfig config, ConfigSetService configSetService) {
     SolrResourceLoader loader = config.getResourceLoader();
     InputStream schemaInputStream = null;
 
     if (null == resourceName) {
       resourceName = IndexSchema.DEFAULT_SCHEMA_FILE;
     }
-
     try {
       schemaInputStream = loader.openResource(resourceName);
+      return new IndexSchema(resourceName, getConfigResource(configSetService, schemaInputStream, loader, resourceName), config.luceneMatchVersion, loader, config.getSubstituteProperties());
+    } catch (RuntimeException rte) {
+      throw rte;
     } catch (Exception e) {
       final String msg = "Error loading schema resource " + resourceName;
       log.error(msg, e);
       throw new SolrException(ErrorCode.SERVER_ERROR, msg, e);
     }
-    InputSource inputSource = new InputSource(schemaInputStream);
-    inputSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(resourceName));
-    IndexSchema schema = new IndexSchema(resourceName, inputSource, config.luceneMatchVersion, loader, config.getSubstituteProperties());
-    return schema;
+  }
+
+  @SuppressWarnings("unchecked")
+  public static ConfigSetService.ConfigResource getConfigResource(ConfigSetService configSetService, InputStream schemaInputStream, SolrResourceLoader loader, String name) throws IOException {
+    if (configSetService instanceof CloudConfigSetService && schemaInputStream instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
+      ZkSolrResourceLoader.ZkByteArrayInputStream is = (ZkSolrResourceLoader.ZkByteArrayInputStream) schemaInputStream;
+      Map<String, VersionedConfig> configCache = (Map<String, VersionedConfig>) ((CloudConfigSetService) configSetService).getSolrCloudManager().getObjectCache()
+              .computeIfAbsent(ConfigSetService.ConfigResource.class.getName(), s -> new ConcurrentHashMap<>());
+      VersionedConfig cached = configCache.get(is.fileName);
+      if (cached != null) {
+        if (cached.version != is.getStat().getVersion()) {
+          configCache.remove(is.fileName);// this is stale. remove from cache
+        } else {
+          return () -> cached.data;
+        }
+      }
+      return () -> {
+        ConfigNode data = ConfigSetService.getParsedSchema(schemaInputStream, loader, name);// either missing or stale. create a new one
+        configCache.put(is.fileName, new VersionedConfig(is.getStat().getVersion(), data));
+        return data;
+      };
+    }
+    //this is not cacheable as it does not come from ZK
+    return () -> ConfigSetService.getParsedSchema(schemaInputStream,loader, name);
+  }
+
+  public static class VersionedConfig {
+    final int version;
+    final ConfigNode data;
+
+    public VersionedConfig(int version, ConfigNode data) {
+      this.version = version;
+      this.data = data;
+    }
   }
 
 }
diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
index e7a748c..dd17c5b 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
@@ -67,9 +67,11 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.rest.schema.FieldTypeXmlAdapter;
+import org.apache.solr.util.DOMConfigNode;
 import org.apache.solr.util.FileUtils;
 import org.apache.solr.util.RTimer;
 import org.apache.zookeeper.CreateMode;
@@ -77,7 +79,6 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.xml.sax.InputSource;
 
 import static org.apache.solr.core.SolrResourceLoader.informAware;
 
@@ -102,7 +103,7 @@ public final class ManagedIndexSchema extends IndexSchema {
    * By default, this follows the normal config path directory searching rules.
    * @see org.apache.solr.core.SolrResourceLoader#openResource
    */
-  ManagedIndexSchema(SolrConfig solrConfig, String name, InputSource is, boolean isMutable,
+  ManagedIndexSchema(SolrConfig solrConfig, String name, ConfigSetService.ConfigResource is, boolean isMutable,
                      String managedSchemaResourceName, int schemaZkVersion, Object schemaUpdateLock) {
     super(name, is, solrConfig.luceneMatchVersion, solrConfig.getResourceLoader(), solrConfig.getSubstituteProperties());
     this.isMutable = isMutable;
@@ -1308,7 +1309,7 @@ public final class ManagedIndexSchema extends IndexSchema {
     Map<String,FieldType> newFieldTypes = new HashMap<>();
     List<SchemaAware> schemaAwareList = new ArrayList<>();
     FieldTypePluginLoader typeLoader = new FieldTypePluginLoader(this, newFieldTypes, schemaAwareList);
-    typeLoader.loadSingle(solrClassLoader, FieldTypeXmlAdapter.toNode(options));
+    typeLoader.loadSingle(solrClassLoader, new DOMConfigNode(FieldTypeXmlAdapter.toNode(options)));
     FieldType ft = newFieldTypes.get(typeName);
     if (!schemaAwareList.isEmpty())
       schemaAware.addAll(schemaAwareList);
diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
index ef5df42..884751b 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
@@ -30,6 +30,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCmdExecutor;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -109,7 +110,7 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
    * renamed by appending the extension named in {@link #UPGRADED_SCHEMA_EXTENSION}.
    */
   @Override
-  public ManagedIndexSchema create(String resourceName, SolrConfig config) {
+  public ManagedIndexSchema create(String resourceName, SolrConfig config, ConfigSetService configSetService) {
     this.resourceName = resourceName;
     this.config = config;
     this.loader = config.getResourceLoader();
@@ -131,7 +132,7 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
         // Attempt to load the managed schema
         byte[] data = zkClient.getData(managedSchemaPath, null, stat, true);
         schemaZkVersion = stat.getVersion();
-        schemaInputStream = new ByteArrayInputStream(data);
+        schemaInputStream = new ZkSolrResourceLoader.ZkByteArrayInputStream(data, managedSchemaPath, stat);
         loadedResource = managedSchemaResourceName;
         warnIfNonManagedSchemaExists();
       } catch (InterruptedException e) {
@@ -174,8 +175,12 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
     }
     InputSource inputSource = new InputSource(schemaInputStream);
     inputSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(loadedResource));
-    schema = new ManagedIndexSchema(config, loadedResource, inputSource, isMutable,
-                                    managedSchemaResourceName, schemaZkVersion, getSchemaUpdateLock());
+    try {
+      schema = new ManagedIndexSchema(config, loadedResource,IndexSchemaFactory.getConfigResource(configSetService, schemaInputStream, loader, managedSchemaResourceName) , isMutable,
+              managedSchemaResourceName, schemaZkVersion, getSchemaUpdateLock());
+    } catch (IOException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error loading parsing schema", e);
+    }
     if (shouldUpgrade) {
       // Persist the managed schema if it doesn't already exist
       synchronized (schema.getSchemaUpdateLock()) {
@@ -210,7 +215,7 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
       }
     }
-    return schemaInputStream; 
+    return schemaInputStream;
   }
 
   /**
@@ -251,7 +256,7 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
       }
     }
   }
-  
+
   /**
    * Persist the managed schema and rename the non-managed schema 
    * by appending {@link #UPGRADED_SCHEMA_EXTENSION}.
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
index f539969..fe2db9a 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -42,7 +43,6 @@ import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.xml.sax.InputSource;
 
 import static java.util.Collections.singleton;
 import static java.util.Collections.singletonList;
@@ -454,8 +454,8 @@ public class SchemaManager {
       if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
         int version = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
         log.info("managed schema loaded . version : {} ", version);
-        return new ManagedIndexSchema(core.getSolrConfig(), name, new InputSource(in), true, name, version,
-            core.getLatestSchema().getSchemaUpdateLock());
+        return new ManagedIndexSchema(core.getSolrConfig(), name, () -> ConfigSetService.getParsedSchema(in, zkLoader,  core.getLatestSchema().getResourceName()), true, name, version,
+                core.getLatestSchema().getSchemaUpdateLock());
       } else {
         return (ManagedIndexSchema) core.getLatestSchema();
       }
diff --git a/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java b/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
index 3b867ce..bc69630 100644
--- a/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
+++ b/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
@@ -25,6 +25,7 @@ import org.apache.solr.common.cloud.OnReconnect;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZooKeeperException;
 import org.apache.solr.core.CloseHook;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.zookeeper.KeeperException;
@@ -33,7 +34,6 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.xml.sax.InputSource;
 
 /** Keeps a ManagedIndexSchema up-to-date when changes are made to the serialized managed schema in ZooKeeper */
 public class ZkIndexSchemaReader implements OnReconnect {
@@ -43,10 +43,11 @@ public class ZkIndexSchemaReader implements OnReconnect {
   private String managedSchemaPath;
   private final String uniqueCoreId; // used in equals impl to uniquely identify the core that we're dependent on
   private SchemaWatcher schemaWatcher;
+  private ZkSolrResourceLoader zkLoader;
 
   public ZkIndexSchemaReader(ManagedIndexSchemaFactory managedIndexSchemaFactory, SolrCore solrCore) {
     this.managedIndexSchemaFactory = managedIndexSchemaFactory;
-    ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)managedIndexSchemaFactory.getResourceLoader();
+    zkLoader = (ZkSolrResourceLoader)managedIndexSchemaFactory.getResourceLoader();
     this.zkClient = zkLoader.getZkController().getZkClient();
     this.managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedIndexSchemaFactory.getManagedSchemaResourceName();
     this.uniqueCoreId = solrCore.getName()+":"+solrCore.getStartNanoTime();
@@ -78,13 +79,13 @@ public class ZkIndexSchemaReader implements OnReconnect {
     zkLoader.getZkController().addOnReconnectListener(this);
   }
 
-  public Object getSchemaUpdateLock() { 
-    return managedIndexSchemaFactory.getSchemaUpdateLock(); 
+  public Object getSchemaUpdateLock() {
+    return managedIndexSchemaFactory.getSchemaUpdateLock();
   }
 
   /**
    * Creates a schema watcher and returns it for controlling purposes.
-   * 
+   *
    * @return the registered {@linkplain SchemaWatcher}.
    */
   public SchemaWatcher createSchemaWatcher() {
@@ -102,10 +103,10 @@ public class ZkIndexSchemaReader implements OnReconnect {
       Thread.currentThread().interrupt();
       log.warn("", e);
     }
-    
+
     return watcher;
   }
-  
+
   /**
    * Watches for schema changes and triggers updates in the {@linkplain ZkIndexSchemaReader}.
    */
@@ -171,11 +172,11 @@ public class ZkIndexSchemaReader implements OnReconnect {
             log.info("Retrieved schema version {} from Zookeeper", stat.getVersion());
           }
           long start = System.nanoTime();
-          InputSource inputSource = new InputSource(new ByteArrayInputStream(data));
           String resourceName = managedIndexSchemaFactory.getManagedSchemaResourceName();
           ManagedIndexSchema newSchema = new ManagedIndexSchema
-              (managedIndexSchemaFactory.getConfig(), resourceName, inputSource, managedIndexSchemaFactory.isMutable(), 
-                  resourceName, stat.getVersion(), oldSchema.getSchemaUpdateLock());
+                  (managedIndexSchemaFactory.getConfig(), resourceName,
+                          () -> ConfigSetService.getParsedSchema(new ByteArrayInputStream(data),zkLoader , resourceName), managedIndexSchemaFactory.isMutable(),
+                          resourceName, stat.getVersion(), oldSchema.getSchemaUpdateLock());
           managedIndexSchemaFactory.setSchema(newSchema);
           long stop = System.nanoTime();
           log.info("Finished refreshing schema in {} ms", TimeUnit.MILLISECONDS.convert(stop - start, TimeUnit.NANOSECONDS));
diff --git a/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java b/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java
new file mode 100644
index 0000000..6afc5a5
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java
@@ -0,0 +1,89 @@
+/*
+ * 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.solr.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+import org.apache.solr.cluster.api.SimpleMap;
+import org.apache.solr.common.ConfigNode;
+import org.apache.solr.common.util.DOMUtil;
+import org.apache.solr.common.util.WrappedSimpleMap;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+/**
+ * Read using DOM
+ */
+public class DOMConfigNode implements ConfigNode {
+
+  private final Node node;
+  SimpleMap<String> attrs;
+
+
+  @Override
+  public String name() {
+    return node.getNodeName();
+  }
+
+  @Override
+  public String textValue() {
+    return DOMUtil.getText(node);
+  }
+
+  public DOMConfigNode(Node node) {
+    this.node = node;
+  }
+
+  @Override
+  public SimpleMap<String> attributes() {
+    if (attrs != null) return attrs;
+    return attrs = new WrappedSimpleMap<>(DOMUtil.toMap(node.getAttributes()));
+  }
+
+  @Override
+  public ConfigNode child(String name) {
+    Node n  =  DOMUtil.getChild(node, name);
+    return n == null? null: new DOMConfigNode(n);
+  }
+
+  @Override
+  public List<ConfigNode> children(String name) {
+    List<ConfigNode> result = new ArrayList<>();
+    forEachChild(it -> {
+      if (name.equals(it.name())) {
+        result.add(it);
+      }
+      return Boolean.TRUE;
+    });
+    return result;
+  }
+
+  @Override
+  public void forEachChild(Function<ConfigNode, Boolean> fun) {
+    NodeList nlst = node.getChildNodes();
+    for (int i = 0; i < nlst.getLength(); i++) {
+      Node item = nlst.item(i);
+      if(item.getNodeType() != Node.ELEMENT_NODE) continue;
+      Boolean toContinue = fun.apply(new DOMConfigNode(item));
+      if (Boolean.FALSE == toContinue) break;
+    }
+  }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/util/DataConfigNode.java b/solr/core/src/java/org/apache/solr/util/DataConfigNode.java
new file mode 100644
index 0000000..38940ee
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/DataConfigNode.java
@@ -0,0 +1,132 @@
+/*
+ * 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.solr.util;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import org.apache.solr.cluster.api.SimpleMap;
+import org.apache.solr.common.ConfigNode;
+import org.apache.solr.common.util.PropertiesUtil;
+
+/**
+ * ConfigNode impl that copies and maintains data internally from DOM
+ */
+public class DataConfigNode implements ConfigNode {
+  final String name;
+  final SimpleMap<String> attributes;
+  private final Map<String, List<ConfigNode>> kids = new HashMap<>();
+  private final String textData;
+
+  public DataConfigNode(ConfigNode root) {
+    name = root.name();
+    attributes = wrap(root.attributes());
+    textData = root.textValue();
+    root.forEachChild(it -> {
+      List<ConfigNode> nodes = kids.computeIfAbsent(it.name(),
+          k -> new ArrayList<>());
+
+     nodes.add(new DataConfigNode(it));
+      return Boolean.TRUE;
+    });
+
+  }
+
+  public String subtituteVal(String s) {
+    Function<String, String> props = SUBSTITUTES.get();
+    if (props == null) return s;
+    return PropertiesUtil.substitute(s, props);
+  }
+
+  private SimpleMap<String> wrap(SimpleMap<String> delegate) {
+    return new SimpleMap<>() {
+          @Override
+          public String get(String key) {
+            return subtituteVal(delegate.get(key));
+          }
+
+          @Override
+          public void forEachEntry(BiConsumer<String, ? super String> fun) {
+            delegate.forEachEntry((k, v) -> fun.accept(k, subtituteVal(v)));
+          }
+
+          @Override
+          public int size() {
+            return delegate.size();
+          }
+        };
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  public String textValue() {
+    return  subtituteVal(textData);
+  }
+
+  @Override
+  public SimpleMap<String> attributes() {
+    return attributes;
+  }
+
+  @Override
+  public ConfigNode child(String name) {
+    List<ConfigNode> val = kids.get(name);
+    return val == null || val.isEmpty() ? null : val.get(0);
+  }
+
+  @Override
+  public List<ConfigNode> children(String name) {
+    return kids.getOrDefault(name, Collections.emptyList());
+  }
+
+  @Override
+  public List<ConfigNode> children(Predicate<ConfigNode> test, Set<String> matchNames) {
+    List<ConfigNode> result = new ArrayList<>();
+    for (String s : matchNames) {
+      List<ConfigNode> vals = kids.get(s);
+      if (vals != null) {
+        vals.forEach(it -> {
+          if (test == null || test.test(it)) {
+            result.add(it);
+          }
+        });
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public void forEachChild(Function<ConfigNode, Boolean> fun) {
+    kids.forEach((s, configNodes) -> {
+      if (configNodes != null) {
+        configNodes.forEach(fun::apply);
+      }
+    });
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java b/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
index d042531..1077f14 100644
--- a/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
@@ -21,14 +21,13 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrClassLoader;
 import org.apache.solr.common.util.DOMUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 
@@ -86,7 +85,7 @@ public abstract class AbstractPluginLoader<T>
    * @param node - the XML node defining this plugin
    */
   @SuppressWarnings("unchecked")
-  protected T create(SolrClassLoader loader, String name, String className, Node node ) throws Exception
+  protected T create(SolrClassLoader loader, String name, String className, ConfigNode node ) throws Exception
   {
     return loader.newInstance(className, pluginClassType, getDefaultPackages());
   }
@@ -103,7 +102,7 @@ public abstract class AbstractPluginLoader<T>
    * @param plugin - the plugin to initialize
    * @param node - the XML node defining this plugin
    */
-  abstract protected void init( T plugin, Node node ) throws Exception;
+  abstract protected void init( T plugin, ConfigNode node ) throws Exception;
 
   /**
    * Initializes and registers each plugin in the list.
@@ -135,15 +134,13 @@ public abstract class AbstractPluginLoader<T>
    * If a default element is defined, it will be returned from this function.
    * 
    */
-  public T load(SolrClassLoader loader, NodeList nodes )
+  public T load(SolrClassLoader loader, List<ConfigNode> nodes )
   {
     List<PluginInitInfo> info = new ArrayList<>();
     T defaultPlugin = null;
     
     if (nodes !=null ) {
-      for (int i=0; i<nodes.getLength(); i++) {
-        Node node = nodes.item(i);
-  
+      for (ConfigNode node : nodes) {
         String name = null;
         try {
           name = DOMUtil.getAttr(node, NAME, requireName ? type : null);
@@ -225,7 +222,7 @@ public abstract class AbstractPluginLoader<T>
    * The created class for the plugin will be returned from this function.
    * 
    */
-  public T loadSingle(SolrClassLoader loader, Node node) {
+  public T loadSingle(SolrClassLoader loader, ConfigNode node) {
     List<PluginInitInfo> info = new ArrayList<>();
     T plugin = null;
 
@@ -277,9 +274,9 @@ public abstract class AbstractPluginLoader<T>
    */
   private class PluginInitInfo {
     final T plugin;
-    final Node node;
+    final ConfigNode node;
 
-    PluginInitInfo(T plugin, Node node) {
+    PluginInitInfo(T plugin, ConfigNode node) {
       this.plugin = plugin;
       this.node = node;
     }
diff --git a/solr/core/src/java/org/apache/solr/util/plugin/MapPluginLoader.java b/solr/core/src/java/org/apache/solr/util/plugin/MapPluginLoader.java
index 9cf68f6..46bd79f 100644
--- a/solr/core/src/java/org/apache/solr/util/plugin/MapPluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/util/plugin/MapPluginLoader.java
@@ -19,7 +19,7 @@ package org.apache.solr.util.plugin;
 import java.util.Map;
 
 import org.apache.solr.common.util.DOMUtil;
-import org.w3c.dom.Node;
+import org.apache.solr.common.ConfigNode;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 
@@ -38,8 +38,8 @@ public class MapPluginLoader<T extends MapInitializedPlugin> extends AbstractPlu
   }
 
   @Override
-  protected void init(T plugin, Node node) throws Exception {
-    Map<String, String> params = DOMUtil.toMapExcept(node.getAttributes(), NAME, "class");
+  protected void init(T plugin, ConfigNode node) throws Exception {
+    Map<String, String> params = DOMUtil.toMapExcept(node, NAME, "class");
     plugin.init( params );
   }
 
diff --git a/solr/core/src/java/org/apache/solr/util/plugin/NamedListPluginLoader.java b/solr/core/src/java/org/apache/solr/util/plugin/NamedListPluginLoader.java
index ae202db..5736abf 100644
--- a/solr/core/src/java/org/apache/solr/util/plugin/NamedListPluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/util/plugin/NamedListPluginLoader.java
@@ -19,7 +19,7 @@ package org.apache.solr.util.plugin;
 import java.util.Map;
 
 import org.apache.solr.common.util.DOMUtil;
-import org.w3c.dom.Node;
+import org.apache.solr.common.ConfigNode;
 
 /**
  *
@@ -35,7 +35,7 @@ public class NamedListPluginLoader<T extends NamedListInitializedPlugin> extends
   }
 
   @Override
-  protected void init(T plugin,Node node) throws Exception {
+  protected void init(T plugin, ConfigNode node) throws Exception {
     plugin.init( DOMUtil.childNodesToNamedList(node) );
   }
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-schema-daterangefield-instance-options.xml b/solr/core/src/test-files/solr/collection1/conf/bad-schema-daterangefield-instance-options.xml
index 4ee6e39..3b040c2 100644
--- a/solr/core/src/test-files/solr/collection1/conf/bad-schema-daterangefield-instance-options.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-schema-daterangefield-instance-options.xml
@@ -30,6 +30,5 @@
 
   <dynamicField name="*_s" type="string" multiValued="false"/>
   <dynamicField name="*_l" type="plong" multiValued="false"/>
-  <dynamicField name="*_c" type="currency" indexed="true" stored="true"/>
 
 </schema>
diff --git a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
index c611359..e2d8737 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
@@ -17,9 +17,6 @@
 
 package org.apache.solr.schema;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
@@ -50,6 +47,9 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 public class TestManagedSchemaThreadSafety extends SolrTestCaseJ4 {
 
   private static final class SuspendingZkClient extends SolrZkClient {
@@ -180,7 +180,7 @@ public class TestManagedSchemaThreadSafety extends SolrTestCaseJ4 {
 
         ManagedIndexSchemaFactory factory = new ManagedIndexSchemaFactory();
         factory.init(new NamedList());
-        factory.create("schema.xml", solrConfig);
+        factory.create("schema.xml", solrConfig, null);
       }
       catch (Exception e) {
         throw new RuntimeException(e);
diff --git a/solr/solrj/src/java/org/apache/solr/cluster/api/SimpleMap.java b/solr/solrj/src/java/org/apache/solr/cluster/api/SimpleMap.java
index ca747b9..81da171 100644
--- a/solr/solrj/src/java/org/apache/solr/cluster/api/SimpleMap.java
+++ b/solr/solrj/src/java/org/apache/solr/cluster/api/SimpleMap.java
@@ -35,6 +35,11 @@ public interface SimpleMap<T> extends MapWriter {
   /**get a value by key. If not present , null is returned */
   T get(String key);
 
+  default T get(String key, T def) {
+    T val = get(key);
+    return val == null ? def : val;
+  }
+
   /**Navigate through all keys and values */
   void forEachEntry(BiConsumer<String, ? super T> fun);
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java b/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java
new file mode 100644
index 0000000..1a67b52
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/ConfigNode.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.solr.common;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import org.apache.solr.cluster.api.SimpleMap;
+
+/**
+ * A generic interface that represents a config file, mostly XML
+ */
+public interface ConfigNode {
+  ThreadLocal<Function<String,String>> SUBSTITUTES = new ThreadLocal<>();
+
+  /**
+   * Name of the tag
+   */
+  String name();
+
+  /**
+   * Text value of the node
+   */
+  String textValue();
+
+  /**
+   * Attributes
+   */
+  SimpleMap<String> attributes();
+
+  /**
+   * Child by name
+   */
+  default ConfigNode child(String name) {
+    return child(null, name);
+  }
+
+  /**Iterate through child nodes with the name and return the first child that matches
+   */
+  default ConfigNode child(Predicate<ConfigNode> test, String name) {
+    ConfigNode[] result = new ConfigNode[1];
+    forEachChild(it -> {
+      if (name!=null && !name.equals(it.name())) return Boolean.TRUE;
+      if (test == null || test.test(it)) {
+        result[0] = it;
+        return Boolean.FALSE;
+      }
+      return Boolean.TRUE;
+    });
+    return result[0];
+  }
+
+  /**Iterate through child nodes with the names and return all the matching children
+   * @param nodeNames names of tags to be returned
+   * @param  test check for the nodes to be returned
+   */
+  default List<ConfigNode> children(Predicate<ConfigNode> test, String... nodeNames) {
+    return children(test, nodeNames == null ? Collections.emptySet() : Set.of(nodeNames));
+  }
+
+  /**Iterate through child nodes with the names and return all the matching children
+   * @param matchNames names of tags to be returned
+   * @param  test check for the nodes to be returned
+   */
+  default List<ConfigNode> children(Predicate<ConfigNode> test, Set<String> matchNames) {
+    List<ConfigNode> result = new ArrayList<>();
+    forEachChild(it -> {
+      if (matchNames != null && !matchNames.isEmpty() && !matchNames.contains(it.name())) return Boolean.TRUE;
+      if (test == null || test.test(it)) result.add(it);
+      return Boolean.TRUE;
+    });
+    return result;
+  }
+
+  default List<ConfigNode> children(String name) {
+    return children(null, Collections.singleton(name));
+  }
+
+  /** abortable iterate through children
+   *
+   * @param fun consume the node and return true to continue or false to abort
+   */
+  void forEachChild(Function<ConfigNode, Boolean> fun);
+
+
+}
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/DOMUtil.java b/solr/solrj/src/java/org/apache/solr/common/util/DOMUtil.java
index c86fcea..d0b1f7b 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/DOMUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/DOMUtil.java
@@ -22,7 +22,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.w3c.dom.NamedNodeMap;
 import org.w3c.dom.Node;
@@ -37,9 +39,23 @@ public class DOMUtil {
 
   public static final String XML_RESERVED_PREFIX = "xml";
 
+  public static final Set<String>  NL_TAGS = Set.of("str", "int","long","float","double","bool");
+
+
   public static Map<String,String> toMap(NamedNodeMap attrs) {
     return toMapExcept(attrs);
   }
+  public static Map<String,String> toMap(ConfigNode node) {
+    return toMapExcept(node);
+  }
+  public static Map<String,String> toMapExcept(ConfigNode node, String... exclusions) {
+    Map<String,String> args = new HashMap<>();
+    node.attributes().forEachEntry((k, v) -> {
+      for (String ex : exclusions) if (ex.equals(k)) return;
+        args.put(k,v);
+    });
+    return args;
+  }
 
   public static Map<String,String> toMapExcept(NamedNodeMap attrs, String... exclusions) {
     Map<String,String> args = new HashMap<>();
@@ -92,6 +108,16 @@ public class DOMUtil {
     return val;
   }
 
+  public static String getAttr(ConfigNode node, String name, String missing_err) {
+    String attr = node.attributes().get(name);
+    if (attr == null) {
+      if (missing_err == null) return null;
+      throw new RuntimeException(missing_err + ": missing mandatory attribute '" + name + "'");
+    }
+    return attr;
+
+  }
+
   public static String getAttr(Node node, String name, String missing_err) {
     return getAttr(node.getAttributes(), name, missing_err);
   }
@@ -110,6 +136,10 @@ public class DOMUtil {
     return nodesToList(nd.getChildNodes());
   }
 
+  public static NamedList<Object> childNodesToNamedList(ConfigNode node) {
+    return readNamedListChildren(node);
+  }
+
   public static NamedList<Object> nodesToNamedList(NodeList nlst) {
     NamedList<Object> clst = new NamedList<>();
     for (int i=0; i<nlst.getLength(); i++) {
@@ -160,35 +190,66 @@ public class DOMUtil {
       val = childNodesToList(nd);
     } else {
       final String textValue = getText(nd);
-      try {
-        if ("str".equals(type)) {
-          val = textValue;
-        } else if ("int".equals(type)) {
-          val = Integer.valueOf(textValue);
-        } else if ("long".equals(type)) {
-          val = Long.valueOf(textValue);
-        } else if ("float".equals(type)) {
-          val = Float.valueOf(textValue);
-        } else if ("double".equals(type)) {
-          val = Double.valueOf(textValue);
-        } else if ("bool".equals(type)) {
-          val = StrUtils.parseBool(textValue);
-        }
-        // :NOTE: Unexpected Node names are ignored
-        // :TODO: should we generate an error here?
-      } catch (NumberFormatException nfe) {
-        throw new SolrException
-          (SolrException.ErrorCode.SERVER_ERROR,
-           "Value " + (null != name ? ("of '" +name+ "' ") : "") +
-           "can not be parsed as '" +type+ "': \"" + textValue + "\"",
-           nfe);
-      }
+      val = parseVal(type, name, textValue);
     }
 
     if (nlst != null) nlst.add(name,val);
     if (arr != null) arr.add(val);
   }
 
+  private static Object parseVal(String type, String name, String textValue) {
+    Object val = null;
+    try {
+      if ("str".equals(type)) {
+        val = textValue;
+      } else if ("int".equals(type)) {
+        val = Integer.valueOf(textValue);
+      } else if ("long".equals(type)) {
+        val = Long.valueOf(textValue);
+      } else if ("float".equals(type)) {
+        val = Float.valueOf(textValue);
+      } else if ("double".equals(type)) {
+        val = Double.valueOf(textValue);
+      } else if ("bool".equals(type)) {
+        val = StrUtils.parseBool(textValue);
+      }
+      // :NOTE: Unexpected Node names are ignored
+      // :TODO: should we generate an error here?
+    } catch (NumberFormatException nfe) {
+      throw new SolrException
+        (SolrException.ErrorCode.SERVER_ERROR,
+         "Value " + (null != name ? ("of '" + name + "' ") : "") +
+         "can not be parsed as '" + type + "': \"" + textValue + "\"",
+         nfe);
+    }
+    return val;
+  }
+
+  public static NamedList<Object> readNamedListChildren(ConfigNode configNode) {
+    NamedList<Object> result = new NamedList<>();
+    configNode.forEachChild(it -> {
+      String tag = it.name();
+      String varName = it.attributes().get("name");
+      if (NL_TAGS.contains(tag)) {
+        result.add(varName, parseVal(tag, varName, it.textValue()));
+      }
+      if ("lst".equals(tag)) {
+        result.add(varName, readNamedListChildren(it));
+      } else if ("arr".equals(tag)) {
+        List<Object> l = new ArrayList<>();
+        result.add(varName, l);
+        it.forEachChild(n -> {
+          if (NL_TAGS.contains(n.name())) {
+            l.add(parseVal(n.name(), null, n.textValue()));
+          }
+          return Boolean.TRUE;
+        });
+      }
+      return Boolean.TRUE;
+    });
+    return result;
+  }
+
   /**
    * Drop in replacement for Node.getTextContent().
    *
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/PropertiesUtil.java b/solr/solrj/src/java/org/apache/solr/common/util/PropertiesUtil.java
index 7650a5c..9085ffc 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/PropertiesUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/PropertiesUtil.java
@@ -22,17 +22,23 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
+import java.util.function.Function;
+
 
 /**
  * Breaking out some utility methods into a separate class as part of SOLR-4196. These utils have nothing to do with
  * the DOM (they came from DomUtils) and it's really confusing to see them in something labeled DOM
  */
 public class PropertiesUtil {
+  public static String substituteProperty(String value, Properties coreProperties) {
+    if(coreProperties == null) return substitute(value, null);
+    return substitute(value, coreProperties::getProperty);
+  }
   /*
   * This method borrowed from Ant's PropertyHelper.replaceProperties:
   *   http://svn.apache.org/repos/asf/ant/core/trunk/src/main/org/apache/tools/ant/PropertyHelper.java
   */
-  public static String substituteProperty(String value, Properties coreProperties) {
+  public static String substitute(String value, Function<String,String> coreProperties) {
     if (value == null || value.indexOf('$') == -1) {
       return value;
     }
@@ -56,7 +62,7 @@ public class PropertiesUtil {
           propertyName = propertyName.substring(0, colon_index);
         }
         if (coreProperties != null) {
-          fragment = coreProperties.getProperty(propertyName);
+          fragment = coreProperties.apply(propertyName);
         }
         if (fragment == null) {
           fragment = System.getProperty(propertyName, defaultValue);