You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2021/03/02 03:01:52 UTC

[lucene-solr] 02/02: @1420 Finish up more xpath config stuff, tweak replica id code.

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

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

commit 42216fadacbf4dc64925a0f0d03dee7d8378ee2e
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Mon Mar 1 20:58:53 2021 -0600

    @1420 Finish up more xpath config stuff, tweak replica id code.
---
 .../java/org/apache/solr/cloud/LeaderElector.java  |  23 +-
 .../solr/cloud/ShardLeaderElectionContext.java     |   2 +-
 .../java/org/apache/solr/cloud/StatePublisher.java |   7 +-
 .../java/org/apache/solr/cloud/ZkController.java   |   3 +-
 .../apache/solr/cloud/ZkSolrResourceLoader.java    |   2 +-
 .../solr/cloud/api/collections/AddReplicaCmd.java  |   3 +-
 .../java/org/apache/solr/core/ConfigOverlay.java   |   4 +
 .../org/apache/solr/core/ConfigSetService.java     |   2 +-
 .../apache/solr/core/ConfigXpathExpressions.java   | 575 +++++++++++++++++++++
 .../java/org/apache/solr/core/CoreContainer.java   |  24 -
 .../src/java/org/apache/solr/core/SolrConfig.java  | 121 ++---
 .../org/apache/solr/core/SolrResourceLoader.java   | 263 +---------
 .../java/org/apache/solr/core/SolrTinyBuilder.java |  27 +-
 .../java/org/apache/solr/core/SolrXmlConfig.java   |  28 +-
 .../java/org/apache/solr/core/XmlConfigFile.java   |  20 +-
 .../java/org/apache/solr/pkg/PackageLoader.java    |   4 +-
 .../apache/solr/schema/FieldTypePluginLoader.java  |  14 +-
 .../java/org/apache/solr/schema/IndexSchema.java   |  22 +-
 .../java/org/apache/solr/search/CacheConfig.java   |  26 +-
 .../org/apache/solr/update/SolrIndexConfig.java    |  82 +--
 .../java/org/apache/solr/update/VersionInfo.java   |   5 +-
 .../java/org/apache/solr/util/PropertiesUtil.java  |  33 +-
 .../apache/solr/core/TestSolrConfigHandler.java    |   3 +-
 .../org/apache/solr/util/TestSystemIdResolver.java |   4 +-
 24 files changed, 802 insertions(+), 495 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
index 8637f2d..8ac4d79 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -20,7 +20,6 @@ import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.SolrZooKeeper;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.logging.MDCLoggingContext;
@@ -67,13 +66,13 @@ public class LeaderElector implements Closeable {
   public final static Pattern LEADER_SEQ = Pattern.compile(".*?/?.*?-n_(\\d+)");
   private final static Pattern SESSION_ID = Pattern.compile(".*?/?(.*?-.*?)-n_\\d+");
 
-  private static final String JOIN = "j1";
-  private static final String CHECK_IF_LEADER = "lc";
-  private static final String OUT_OF_ELECTION = "o";
-  private static final String POT_LEADER = "pt";
-  private static final String LEADER = "l";
-  private static final String CLOSED = "c";
-  private static final String WAITING_IN_ELECTION = "w";
+  private static final char JOIN = 'j';
+  private static final char CHECK_IF_LEADER = 'k';
+  private static final char OUT_OF_ELECTION = 'o';
+  private static final char POT_LEADER = 'p';
+  private static final char LEADER = 'l';
+  private static final char CLOSED = 'c';
+  private static final char WAITING_IN_ELECTION = 'w';
 
   protected final SolrZkClient zkClient;
   private final ZkController zkController;
@@ -88,7 +87,7 @@ public class LeaderElector implements Closeable {
 
   private final ExecutorService executor = ParWork.getExecutorService(1);
 
-  private volatile String state = OUT_OF_ELECTION;
+  private volatile char state = OUT_OF_ELECTION;
 
   //  public LeaderElector(SolrZkClient zkClient) {
 //    this.zkClient = zkClient;
@@ -514,7 +513,7 @@ public class LeaderElector implements Closeable {
     return isClosed;
   }
 
-  public String getState() {
+  public char getState() {
     return state;
   }
 
@@ -538,7 +537,7 @@ public class LeaderElector implements Closeable {
 
       if (log.isDebugEnabled()) log.debug("Got event on node we where watching in leader line {} watchedNode={}", myNode, watchedNode);
 
-      if (state.equals(LEADER)) {
+      if (state == LEADER) {
         log.info("Election watcher fired, but we are already leader");
         return;
       }
@@ -626,6 +625,6 @@ public class LeaderElector implements Closeable {
   }
 
   public boolean isLeader() {
-    return LEADER.equals(state);
+    return LEADER == state;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
index 5bf92b4..923a6ce 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
@@ -270,7 +270,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         }
 
         ZkNodeProps zkNodes = ZkNodeProps
-            .fromKeyVals(StatePublisher.OPERATION, OverseerAction.STATE.toLower(), ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.CORE_NAME_PROP, leaderProps.getName(),
+            .fromKeyVals(StatePublisher.OPERATION, OverseerAction.STATE.toLower(), ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.CORE_NAME_PROP, leaderProps.getName(), "id", leaderProps.getId(),
                 ZkStateReader.STATE_PROP, "leader");
 
         log.info("I am the new leader, publishing as active: " + leaderProps.getCoreUrl() + " " + shardId);
diff --git a/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java b/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
index 845b1aa..fa66fe0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
+++ b/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
@@ -196,7 +196,7 @@ public class StatePublisher implements Closeable {
     try {
       if (stateMessage != TERMINATE_OP) {
         String operation = stateMessage.getStr(OPERATION);
-        String id = null;
+        String id = stateMessage.getStr("id");
         if (operation.equals("state")) {
           String core = stateMessage.getStr(ZkStateReader.CORE_NAME_PROP);
           String collection = stateMessage.getStr(ZkStateReader.COLLECTION_PROP);
@@ -207,11 +207,12 @@ public class StatePublisher implements Closeable {
             Replica replica = coll.getReplica(core);
             id = replica.getId();
             String lastState = stateCache.get(id);
-            if (collection != null && replica != null && !state.equals(Replica.State.ACTIVE) && state.equals(lastState) && replica.getState().toString().equals(state)) {
+            if (collection != null && !state.equals(Replica.State.ACTIVE) && state.equals(lastState) && replica.getState().toString().equals(state)) {
               log.info("Skipping publish state as {} for {}, because it was the last state published", state, core);
               return;
             }
           }
+
           if (core == null || state == null) {
             log.error("Nulls in published state");
             throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Nulls in published state " + stateMessage);
@@ -223,7 +224,7 @@ public class StatePublisher implements Closeable {
         } else if (operation.equalsIgnoreCase(OverseerAction.DOWNNODE.toLower())) {
           // set all statecache entries for replica to a state
 
-          Collection<CoreDescriptor> cds= cc.getCoreDescriptors();
+          Collection<CoreDescriptor> cds = cc.getCoreDescriptors();
           for (CoreDescriptor cd : cds) {
             DocCollection doc = zkStateReader.getClusterState().getCollectionOrNull(cd.getCollectionName());
             Replica replica = null;
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 4363bc1..9c8fe19 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1533,7 +1533,7 @@ public class ZkController implements Closeable, Runnable {
     // we only put a subset of props into the leader node
     props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
     props.put(CORE_NAME_PROP, cd.getName());
-    props.put("id", "-1");
+    props.put("id", cd.getCoreProperty("id", "-1"));
 
     Replica replica = new Replica(cd.getName(), props, collection, shardId, zkStateReader);
     LeaderElector leaderElector;
@@ -1600,6 +1600,7 @@ public class ZkController implements Closeable, Runnable {
 
       props.put(Overseer.QUEUE_OPERATION, "state");
       props.put(ZkStateReader.STATE_PROP, state.toString());
+      props.put("id", cd.getCoreProperty("id", "-1"));
       //  props.put(ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles());
       props.put(CORE_NAME_PROP, cd.getName());
       //  props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
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 f7bbccc..981fac6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
@@ -52,7 +52,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader implements Resource
    * otherwise it will attempt to resolve resources using any jar files found in
    * the "lib/" directory in the specified instance directory.
    */
-  public ZkSolrResourceLoader(Path instanceDir, String configSet, ClassLoader parent,
+  public ZkSolrResourceLoader(Path instanceDir, String configSet, SolrResourceLoader parent,
                               ZkController zooKeeperController) {
     super(instanceDir, parent);
     this.zkClient = zooKeeperController.getZkClient();
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
index 7e7afac..0a66b8e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
@@ -268,7 +268,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     }
   }
 
-  private ModifiableSolrParams getReplicaParams(   DocCollection collection, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results,
+  private ModifiableSolrParams getReplicaParams(DocCollection collection, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results,
       boolean skipCreateReplicaInClusterState,
       ShardHandler shardHandler, CreateReplica createReplica) throws IOException, InterruptedException, KeeperException {
 
@@ -319,6 +319,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     if (bufferOnStart) {
       params.set(CoreAdminParams.PROPERTY_PREFIX + "bufferOnStart", "true");
     }
+    params.set(CoreAdminParams.PROPERTY_PREFIX + "id", Long.toString(createReplica.id));
 
     log.info("Creating SolrCore with name={}", createReplica.coreName);
     if (createReplica.sliceName != null) {
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java b/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
index 4726ae2..fe4ee89 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
@@ -57,6 +57,10 @@ public class ConfigOverlay implements MapSerializable {
   }
 
   public Object getXPathProperty(String xpath, boolean onlyPrimitive) {
+    if (xpath.startsWith("/")) {
+      xpath = xpath.substring(xpath.indexOf('/', 1) + 1);
+    }
+
     List<String> hierarchy = checkEditable(xpath, true, false);
     if (hierarchy == null) return null;
     return Utils.getObjectByPath(props, onlyPrimitive, hierarchy);
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 e88fe5e..0dea26c 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
@@ -234,7 +234,7 @@ public abstract class ConfigSetService {
     @Override
     public SolrResourceLoader createCoreResourceLoader(CoreDescriptor cd) {
       Path instanceDir = locateInstanceDir(cd);
-      return new SolrResourceLoader(instanceDir, parentLoader.getClassLoader());
+      return new SolrResourceLoader(instanceDir, parentLoader);
     }
 
     @Override
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigXpathExpressions.java b/solr/core/src/java/org/apache/solr/core/ConfigXpathExpressions.java
new file mode 100644
index 0000000..8d020a6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/ConfigXpathExpressions.java
@@ -0,0 +1,575 @@
+/*
+ * 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.core;
+
+import net.sf.saxon.Configuration;
+import net.sf.saxon.xpath.XPathFactoryImpl;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.schema.IndexSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathExpression;
+import javax.xml.xpath.XPathExpressionException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+
+public class ConfigXpathExpressions implements Closeable {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  final XPathFactoryImpl xpathFactory;
+  static String luceneMatchVersionPath = "/config/" + IndexSchema.LUCENE_MATCH_VERSION_PARAM;
+  static String indexDefaultsPath = "/config/indexDefaults";
+  static String mainIndexPath = "/config/mainIndex";
+  static String nrtModePath = "/config/indexConfig/nrtmode";
+  static String unlockOnStartupPath = "/config/indexConfig/unlockOnStartup";
+
+  static String shardHandlerFactoryPath = "/solr/shardHandlerFactory";
+  static String counterExpPath = "/solr/metrics/suppliers/counter";
+  static String meterPath = "/solr/metrics/suppliers/meter";
+  static String timerPath = "/solr/metrics/suppliers/timer";
+  static String histoPath = "/solr/metrics/suppliers/histogram";
+  static String historyPath = "/solr/metrics/history";
+  static String  transientCoreCacheFactoryPath =  "/solr/transientCoreCacheFactory";
+  static String  tracerConfigPath = "/solr/tracerConfig";
+
+  static String  coreLoadThreadsPath = "/solr/@coreLoadThreads";
+  static String  persistentPath = "/solr/@persistent";
+  static String  sharedLibPath = "/solr/@sharedLib";
+  static String  zkHostPath = "/solr/@zkHost";
+  static String  coresPath = "/solr/cores";
+
+  static String  metricsReporterPath = "/solr/metrics/reporter";
+
+
+  public static String schemaNamePath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.AT + IndexSchema.NAME);
+  public static String schemaVersionPath = "/schema/@version";
+
+  public static String copyFieldPath = "//" + IndexSchema.COPY_FIELD;
+
+  public static String fieldTypeXPathExpressions = IndexSchema.getFieldTypeXPathExpressions();
+
+  public static String schemaSimPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.SIMILARITY); //   /schema/similarity
+
+  public static String defaultSearchFieldPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, "defaultSearchField", IndexSchema.TEXT_FUNCTION);
+
+  public static String solrQueryParserDefaultOpPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, "solrQueryParser", IndexSchema.AT + "defaultOperator");
+
+  public static String schemaUniqueKeyPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.UNIQUE_KEY, IndexSchema.TEXT_FUNCTION);
+
+  public static String filterCachePath = "/config/query/filterCache";
+  public static String queryResultCachePath = "/config/query/queryResultCache";
+  public static String documentCachePath = "/config/query/documentCache";
+  public static String fieldValueCachePath = "/config/query/fieldValueCache";
+
+  public static String useFilterForSortedQueryPath = "/config/query/useFilterForSortedQuery";
+  public static String queryResultWindowSizePath = "/config/query/queryResultWindowSize";
+  public static String enableLazyFieldLoadingPath = "/config/query/enableLazyFieldLoading";
+  public static String queryResultMaxDocsCachedPath = "/config/query/queryResultMaxDocsCached";
+  public static String useRangeVersionsPath = "/config/peerSync/useRangeVersions";
+  public static String maxBooleanClausesPath = "/config/query/maxBooleanClauses";
+  public static String useColdSearcherPath = "/config/query/useColdSearcher";
+
+  public static String  multipartUploadLimitInKBPath = "/config/requestDispatcher/requestParsers/@multipartUploadLimitInKB";
+  public static String  formdataUploadLimitInKBPath = "/config/requestDispatcher/requestParsers/@formdataUploadLimitInKB";
+  public static String  enableRemoteStreamingPath = "/config/requestDispatcher/requestParsers/@enableRemoteStreaming";
+  public static String  enableStreamBodyPath = "/config/requestDispatcher/requestParsers/@enableStreamBody";
+
+  public static String  handleSelectPath = "/config/requestDispatcher/@handleSelect";
+  public static String  addHttpRequestToContextPath = "/config/requestDispatcher/requestParsers/@addHttpRequestToContext";
+
+  public static String  maxWarmingSearchersPath = "/config/query/maxWarmingSearchers";
+  public static String  slowQueryThresholdMillisPath = "/config/query/slowQueryThresholdMillis";
+
+  public static String  never304Path = "/config/requestDispatcher/httpCaching/@never304";
+
+  public static String  updateHandlerClassPath = "/config/updateHandler/@class";
+  public static String  autoCommitMaxDocsPath = "/config/updateHandler/autoCommit/maxDocs";
+  public static String  autoCommitMaxSizePath = "/config/updateHandler/autoCommit/maxSize";
+  public static String  autoCommitMaxTimePath = "/config/updateHandler/autoCommit/maxTime";
+  public static String  indexWriterCloseWaitsForMergesPath = "/config/updateHandler/indexWriter/closeWaitsForMerge";
+  public static String  autoCommitOpenSearcherPath = "/config/updateHandler/autoCommit/openSearcher";
+  public static String  autoSoftCommitMaxTimePath = "/config/updateHandler/autoSoftCommit/maxTime";
+  public static String  autoSoftCommitMaxDocsPath = "/config/updateHandler/autoSoftCommit/maxDocs";
+  public static String  commitWithinSoftCommitPath = "/config/updateHandler/commitWithin/softCommit";
+
+  public static String  useCompoundFilePath = "/config/indexConfig/useCompoundFile";
+  public static String  maxBufferedDocsPath = "/config/indexConfig/maxBufferedDocs";
+  public static String  ramPerThreadHardLimitMBPath = "/config/indexConfig/ramPerThreadHardLimitMB";
+  public static String  writeLockTimeoutPath = "/config/indexConfig/writeLockTimeout";
+
+  public static String  maxMergeDocPath = "/config/indexConfig/maxMergeDocs";
+  public static String  mergeFactorPath = "/config/indexConfig/mergeFactor";
+  public static String  infoStreamPath = "/config/indexConfig/infoStream";
+
+  public static String indexConfigPath = "/config/indexConfig";
+  public static String mergeSchedulerPath = indexConfigPath + "/mergeScheduler";
+  public static String mergePolicyPath = indexConfigPath + "/mergePolicy";
+  public static String ramBufferSizeMBPath = indexConfigPath + "/ramBufferSizeMB";
+  public static String checkIntegrityAtMergePath = indexConfigPath + "/checkIntegrityAtMerge";
+
+  public static String  versionBucketLockTimeoutMsPath = "/config/updateHandler/versionBucketLockTimeoutMs";
+
+  public XPathExpression shardHandlerFactoryExp;
+  public XPathExpression counterExp;
+  public XPathExpression meterExp;
+  public XPathExpression timerExp;
+  public XPathExpression histoExp;
+  public XPathExpression historyExp;
+  public XPathExpression transientCoreCacheFactoryExp;
+  public XPathExpression tracerConfigExp;
+
+  public XPathExpression coreLoadThreadsExp;
+  public XPathExpression persistentExp;
+  public XPathExpression sharedLibExp;
+  public XPathExpression zkHostExp;
+  public XPathExpression coresExp;
+
+  public XPathExpression xpathOrExp;
+  public XPathExpression schemaNameExp;
+  public XPathExpression schemaVersionExp;
+  public XPathExpression schemaSimExp;
+  public XPathExpression defaultSearchFieldExp;
+  public XPathExpression solrQueryParserDefaultOpExp;
+  public XPathExpression schemaUniqueKeyExp;
+  public XPathExpression fieldTypeXPathExpressionsExp;
+  public XPathExpression copyFieldsExp;
+
+  public XPathExpression luceneMatchVersionExp;
+  public XPathExpression indexDefaultsExp;
+  public XPathExpression mainIndexExp;
+  public XPathExpression nrtModeExp;
+  public XPathExpression unlockOnStartupExp;
+
+  public XPathExpression metricsReporterExp;
+
+  public XPathExpression analyzerQueryExp;
+  public XPathExpression analyzerMultiTermExp;
+
+  public XPathExpression analyzerIndexExp;
+  public XPathExpression similarityExp;
+  public XPathExpression charFilterExp;
+  public XPathExpression tokenizerExp;
+  public XPathExpression filterExp;
+
+  public XPathExpression filterCacheExp;
+  public XPathExpression documentCacheExp;
+  public XPathExpression queryResultCacheExp;
+  public XPathExpression fieldValueCacheExp;
+
+
+  public XPathExpression useFilterForSortedQueryExp;
+  public XPathExpression queryResultWindowSizeeExp;
+  public XPathExpression enableLazyFieldLoadingExp;
+  public XPathExpression queryResultMaxDocsCachedExp;
+  public XPathExpression useRangeVersionsExp;
+  public XPathExpression maxBooleanClausesExp;
+  public XPathExpression useColdSearcherExp;
+
+  public XPathExpression multipartUploadLimitInKBExp;
+  public XPathExpression formdataUploadLimitInKBExp;
+  public XPathExpression enableRemoteStreamingExp;
+  public XPathExpression enableStreamBodyExp;
+
+  public XPathExpression handleSelectExp;
+  public XPathExpression addHttpRequestToContextExp;
+
+  public XPathExpression maxWarmingSearchersExp;
+  public XPathExpression never304Exp;
+  public XPathExpression slowQueryThresholdMillisExp;
+
+  public XPathExpression  updateHandlerClassExp;
+  public XPathExpression autoCommitMaxDocsExp;
+  public XPathExpression  autoCommitMaxSizeExp;
+  public XPathExpression autoCommitMaxTimeExp;
+  public XPathExpression  indexWriterCloseWaitsForMergesExp;
+  public XPathExpression autoCommitOpenSearcherExp;
+  public XPathExpression autoSoftCommitMaxTimeExp;
+  public XPathExpression autoSoftCommitMaxDocsExp;
+  public XPathExpression  commitWithinSoftCommitExp;
+
+  public XPathExpression useCompoundFileExp;
+  public XPathExpression maxBufferedDocsExp;
+  public XPathExpression ramPerThreadHardLimitMBExp;
+  public XPathExpression writeLockTimeoutExp;
+
+  public XPathExpression maxMergeDocsExp;
+  public XPathExpression infoStreamExp;
+  public XPathExpression mergeFactorExp;
+
+  public XPathExpression indexConfigExp;
+  public XPathExpression mergeSchedulerExp;
+  public XPathExpression mergePolicyExp;
+  public XPathExpression ramBufferSizeMBExp;
+  public XPathExpression checkIntegrityAtMergeExp;
+
+  public XPathExpression versionBucketLockTimeoutMsExp;
+
+  Configuration conf;
+
+  com.fasterxml.aalto.sax.SAXParserFactoryImpl parser = new com.fasterxml.aalto.sax.SAXParserFactoryImpl();
+
+
+  {
+
+    parser.setValidating(false);
+    parser.setXIncludeAware(false);
+    conf = Configuration.newConfiguration();
+    //conf.setSourceParserClass("com.fasterxml.aalto.sax.SAXParserFactoryImpl");
+    //    conf.setNamePool(this.conf.getNamePool());
+    //    conf.setDocumentNumberAllocator(this.conf.getDocumentNumberAllocator());
+    // conf.setXIncludeAware(true);
+    conf.setExpandAttributeDefaults(false);
+    conf.setValidation(false);
+
+    xpathFactory = new XPathFactoryImpl(conf);;
+
+    refreshConf();
+  }
+
+  public void refreshConf() {
+    try {
+
+      XPath xpath = xpathFactory.newXPath();
+
+      luceneMatchVersionExp = xpath.compile(luceneMatchVersionPath);
+
+      indexDefaultsExp = xpath.compile(indexDefaultsPath);
+
+      mainIndexExp = xpath.compile(mainIndexPath);
+
+      nrtModeExp = xpath.compile(nrtModePath);
+
+      unlockOnStartupExp = xpath.compile(unlockOnStartupPath);
+
+      shardHandlerFactoryExp = xpath.compile(shardHandlerFactoryPath);
+
+      counterExp = xpath.compile(counterExpPath);
+
+      meterExp = xpath.compile(meterPath);
+
+      timerExp = xpath.compile(timerPath);
+
+      histoExp = xpath.compile(histoPath);
+
+      historyExp = xpath.compile(historyPath);
+
+      transientCoreCacheFactoryExp = xpath.compile(transientCoreCacheFactoryPath);
+
+      tracerConfigExp = xpath.compile(tracerConfigPath);
+
+      coreLoadThreadsExp = xpath.compile(coreLoadThreadsPath);
+
+      persistentExp = xpath.compile(persistentPath);
+
+      sharedLibExp = xpath.compile(sharedLibPath);
+
+      zkHostExp = xpath.compile(zkHostPath);
+
+      coresExp = xpath.compile(coresPath);
+
+      String expression =
+          IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.FIELD) + IndexSchema.XPATH_OR + IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.DYNAMIC_FIELD) + IndexSchema.XPATH_OR
+              + IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.FIELDS, IndexSchema.FIELD) + IndexSchema.XPATH_OR + IndexSchema
+              .stepsToPath(IndexSchema.SCHEMA, IndexSchema.FIELDS, IndexSchema.DYNAMIC_FIELD);
+      xpathOrExp = xpath.compile(expression);
+
+      schemaNameExp = xpath.compile(schemaNamePath);
+
+      schemaVersionExp = xpath.compile(schemaVersionPath);
+
+      schemaSimExp = xpath.compile(schemaSimPath);
+
+      defaultSearchFieldExp = xpath.compile(defaultSearchFieldPath);
+
+      solrQueryParserDefaultOpExp = xpath.compile(solrQueryParserDefaultOpPath);
+
+      schemaUniqueKeyExp = xpath.compile(schemaUniqueKeyPath);
+
+      fieldTypeXPathExpressionsExp = xpath.compile(fieldTypeXPathExpressions);
+
+      copyFieldsExp = xpath.compile(copyFieldPath);
+
+      metricsReporterExp = xpath.compile(metricsReporterPath);
+
+      try {
+        analyzerQueryExp = xpath.compile("./analyzer[@type='query']");
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        analyzerMultiTermExp = xpath.compile("./analyzer[@type='multiterm']");
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+
+      try {
+        analyzerIndexExp = xpath.compile("./analyzer[not(@type)] | ./analyzer[@type='index']");
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        similarityExp = xpath.compile("./similarity");
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        charFilterExp = xpath.compile("./charFilter");
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        tokenizerExp = xpath.compile("./tokenizer");
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        filterExp = xpath.compile("./filter");
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        filterCacheExp = xpath.compile(filterCachePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        documentCacheExp = xpath.compile(documentCachePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        queryResultCacheExp = xpath.compile(queryResultCachePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        fieldValueCacheExp = xpath.compile(fieldValueCachePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+
+      try {
+        useFilterForSortedQueryExp = xpath.compile(useFilterForSortedQueryPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        queryResultWindowSizeeExp = xpath.compile(queryResultWindowSizePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        enableLazyFieldLoadingExp = xpath.compile(enableLazyFieldLoadingPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        queryResultMaxDocsCachedExp = xpath.compile(queryResultMaxDocsCachedPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        useRangeVersionsExp = xpath.compile(useRangeVersionsPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        maxBooleanClausesExp = xpath.compile(maxBooleanClausesPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        useColdSearcherExp = xpath.compile(useColdSearcherPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+
+      try {
+        multipartUploadLimitInKBExp = xpath.compile(multipartUploadLimitInKBPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        formdataUploadLimitInKBExp = xpath.compile(formdataUploadLimitInKBPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        enableRemoteStreamingExp = xpath.compile(enableRemoteStreamingPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        enableStreamBodyExp = xpath.compile(enableStreamBodyPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        handleSelectExp = xpath.compile(handleSelectPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        addHttpRequestToContextExp = xpath.compile(addHttpRequestToContextPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        maxWarmingSearchersExp = xpath.compile(maxWarmingSearchersPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        never304Exp = xpath.compile(never304Path);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        slowQueryThresholdMillisExp = xpath.compile(slowQueryThresholdMillisPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+
+
+      try {
+        updateHandlerClassExp = xpath.compile(updateHandlerClassPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        autoCommitMaxDocsExp = xpath.compile(autoCommitMaxDocsPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        autoCommitMaxTimeExp = xpath.compile(autoCommitMaxTimePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        autoCommitMaxSizeExp = xpath.compile(autoCommitMaxSizePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        indexWriterCloseWaitsForMergesExp = xpath.compile(indexWriterCloseWaitsForMergesPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        autoCommitOpenSearcherExp = xpath.compile(autoCommitOpenSearcherPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        autoSoftCommitMaxTimeExp = xpath.compile(autoSoftCommitMaxTimePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        autoSoftCommitMaxDocsExp = xpath.compile(autoSoftCommitMaxDocsPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        commitWithinSoftCommitExp = xpath.compile(commitWithinSoftCommitPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+
+
+      try {
+        useCompoundFileExp = xpath.compile(useCompoundFilePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        maxBufferedDocsExp = xpath.compile(maxBufferedDocsPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        ramPerThreadHardLimitMBExp = xpath.compile(ramPerThreadHardLimitMBPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        writeLockTimeoutExp = xpath.compile(writeLockTimeoutPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+
+      try {
+        maxMergeDocsExp = xpath.compile(maxMergeDocPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        infoStreamExp = xpath.compile(infoStreamPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        mergeFactorExp = xpath.compile(mergeFactorPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        indexConfigExp = xpath.compile(indexConfigPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        mergeSchedulerExp = xpath.compile(mergeSchedulerPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        mergePolicyExp = xpath.compile(mergePolicyPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        ramBufferSizeMBExp = xpath.compile(ramBufferSizeMBPath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+      try {
+        checkIntegrityAtMergeExp = xpath.compile(checkIntegrityAtMergePath);
+      } catch (XPathExpressionException e) {
+        log.error("", e);
+      }
+    } catch (Exception e) {
+      log.error("", e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    }
+  }
+
+  @Override public void close() throws IOException {
+    if (conf != null) {
+      try {
+        conf.close();
+      } catch (Exception e) {
+        log.info("Exception closing Configuration " + e.getClass().getName() + " " + e.getMessage());
+      }
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index ddd0bd2..af5ae29 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -27,7 +27,6 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
 import org.apache.solr.client.solrj.impl.CloudHttp2SolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
@@ -93,7 +92,6 @@ import org.apache.solr.security.SecurityPluginHolder;
 import org.apache.solr.update.SolrCoreState;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.util.RefCounted;
-import org.apache.solr.util.SystemIdResolver;
 import org.apache.zookeeper.KeeperException;
 import org.eclipse.jetty.util.BlockingArrayQueue;
 import org.slf4j.Logger;
@@ -141,7 +139,6 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Consumer;
@@ -252,27 +249,6 @@ public class CoreContainer implements Closeable {
   private PackageStoreAPI packageStoreAPI;
   private PackageLoader packageLoader;
 
-  // private Set<Future> zkRegFutures = zkRegFutures = ConcurrentHashMap.newKeySet();
-
-  private SystemIdResolver sysIdResolver;
- // public XPathFactoryImpl xpathFactory = new XPathFactoryImpl();
-
-
-  {
-//    PipelineConfiguration plc = XmlConfigFile.conf1.makePipelineConfiguration();
-//
-//    ParseOptions po = plc.getParseOptions();
-//    // if (is.getSystemId() != null) {
-//    po.setEntityResolver(CoreContainer..getSysIdResolver());
-//    // }
-//
-//    po.setXIncludeAware(true);
-//    po.setSchemaValidationMode(0);
-//    po.setExpandAttributeDefaults(true);
-//    //conf.setURIResolver(sysIdResolver.asURIResolver());
-//    xpathFactory.setConfiguration(plc.getConfiguration());
-  }
-
   // Bits for the state variable.
   public final static long LOAD_COMPLETE = 0x1L;
   public final static long CORE_DISCOVERY_COMPLETE = 0x2L;
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index fb52f37..26315eb 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -21,6 +21,7 @@ import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.stream.XMLStreamException;
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpression;
 import javax.xml.xpath.XPathExpressionException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -190,21 +191,21 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     getOverlay();//just in case it is not initialized
     getRequestParams();
     initLibs(loader, isConfigsetTrusted);
-    luceneMatchVersion = SolrConfig.parseLuceneVersionString(getVal(loader.luceneMatchVersionExp, SolrResourceLoader.luceneMatchVersionPath, true));
+    luceneMatchVersion = SolrConfig.parseLuceneVersionString(getVal(loader.configXpathExpressions.luceneMatchVersionExp, ConfigXpathExpressions.luceneMatchVersionPath, true));
     log.info("Using Lucene MatchVersion: {}", luceneMatchVersion);
 
     String indexConfigPrefix;
 
     // Old indexDefaults and mainIndex sections are deprecated and fails fast for luceneMatchVersion=>LUCENE_4_0_0.
     // For older solrconfig.xml's we allow the old sections, but never mixed with the new <indexConfig>
-    boolean hasDeprecatedIndexConfig = (getNode(loader.indexDefaultsExp, SolrResourceLoader.indexDefaultsPath, false) != null) || (getNode(loader.mainIndexExp, SolrResourceLoader.mainIndexPath, false) != null);
+    boolean hasDeprecatedIndexConfig = (getNode(loader.configXpathExpressions.indexDefaultsExp, ConfigXpathExpressions.indexDefaultsPath, false) != null) || (getNode(loader.configXpathExpressions.mainIndexExp, ConfigXpathExpressions.mainIndexPath, false) != null);
     if (hasDeprecatedIndexConfig) {
       throw new SolrException(ErrorCode.FORBIDDEN, "<indexDefaults> and <mainIndex> configuration sections are discontinued. Use <indexConfig> instead.");
     } else {
       indexConfigPrefix = "indexConfig";
     }
     assertWarnOrFail("The <nrtMode> config has been discontinued and NRT mode is always used by Solr." +
-            " This config will be removed in future versions.", getNode(loader.nrtModeExp, SolrResourceLoader.nrtModePath, false) == null,
+            " This config will be removed in future versions.", getNode(loader.configXpathExpressions.nrtModeExp, ConfigXpathExpressions.nrtModePath, false) == null,
         true
     );
     assertWarnOrFail("Solr no longer supports forceful unlocking via the 'unlockOnStartup' option.  "+
@@ -212,14 +213,14 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
                      "it would be dangerous and should not be done.  For other lockTypes and/or "+
                      "directoryFactory options it may also be dangerous and users must resolve "+
                      "problematic locks manually.",
-                     null == getNode(loader.unlockOnStartupExp, SolrResourceLoader.unlockOnStartupPath, false),
+                     null == getNode(loader.configXpathExpressions.unlockOnStartupExp, ConfigXpathExpressions.unlockOnStartupPath, false),
                      true // 'fail' in trunk
                      );
                      
     // Parse indexConfig section, using mainIndex as backup in case old config is used
     indexConfig = new SolrIndexConfig(this, "indexConfig", null);
 
-    booleanQueryMaxClauseCount = getInt("query/maxBooleanClauses", IndexSearcher.getMaxClauseCount());
+    booleanQueryMaxClauseCount = getInt(loader.configXpathExpressions.maxBooleanClausesExp, ConfigXpathExpressions.maxBooleanClausesPath, IndexSearcher.getMaxClauseCount());
     if (IndexSearcher.getMaxClauseCount() < booleanQueryMaxClauseCount) {
       log.warn("solrconfig.xml: <maxBooleanClauses> of {} is greater than global limit of {} {}"
           , booleanQueryMaxClauseCount, IndexSearcher.getMaxClauseCount()
@@ -238,17 +239,17 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
 //    filtOptCacheSize = getInt("query/boolTofilterOptimizer/@cacheSize",32);
 //    filtOptThreshold = getFloat("query/boolTofilterOptimizer/@threshold",.05f);
 
-    useFilterForSortedQuery = getBool("query/useFilterForSortedQuery", false);
-    queryResultWindowSize = Math.max(1, getInt("query/queryResultWindowSize", 1));
-    queryResultMaxDocsCached = getInt("query/queryResultMaxDocsCached", Integer.MAX_VALUE);
-    enableLazyFieldLoading = getBool("query/enableLazyFieldLoading", false);
+    useFilterForSortedQuery = getBool(loader.configXpathExpressions.useFilterForSortedQueryExp, ConfigXpathExpressions.useFilterForSortedQueryPath, false);
+    queryResultWindowSize = Math.max(1, getInt(loader.configXpathExpressions.queryResultWindowSizeeExp, ConfigXpathExpressions.queryResultWindowSizePath, 1));
+    queryResultMaxDocsCached = getInt(loader.configXpathExpressions.queryResultMaxDocsCachedExp, ConfigXpathExpressions.queryResultMaxDocsCachedPath, Integer.MAX_VALUE);
+    enableLazyFieldLoading = getBool(loader.configXpathExpressions.enableLazyFieldLoadingExp, ConfigXpathExpressions.enableLazyFieldLoadingPath, false);
     
-    useRangeVersionsForPeerSync = getBool("peerSync/useRangeVersions", true);
+    useRangeVersionsForPeerSync = getBool(loader.configXpathExpressions.useRangeVersionsExp, ConfigXpathExpressions.useRangeVersionsPath, true);
 
-    filterCacheConfig = CacheConfig.getConfig(this, "query/filterCache");
-    queryResultCacheConfig = CacheConfig.getConfig(this, "query/queryResultCache");
-    documentCacheConfig = CacheConfig.getConfig(this, "query/documentCache");
-    CacheConfig conf = CacheConfig.getConfig(this, "query/fieldValueCache");
+    filterCacheConfig = CacheConfig.getConfig(this, ConfigXpathExpressions.filterCachePath, loader.configXpathExpressions.filterCacheExp);
+    queryResultCacheConfig = CacheConfig.getConfig(this, ConfigXpathExpressions.queryResultCachePath, loader.configXpathExpressions.queryResultCacheExp);
+    documentCacheConfig = CacheConfig.getConfig(this, ConfigXpathExpressions.documentCachePath, loader.configXpathExpressions.documentCacheExp);
+    CacheConfig conf = CacheConfig.getConfig(this, ConfigXpathExpressions.fieldValueCachePath, loader.configXpathExpressions.fieldValueCacheExp);
     if (conf == null) {
       Map<String, String> args = new HashMap<>();
       args.put(NAME, "fieldValueCache");
@@ -258,7 +259,7 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
       conf = new CacheConfig(CaffeineCache.class, args, null);
     }
     fieldValueCacheConfig = conf;
-    useColdSearcher = getBool("query/useColdSearcher", false);
+    useColdSearcher = getBool(loader.configXpathExpressions.useColdSearcherExp, ConfigXpathExpressions.useColdSearcherPath, false);
     dataDir = get("dataDir", null);
     if (dataDir != null && dataDir.length() == 0) dataDir = null;
 
@@ -271,8 +272,8 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
 
     httpCachingConfig = new HttpCachingConfig(this);
 
-    maxWarmingSearchers = getInt("query/maxWarmingSearchers", 1);
-    slowQueryThresholdMillis = getInt("query/slowQueryThresholdMillis", -1);
+    maxWarmingSearchers = getInt(loader.configXpathExpressions.maxWarmingSearchersExp, ConfigXpathExpressions.maxWarmingSearchersPath, 1);
+    slowQueryThresholdMillis = getInt(loader.configXpathExpressions.slowQueryThresholdMillisExp, ConfigXpathExpressions.slowQueryThresholdMillisPath, -1);
     for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
 
     Map<String, CacheConfig> userCacheConfigs = CacheConfig.getMultipleConfigs(this, "query/cache");
@@ -287,24 +288,24 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     updateHandlerInfo = loadUpdatehandlerInfo();
 
     multipartUploadLimitKB = getInt(
-        "requestDispatcher/requestParsers/@multipartUploadLimitInKB", Integer.MAX_VALUE);
+        loader.configXpathExpressions.multipartUploadLimitInKBExp, ConfigXpathExpressions.multipartUploadLimitInKBPath, Integer.MAX_VALUE);
     if (multipartUploadLimitKB == -1) multipartUploadLimitKB = Integer.MAX_VALUE;
 
     formUploadLimitKB = getInt(
-        "requestDispatcher/requestParsers/@formdataUploadLimitInKB", Integer.MAX_VALUE);
+        loader.configXpathExpressions.formdataUploadLimitInKBExp, ConfigXpathExpressions.formdataUploadLimitInKBPath, Integer.MAX_VALUE);
     if (formUploadLimitKB == -1) formUploadLimitKB = Integer.MAX_VALUE;
 
     enableRemoteStreams = getBool(
-        "requestDispatcher/requestParsers/@enableRemoteStreaming", false);
+        loader.configXpathExpressions.enableRemoteStreamingExp, ConfigXpathExpressions.enableRemoteStreamingPath, false);
 
     enableStreamBody = getBool(
-        "requestDispatcher/requestParsers/@enableStreamBody", false);
+        loader.configXpathExpressions.enableStreamBodyExp, ConfigXpathExpressions.enableStreamBodyPath, false);
 
     handleSelect = getBool(
-        "requestDispatcher/@handleSelect", false);
+        loader.configXpathExpressions.handleSelectExp, ConfigXpathExpressions.handleSelectPath, false);
 
     addHttpRequestToContext = getBool(
-        "requestDispatcher/requestParsers/@addHttpRequestToContext", false);
+       loader.configXpathExpressions.addHttpRequestToContextExp, ConfigXpathExpressions.addHttpRequestToContextPath, false);
 
     Collection<PluginInfo> argsInfos = getPluginInfos(InitParams.class.getName());
     if (argsInfos != null) {
@@ -464,14 +465,14 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
 
   protected UpdateHandlerInfo loadUpdatehandlerInfo() {
     return new UpdateHandlerInfo(get("updateHandler/@class", null),
-        getInt("updateHandler/autoCommit/maxDocs", -1),
-        getInt("updateHandler/autoCommit/maxTime", -1),
-        convertHeapOptionStyleConfigStringToBytes(get("updateHandler/autoCommit/maxSize", "")),
-        getBool("updateHandler/indexWriter/closeWaitsForMerges", true),
-        getBool("updateHandler/autoCommit/openSearcher", true),
-        getInt("updateHandler/autoSoftCommit/maxDocs", -1),
-        getInt("updateHandler/autoSoftCommit/maxTime", -1),
-        getBool("updateHandler/commitWithin/softCommit", true));
+        getInt(loader.configXpathExpressions.autoCommitMaxDocsExp, ConfigXpathExpressions.autoCommitMaxDocsPath, -1),
+        getInt(loader.configXpathExpressions.autoCommitMaxTimeExp, ConfigXpathExpressions.autoCommitMaxTimePath, -1),
+        convertHeapOptionStyleConfigStringToBytes(get(loader.configXpathExpressions.autoCommitMaxSizeExp, ConfigXpathExpressions.autoCommitMaxSizePath, "")),
+        getBool(loader.configXpathExpressions.indexWriterCloseWaitsForMergesExp, ConfigXpathExpressions.indexWriterCloseWaitsForMergesPath, true),
+        getBool(loader.configXpathExpressions.autoCommitOpenSearcherExp, ConfigXpathExpressions.autoCommitOpenSearcherPath, true),
+        getInt(loader.configXpathExpressions.autoSoftCommitMaxDocsExp, ConfigXpathExpressions.autoSoftCommitMaxDocsPath, -1),
+        getInt(loader.configXpathExpressions.autoSoftCommitMaxTimeExp, ConfigXpathExpressions.autoSoftCommitMaxTimePath, -1),
+        getBool(loader.configXpathExpressions.commitWithinSoftCommitExp, ConfigXpathExpressions.commitWithinSoftCommitPath,true));
   }
 
   /**
@@ -627,7 +628,7 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
 
     private HttpCachingConfig(SolrConfig conf) {
 
-      never304 = conf.getBool(CACHE_PRE + "@never304", false);
+      never304 = conf.getBool(conf.getResourceLoader().configXpathExpressions.never304Exp, ConfigXpathExpressions.never304Path, false);
 
       etagSeed = conf.get(CACHE_PRE + "@etagSeed", "Solr");
 
@@ -868,46 +869,37 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     return enableStreamBody;
   }
 
-  public int getInt(String path) {
-    return getInt(path, 0);
+  public int getInt(XPathExpression expression, String path) {
+    return getInt(expression, path, 0);
   }
 
-  // MRM TODO: don't like these syncs I guess?...
-  public int getInt(String path, int def) {
+  public int getInt(XPathExpression expression, String path, int def) {
     Object val;
-    synchronized (this) {
-      val = overlay.getXPathProperty(path);
-    }
+
+    val = overlay.getXPathProperty(path);
+
     if (val != null) return Integer.parseInt(val.toString());
-    try {
-      path = super.normalize(path);
-      XPath xPath = loader.getXPath();
-      return super.getInt(xPath.compile(path), path, def);
-    } catch (XPathExpressionException e) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, e);
-    }
+    path = super.normalize(path);
+    return super.getInt(expression, path, def);
   }
 
-  public boolean getBool(String path, boolean def) {
+  public boolean getBool(XPathExpression expression, String path, boolean def) {
     Object val;
-    synchronized (this) {
-      val = overlay.getXPathProperty(path);
+    if (path.contains("lazy")) {
+      log.info("");
     }
+    val = overlay.getXPathProperty(path);
+
     if (val != null) return Boolean.parseBoolean(val.toString());
-    try {
-      path = super.normalize(path);
-      XPath xPath = loader.getXPath();
-      return super.getBool(xPath.compile(path), path, def);
-    } catch (XPathExpressionException e) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, e);
-    }
+    path = super.normalize(path);
+    return super.getBool(expression, path, def);
   }
 
   public String get(String path) {
     Object val;
-    synchronized (this) {
-      val = overlay.getXPathProperty(path, true);
-    }
+
+    val = overlay.getXPathProperty(path);
+
     try {
       path = super.normalize(path);
       XPath xPath = loader.getXPath();
@@ -919,9 +911,9 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
 
   public String get(String path, String def) {
     Object val;
-    synchronized (this) {
-      val = overlay.getXPathProperty(path, true);
-    }
+
+    val = overlay.getXPathProperty(path);
+
     try {
       path = super.normalize(path);
       XPath xPath = loader.getXPath();
@@ -956,9 +948,8 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
           items.put(info.name, info);
         }
 
-        synchronized (this) {
-          for (Map.Entry e : overlay.getNamedPlugins(plugin.tag).entrySet()) items.put(e.getKey(), e.getValue());
-        }
+        for (Map.Entry e : overlay.getNamedPlugins(plugin.tag).entrySet()) items.put(e.getKey(), e.getValue());
+
         result.put(tag, items);
       } else {
         if (plugin.options.contains(MULTI_OK)) {
diff --git a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
index c7046d8..1b60739 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
@@ -48,7 +48,6 @@ import java.util.stream.Collectors;
 
 import com.google.common.collect.ImmutableMap;
 import net.sf.saxon.Configuration;
-import net.sf.saxon.xpath.XPathFactoryImpl;
 import org.apache.lucene.analysis.WordlistLoader;
 import org.apache.lucene.analysis.util.CharFilterFactory;
 import org.apache.lucene.analysis.util.ResourceLoader;
@@ -69,7 +68,6 @@ import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.rest.RestManager;
 import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.ManagedIndexSchemaFactory;
 import org.apache.solr.schema.SimilarityFactory;
 import org.apache.solr.search.QParserPlugin;
@@ -81,8 +79,6 @@ import org.slf4j.LoggerFactory;
 import org.xml.sax.XMLReader;
 
 import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathExpression;
-import javax.xml.xpath.XPathExpressionException;
 
 /**
  * @since solr 1.3
@@ -98,228 +94,10 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
   };
   private static final Charset UTF_8 = StandardCharsets.UTF_8;
   public static final URL[] EMPTY_URL_ARRAY = new URL[0];
-  private XPathFactoryImpl xpathFactory;
-  private final SystemIdResolver sysIdResolver;
-
-  public XPathFactoryImpl getXpathFactory() {
-    return xpathFactory;
-  }
-
-  static String luceneMatchVersionPath = "/config/" + IndexSchema.LUCENE_MATCH_VERSION_PARAM;
-  static String indexDefaultsPath = "/config/indexDefaults";
-  static String mainIndexPath = "/config/mainIndex";
-  static String nrtModePath = "/config/indexConfig/nrtmode";
-  static String unlockOnStartupPath = "/config/indexConfig/unlockOnStartup";
-
-  static String shardHandlerFactoryPath = "solr/shardHandlerFactory";
-  static String counterExpPath = "solr/metrics/suppliers/counter";
-  static String meterPath = "solr/metrics/suppliers/meter";
-  static String timerPath = "solr/metrics/suppliers/timer";
-  static String histoPath = "solr/metrics/suppliers/histogram";
-  static String historyPath = "solr/metrics/history";
-  static String  transientCoreCacheFactoryPath =  "solr/transientCoreCacheFactory";
-  static String  tracerConfigPath = "solr/tracerConfig";
-
-  static String  coreLoadThreadsPath = "solr/@coreLoadThreads";
-  static String  persistentPath = "solr/@persistent";
-  static String  sharedLibPath = "solr/@sharedLib";
-  static String  zkHostPath = "solr/@zkHost";
-  static String  coresPath = "solr/cores";
-
-  static String  metricsReporterPath = "solr/metrics/reporter";
-
-
-  public static String schemaNamePath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.AT + IndexSchema.NAME);
-  public static String schemaVersionPath = "/schema/@version";
-
-  public static String copyFieldPath = "//" + IndexSchema.COPY_FIELD;
-
-  public static String fieldTypeXPathExpressions = IndexSchema.getFieldTypeXPathExpressions();
-
-  public static String schemaSimPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.SIMILARITY); //   /schema/similarity
-
-  public static String defaultSearchFieldPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, "defaultSearchField", IndexSchema.TEXT_FUNCTION);
-
-  public static String solrQueryParserDefaultOpPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, "solrQueryParser", IndexSchema.AT + "defaultOperator");
-
-  public static String schemaUniqueKeyPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.UNIQUE_KEY, IndexSchema.TEXT_FUNCTION);
-
-
-  public XPathExpression shardHandlerFactoryExp;
-  public XPathExpression counterExp;
-  public XPathExpression meterExp;
-  public XPathExpression timerExp;
-  public XPathExpression histoExp;
-  public XPathExpression historyExp;
-  public XPathExpression transientCoreCacheFactoryExp;
-  public XPathExpression tracerConfigExp;
-
-  public XPathExpression coreLoadThreadsExp;
-  public XPathExpression persistentExp;
-  public XPathExpression sharedLibExp;
-  public XPathExpression zkHostExp;
-  public XPathExpression coresExp;
-
-  public XPathExpression xpathOrExp;
-  public XPathExpression schemaNameExp;
-  public XPathExpression schemaVersionExp;
-  public XPathExpression schemaSimExp;
-  public XPathExpression defaultSearchFieldExp;
-  public XPathExpression solrQueryParserDefaultOpExp;
-  public XPathExpression schemaUniqueKeyExp;
-  public XPathExpression fieldTypeXPathExpressionsExp;
-  public XPathExpression copyFieldsExp;
-
-  public XPathExpression luceneMatchVersionExp;
-  public XPathExpression indexDefaultsExp;
-  public XPathExpression mainIndexExp;
-  public XPathExpression nrtModeExp;
-  public XPathExpression unlockOnStartupExp;
-
-  public XPathExpression metricsReporterExp;
-
-  public XPathExpression analyzerQueryExp;
-  public XPathExpression analyzerMultiTermExp;
-
-  public XPathExpression analyzerIndexExp;
-  public XPathExpression similarityExp;
-  public XPathExpression charFilterExp;
-  public XPathExpression tokenizerExp;
-  public XPathExpression filterExp;
-
-  Configuration conf;
-
-  com.fasterxml.aalto.sax.SAXParserFactoryImpl parser = new com.fasterxml.aalto.sax.SAXParserFactoryImpl();
-
-
-  {
-
-    parser.setValidating(false);
-    parser.setXIncludeAware(false);
-    conf = Configuration.newConfiguration();
-    //conf.setSourceParserClass("com.fasterxml.aalto.sax.SAXParserFactoryImpl");
-//    conf.setNamePool(this.conf.getNamePool());
-//    conf.setDocumentNumberAllocator(this.conf.getDocumentNumberAllocator());
-   // conf.setXIncludeAware(true);
-    conf.setExpandAttributeDefaults(false);
-    conf.setValidation(false);
-
-    xpathFactory = new XPathFactoryImpl(conf);;
-
-    refreshConf();
-  }
-
-
-
-  public void refreshConf() {
-    try {
-
-      XPath xpath = xpathFactory.newXPath();
-
-      luceneMatchVersionExp = xpath.compile(luceneMatchVersionPath);
-
-      indexDefaultsExp = xpath.compile(indexDefaultsPath);
-
-      mainIndexExp = xpath.compile(mainIndexPath);
-
-      nrtModeExp = xpath.compile(nrtModePath);
-
-      unlockOnStartupExp = xpath.compile(unlockOnStartupPath);
-
-      shardHandlerFactoryExp = xpath.compile(shardHandlerFactoryPath);
-
-      counterExp = xpath.compile(counterExpPath);
-
-      meterExp = xpath.compile(meterPath);
-
-      timerExp = xpath.compile(timerPath);
-
-      histoExp = xpath.compile(histoPath);
 
-      historyExp = xpath.compile(historyPath);
-
-      transientCoreCacheFactoryExp = xpath.compile(transientCoreCacheFactoryPath);
-
-      tracerConfigExp = xpath.compile(tracerConfigPath);
-
-      coreLoadThreadsExp = xpath.compile(coreLoadThreadsPath);
-
-      persistentExp = xpath.compile(persistentPath);
-
-      sharedLibExp = xpath.compile(sharedLibPath);
-
-      zkHostExp = xpath.compile(zkHostPath);
-
-      coresExp = xpath.compile(coresPath);
-
-      String expression =
-          IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.FIELD) + IndexSchema.XPATH_OR + IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.DYNAMIC_FIELD) + IndexSchema.XPATH_OR
-              + IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.FIELDS, IndexSchema.FIELD) + IndexSchema.XPATH_OR + IndexSchema
-              .stepsToPath(IndexSchema.SCHEMA, IndexSchema.FIELDS, IndexSchema.DYNAMIC_FIELD);
-      xpathOrExp = xpath.compile(expression);
-
-      schemaNameExp = xpath.compile(schemaNamePath);
-
-      schemaVersionExp = xpath.compile(schemaVersionPath);
-
-      schemaSimExp = xpath.compile(schemaSimPath);
-
-      defaultSearchFieldExp = xpath.compile(defaultSearchFieldPath);
-
-      solrQueryParserDefaultOpExp = xpath.compile(solrQueryParserDefaultOpPath);
-
-      schemaUniqueKeyExp = xpath.compile(schemaUniqueKeyPath);
-
-      fieldTypeXPathExpressionsExp = xpath.compile(fieldTypeXPathExpressions);
-
-      copyFieldsExp = xpath.compile(copyFieldPath);
-
-      metricsReporterExp = xpath.compile(metricsReporterPath);
-
-      try {
-        analyzerQueryExp = xpath.compile("./analyzer[@type='query']");
-      } catch (XPathExpressionException e) {
-        log.error("", e);
-      }
-      try {
-        analyzerMultiTermExp = xpath.compile("./analyzer[@type='multiterm']");
-      } catch (XPathExpressionException e) {
-        log.error("", e);
-      }
-
-      try {
-        analyzerIndexExp = xpath.compile("./analyzer[not(@type)] | ./analyzer[@type='index']");
-      } catch (XPathExpressionException e) {
-        log.error("", e);
-      }
-      try {
-        similarityExp = xpath.compile("./similarity");
-      } catch (XPathExpressionException e) {
-        log.error("", e);
-      }
-
-
-      try {
-        charFilterExp = xpath.compile("./charFilter");
-      } catch (XPathExpressionException e) {
-        log.error("", e);
-      }
-      try {
-        tokenizerExp = xpath.compile("./tokenizer");
-      } catch (XPathExpressionException e) {
-        log.error("", e);
-      }
-      try {
-        filterExp = xpath.compile("./filter");
-      } catch (XPathExpressionException e) {
-        log.error("", e);
-      }
+  private final SystemIdResolver sysIdResolver;
 
-    } catch (Exception e) {
-      log.error("", e);
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-    }
-  }
+  public final ConfigXpathExpressions configXpathExpressions;
 
   private String name = "";
   protected volatile URLClassLoader classLoader;
@@ -333,8 +111,8 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
   // Provide a registry so that managed resources can register themselves while the XML configuration
   // documents are being parsed ... after all are registered, they are asked by the RestManager to
   // initialize themselves. This two-step process is required because not all resources are available
-  // (such as the SolrZkClient) when XML docs are being parsed.    
-  private RestManager.Registry managedResourceRegistry;
+  // (such as the SolrZkClient) when XML docs are being parsed.
+  private volatile RestManager.Registry managedResourceRegistry;
 
   /** @see #reloadLuceneSPI() */
   private volatile boolean needToReloadLuceneSPI = false; // requires synchronization
@@ -359,12 +137,10 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
    * Creates a loader.
    * Note: we do NOT call {@link #reloadLuceneSPI()}.
    */
-  public SolrResourceLoader(String name, List<Path> classpath, Path instanceDir, ClassLoader parent) {
+  public SolrResourceLoader(String name, List<Path> classpath, Path instanceDir, SolrResourceLoader parent) {
     this(instanceDir, parent);
     this.name = name;
 
-
-
     final List<URL> libUrls = new ArrayList<>(classpath.size());
     try {
       for (Path path : classpath) {
@@ -389,7 +165,7 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
    * @param instanceDir - base directory for this resource loader, if null locateSolrHome() will be used.
    * @see SolrPaths#locateSolrHome()
    */
-  public SolrResourceLoader(Path instanceDir, ClassLoader parent) {
+  public SolrResourceLoader(Path instanceDir, SolrResourceLoader parent) {
     if (instanceDir == null) {
       this.instanceDir = SolrPaths.locateSolrHome().toAbsolutePath().normalize();
       log.debug("new SolrResourceLoader for deduced Solr Home: '{}'", this.instanceDir);
@@ -398,11 +174,16 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
       log.debug("new SolrResourceLoader for directory: '{}'", this.instanceDir);
     }
 
+    ClassLoader cl;
     if (parent == null) {
-      parent = getClass().getClassLoader();
+      this.configXpathExpressions = new ConfigXpathExpressions();
+      cl = getClass().getClassLoader();
+    } else {
+      this.configXpathExpressions = parent.configXpathExpressions;
+      cl = parent.getClassLoader();
     }
-    this.classLoader = URLClassLoader.newInstance(EMPTY_URL_ARRAY, parent);
-    this.resourceClassLoader = URLClassLoader.newInstance(EMPTY_URL_ARRAY, parent);
+    this.classLoader = URLClassLoader.newInstance(EMPTY_URL_ARRAY, cl);
+    this.resourceClassLoader = URLClassLoader.newInstance(EMPTY_URL_ARRAY, cl);
     this.sysIdResolver = new SystemIdResolver(this);
     // TODO: workout the leak in CollectionsAPIAsyncDistrbutedZkTest
     // assert ObjectReleaseTracker.track(this);
@@ -442,11 +223,11 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
   }
 
   public Configuration getConf() {
-    return conf;
+    return configXpathExpressions.conf;
   }
 
   public XMLReader getXmlReader() {
-    return (XMLReader) parser.newSAXParser();
+    return (XMLReader) configXpathExpressions.parser.newSAXParser();
   }
 
   /**
@@ -1004,13 +785,7 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
 
   @Override
   public void close() throws IOException {
-    if (conf != null) {
-      try {
-        conf.close();
-      } catch (Exception e) {
-        log.info("Exception closing Configuration " + e.getClass().getName() + " " + e.getMessage());
-      }
-    }
+    org.apache.solr.common.util.IOUtils.closeQuietly(configXpathExpressions);
     IOUtils.close(classLoader);
     IOUtils.close(resourceClassLoader);
     assert ObjectReleaseTracker.release(this);
@@ -1036,7 +811,7 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
       try (OutputStream out = Files.newOutputStream(confFile.toPath(), StandardOpenOption.CREATE)) {
         out.write(content);
       }
-      log.info("Written confile {}", resourceName);
+      log.info("Written confile {} to {}", resourceName, confFile.toPath());
     } catch (IOException e) {
       final String msg = "Error persisting conf file " + resourceName;
       log.error(msg, e);
@@ -1052,6 +827,6 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
   }
 
   public XPath getXPath() {
-    return getXpathFactory().newXPath();
+    return configXpathExpressions.xpathFactory.newXPath();
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java b/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
index baa3a36..58c08867 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
@@ -24,35 +24,38 @@ import org.apache.solr.util.PropertiesUtil;
 import java.util.HashMap;
 import java.util.Properties;
 
-public class SolrTinyBuilder extends TinyBuilder  {
+public class SolrTinyBuilder extends TinyBuilder {
   private final Properties substituteProps;
-  private final HashMap sysProperties;
+  private final Properties sysProperties;
 
   /**
    * Create a TinyTree builder
    *
-   * @param pipe information about the pipeline leading up to this Builder
+   * @param pipe            information about the pipeline leading up to this Builder
    * @param substituteProps
    */
   public SolrTinyBuilder(PipelineConfiguration pipe, Properties substituteProps) {
     super(pipe);
     this.substituteProps = substituteProps;
-    this.sysProperties = new HashMap(System.getProperties());
+    this.sysProperties = System.getProperties();
   }
 
   protected int makeTextNode(CharSequence chars, int len) {
-    String sub = PropertiesUtil
-        .substituteProperty(chars.subSequence(0, len).toString(),
-            substituteProps, sysProperties);
+    String val = chars.subSequence(0, len).toString();
+    if (val.contains("${")) {
+      String sub = PropertiesUtil.substituteProperty(val, substituteProps, sysProperties);
+      return super.makeTextNode(sub, sub.length());
+    }
 
-    return super.makeTextNode(sub, sub.length());
+    return super.makeTextNode(chars, len);
   }
 
   protected String getAttValue(AttributeInfo att) {
-    String sub = PropertiesUtil
-        .substituteProperty(att.getValue(),
-            substituteProps, sysProperties);
-    return sub;
+    String attValue = att.getValue();
+    if (attValue.contains("${")) {
+      return PropertiesUtil.substituteProperty(attValue, substituteProps, sysProperties);
+    }
+    return attValue;
   }
 
 }
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 65d49bb..8ff2a4d 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -169,11 +169,11 @@ public class SolrXmlConfig {
   private void checkForIllegalConfig(XmlConfigFile config) {
     // was resource killer - note: perhaps not as bad now that xml is more efficient?
     SolrResourceLoader loader = config.getResourceLoader();
-    failIfFound(config, loader.coreLoadThreadsExp, loader.coreLoadThreadsPath);
-    failIfFound(config, loader.persistentExp, loader.persistentPath);
-    failIfFound(config, loader.sharedLibExp, loader.sharedLibPath);
-    failIfFound(config, loader.zkHostExp, loader.zkHostPath);
-    failIfFound(config, loader.coresExp, loader.coresPath);
+    failIfFound(config, loader.configXpathExpressions.coreLoadThreadsExp, loader.configXpathExpressions.coreLoadThreadsPath);
+    failIfFound(config, loader.configXpathExpressions.persistentExp, loader.configXpathExpressions.persistentPath);
+    failIfFound(config, loader.configXpathExpressions.sharedLibExp, loader.configXpathExpressions.sharedLibPath);
+    failIfFound(config, loader.configXpathExpressions.zkHostExp, loader.configXpathExpressions.zkHostPath);
+    failIfFound(config, loader.configXpathExpressions.coresExp, loader.configXpathExpressions.coresPath);
 
     assertSingleInstance("solrcloud", config);
     assertSingleInstance("logging", config);
@@ -469,7 +469,7 @@ public class SolrXmlConfig {
   }
 
   private PluginInfo getShardHandlerFactoryPluginInfo(XmlConfigFile config) {
-    NodeInfo node = config.getNode(config.getResourceLoader().shardHandlerFactoryExp, SolrResourceLoader.shardHandlerFactoryPath, false);
+    NodeInfo node = config.getNode(config.getResourceLoader().configXpathExpressions.shardHandlerFactoryExp, ConfigXpathExpressions.shardHandlerFactoryPath, false);
     return (node == null) ? null : new PluginInfo(node, "shardHandlerFactory", false, true);
   }
 
@@ -487,23 +487,23 @@ public class SolrXmlConfig {
   private MetricsConfig getMetricsConfig(XmlConfigFile config) throws XPathExpressionException {
     MetricsConfig.MetricsConfigBuilder builder = new MetricsConfig.MetricsConfigBuilder();
     SolrResourceLoader loader = config.getResourceLoader();
-    NodeInfo node = config.getNode(loader.counterExp, SolrResourceLoader.counterExpPath, false);
+    NodeInfo node = config.getNode(loader.configXpathExpressions.counterExp, ConfigXpathExpressions.counterExpPath, false);
     if (node != null) {
       builder = builder.setCounterSupplier(new PluginInfo(node, "counterSupplier", false, false));
     }
-    node = config.getNode(loader.meterExp, SolrResourceLoader.meterPath, false);
+    node = config.getNode(loader.configXpathExpressions.meterExp, ConfigXpathExpressions.meterPath, false);
     if (node != null) {
       builder = builder.setMeterSupplier(new PluginInfo(node, "meterSupplier", false, false));
     }
-    node = config.getNode(loader.timerExp, SolrResourceLoader.timerPath, false);
+    node = config.getNode(loader.configXpathExpressions.timerExp, ConfigXpathExpressions.timerPath, false);
     if (node != null) {
       builder = builder.setTimerSupplier(new PluginInfo(node, "timerSupplier", false, false));
     }
-    node = config.getNode(loader.histoExp, SolrResourceLoader.histoPath, false);
+    node = config.getNode(loader.configXpathExpressions.histoExp, ConfigXpathExpressions.histoPath, false);
     if (node != null) {
       builder = builder.setHistogramSupplier(new PluginInfo(node, "histogramSupplier", false, false));
     }
-    node = config.getNode(loader.historyExp, SolrResourceLoader.historyPath, false);
+    node = config.getNode(loader.configXpathExpressions.historyExp, ConfigXpathExpressions.historyPath, false);
     if (node != null) {
       builder = builder.setHistoryHandler(new PluginInfo(node, "history", false, false));
     }
@@ -516,7 +516,7 @@ public class SolrXmlConfig {
   }
 
   private PluginInfo[] getMetricReporterPluginInfos(XmlConfigFile config) throws XPathExpressionException {
-    ArrayList<NodeInfo> nodes = (ArrayList<NodeInfo>) config.loader.metricsReporterExp.evaluate(config.tree, XPathConstants.NODESET);
+    ArrayList<NodeInfo> nodes = (ArrayList<NodeInfo>) config.loader.configXpathExpressions.metricsReporterExp.evaluate(config.tree, XPathConstants.NODESET);
     List<PluginInfo> configs = new ArrayList<>();
     boolean hasJmxReporter = false;
     if (nodes != null && nodes.size() > 0) {
@@ -566,12 +566,12 @@ public class SolrXmlConfig {
   }
 
   private PluginInfo getTransientCoreCacheFactoryPluginInfo(XmlConfigFile config) {
-    NodeInfo node = config.getNode(config.getResourceLoader().transientCoreCacheFactoryExp, SolrResourceLoader.transientCoreCacheFactoryPath, false);
+    NodeInfo node = config.getNode(config.getResourceLoader().configXpathExpressions.transientCoreCacheFactoryExp, ConfigXpathExpressions.transientCoreCacheFactoryPath, false);
     return (node == null) ? null : new PluginInfo(node, "transientCoreCacheFactory", false, true);
   }
 
   private PluginInfo getTracerPluginInfo(XmlConfigFile config) {
-    NodeInfo node = config.getNode(config.getResourceLoader().tracerConfigExp, SolrResourceLoader.tracerConfigPath, false);
+    NodeInfo node = config.getNode(config.getResourceLoader().configXpathExpressions.tracerConfigExp, ConfigXpathExpressions.tracerConfigPath, false);
     return (node == null) ? null : new PluginInfo(node, "tracerConfig", false, true);
   }
 }
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 ce44a27..3f6036f 100644
--- a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
+++ b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
@@ -334,7 +334,7 @@ public class XmlConfigFile { // formerly simply "Config"
               name + " contains more than one value for config path: " + path);
         }
         NodeInfo nd = nodes.get(0);
-        log.trace("{}:{}={}", name, expression, nd);
+        if (log.isTraceEnabled()) log.trace("{}:{}={}", name, expression, nd);
         return nd;
 
       } catch (XPathExpressionException e) {
@@ -469,15 +469,15 @@ public class XmlConfigFile { // formerly simply "Config"
 
       String txt = DOMUtil.getText(nd);
 
-      log.debug("{} {}={}", name, expression, txt);
+      if (log.isDebugEnabled()) log.debug("{} {}={}", name, expression, txt);
       return txt;
     }
 
-    public String get (XPathExpression expression, String path){
+    public String get(XPathExpression expression, String path){
       return getVal(expression, path, true);
     }
 
-    public String get (XPathExpression expression,  String path, String def){
+    public String get(XPathExpression expression,  String path, String def){
       String val = getVal(expression, path, false);
       if (val == null || val.length() == 0) {
         return def;
@@ -491,6 +491,9 @@ public class XmlConfigFile { // formerly simply "Config"
 
     public int getInt (XPathExpression expression,  String path, int def){
       String val = getVal(expression, path, false);
+      if (val != null && val.equals("-1")) {
+        return def;
+      }
       return val != null ? Integer.parseInt(val) : def;
     }
 
@@ -500,6 +503,9 @@ public class XmlConfigFile { // formerly simply "Config"
 
     public boolean getBool (XPathExpression expression, String path, boolean def){
       String val = getVal(expression, path, false);
+      if (val != null && val.equals("-1")) {
+        return def;
+      }
       return val != null ? Boolean.parseBoolean(val) : def;
     }
 
@@ -509,6 +515,9 @@ public class XmlConfigFile { // formerly simply "Config"
 
     public float getFloat (XPathExpression expression, String path, float def){
       String val = getVal(expression, path, false);
+      if (val != null && val.equals("-1")) {
+        return def;
+      }
       return val != null ? Float.parseFloat(val) : def;
     }
 
@@ -518,6 +527,9 @@ public class XmlConfigFile { // formerly simply "Config"
 
     public double getDouble (XPathExpression expression, String path, double def){
       String val = getVal(expression, path, false);
+      if (val != null && val.equals("-1")) {
+        return def;
+      }
       return val != null ? Double.parseDouble(val) : def;
     }
 
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java b/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
index 42c48fa..0955369 100644
--- a/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
@@ -271,7 +271,7 @@ public class PackageLoader implements Closeable {
             "PACKAGE_LOADER: " + parent.name() + ":" + version,
             paths,
             Paths.get(coreContainer.getSolrHome()),
-            coreContainer.getResourceLoader().getClassLoader());
+            coreContainer.getResourceLoader());
       }
 
       public String getVersion() {
@@ -301,7 +301,7 @@ public class PackageLoader implements Closeable {
   }
   static class PackageResourceLoader extends SolrResourceLoader {
 
-    PackageResourceLoader(String name, List<Path> classpath, Path instanceDir, ClassLoader parent) {
+    PackageResourceLoader(String name, List<Path> classpath, Path instanceDir, SolrResourceLoader parent) {
       super(name, classpath, instanceDir, parent);
     }
 
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 0562094..403c3fc 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
@@ -82,22 +82,22 @@ public final class FieldTypePluginLoader
     FieldType ft = loader.newInstance(className, FieldType.class, "schema.");
     ft.setTypeName(name);
 
-    TinyElementImpl anode = (TinyElementImpl) loader.analyzerQueryExp.evaluate(node, XPathConstants.NODE);
+    TinyElementImpl anode = (TinyElementImpl) loader.configXpathExpressions.analyzerQueryExp.evaluate(node, XPathConstants.NODE);
     Analyzer queryAnalyzer = readAnalyzer(anode);
 
-    anode = (TinyElementImpl) loader.analyzerMultiTermExp.evaluate(node, XPathConstants.NODE);
+    anode = (TinyElementImpl) loader.configXpathExpressions.analyzerMultiTermExp.evaluate(node, XPathConstants.NODE);
     Analyzer multiAnalyzer = readAnalyzer(anode);
 
     // An analyzer without a type specified, or with type="index"
     Analyzer analyzer;
-    Object object = loader.analyzerIndexExp
+    Object object = loader.configXpathExpressions.analyzerIndexExp
         .evaluate(node, XPathConstants.NODE);
 
     anode = (TinyElementImpl) object;
     analyzer = readAnalyzer(anode);
 
     // a custom similarity[Factory]
-    object = loader.similarityExp.evaluate(node, XPathConstants.NODE);
+    object = loader.configXpathExpressions.similarityExp.evaluate(node, XPathConstants.NODE);
     SimilarityFactory simFactory;
     if (object instanceof TinyElementImpl) {
       anode = (TinyElementImpl) object;
@@ -204,11 +204,11 @@ public final class FieldTypePluginLoader
 
     // check for all of these up front, so we can error if used in
     // conjunction with an explicit analyzer class.
-    ArrayList<NodeInfo> charFilterNodes = (ArrayList) loader.charFilterExp.evaluate
+    ArrayList<NodeInfo> charFilterNodes = (ArrayList) loader.configXpathExpressions.charFilterExp.evaluate
       (node, XPathConstants.NODESET);
-    ArrayList<NodeInfo> tokenizerNodes = (ArrayList) loader.tokenizerExp.evaluate
+    ArrayList<NodeInfo> tokenizerNodes = (ArrayList) loader.configXpathExpressions.tokenizerExp.evaluate
       (node, XPathConstants.NODESET);
-    ArrayList<NodeInfo> tokenFilterNodes = (ArrayList) loader.filterExp.evaluate
+    ArrayList<NodeInfo> tokenFilterNodes = (ArrayList) loader.configXpathExpressions.filterExp.evaluate
       (node, XPathConstants.NODESET);
 
     if (analyzerName != null) {
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 0b9cea1..c008c70 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -41,6 +41,7 @@ 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.common.util.StopWatch;
+import org.apache.solr.core.ConfigXpathExpressions;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.XmlConfigFile;
@@ -553,7 +554,7 @@ public class IndexSchema {
       StopWatch timeParseSchemaDom = new StopWatch(SCHEMA + "-parseSchemaDom");
       NodeInfo  document = schemaConf.getTree();
     //  Document domDoc = (Document) DocumentOverNodeInfo.wrap(document);
-      TinyAttributeImpl nd = (TinyAttributeImpl) loader.schemaNameExp.evaluate(document, XPathConstants.NODE);
+      TinyAttributeImpl nd = (TinyAttributeImpl) loader.configXpathExpressions.schemaNameExp.evaluate(document, XPathConstants.NODE);
       StringBuilder sb = new StringBuilder(32);
       // Another case where the initialization from the test harness is different than the "real world"
       if (nd==null) {
@@ -569,10 +570,10 @@ public class IndexSchema {
       }
 
       //                      /schema/@version
-      String path = normalize(SolrResourceLoader.schemaVersionPath, schemaConf.getPrefix());
+      String path = normalize(ConfigXpathExpressions.schemaVersionPath, schemaConf.getPrefix());
       XPathExpression exp;
       if (path.equals("/schema/@version")) {
-        exp = loader.schemaVersionExp;
+        exp = loader.configXpathExpressions.schemaVersionExp;
       } else {
         throw new UnsupportedOperationException();
       }
@@ -584,7 +585,7 @@ public class IndexSchema {
       // load the Field Types
       final FieldTypePluginLoader typeLoader = new FieldTypePluginLoader(this, fieldTypes, schemaAware);
 
-      ArrayList<NodeInfo> nodes = (ArrayList) loader.fieldTypeXPathExpressionsExp.evaluate(document, XPathConstants.NODESET);
+      ArrayList<NodeInfo> nodes = (ArrayList) loader.configXpathExpressions.fieldTypeXPathExpressionsExp.evaluate(document, XPathConstants.NODESET);
       this.fieldTypes = fieldTypes;
 
       typeLoader.load(loader, nodes);
@@ -597,7 +598,7 @@ public class IndexSchema {
       timeLoadFields.done();
 
       StopWatch timeLoadSim = new StopWatch(SCHEMA + "-loadSim");
-      TinyElementImpl node = (TinyElementImpl) loader.schemaSimExp.evaluate(document, XPathConstants.NODE);
+      TinyElementImpl node = (TinyElementImpl) loader.configXpathExpressions.schemaSimExp.evaluate(document, XPathConstants.NODE);
       similarityFactory = readSimilarity(loader, node);
       if (similarityFactory == null) {
         final Class<?> simClass = SchemaSimilarityFactory.class;
@@ -623,20 +624,20 @@ public class IndexSchema {
       timeLoadSim.done();
       //                      /schema/defaultSearchField/text()
 
-      Object node2 = loader.defaultSearchFieldExp.evaluate(document, XPathConstants.NODE);
+      Object node2 = loader.configXpathExpressions.defaultSearchFieldExp.evaluate(document, XPathConstants.NODE);
       if (node2 != null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Setting defaultSearchField in schema not supported since Solr 7");
       }
 
       //                      /schema/solrQueryParser/@defaultOperator
 
-      node2 = loader.solrQueryParserDefaultOpExp.evaluate(document, XPathConstants.NODE);
+      node2 = loader.configXpathExpressions.solrQueryParserDefaultOpExp.evaluate(document, XPathConstants.NODE);
       if (node2 != null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Setting default operator in schema (solrQueryParser/@defaultOperator) not supported");
       }
 
       //                      /schema/uniqueKey/text()
-      TinyTextualElement.TinyTextualElementText tnode = (TinyTextualElement.TinyTextualElementText) loader.schemaUniqueKeyExp.evaluate(document, XPathConstants.NODE);
+      TinyTextualElement.TinyTextualElementText tnode = (TinyTextualElement.TinyTextualElementText) loader.configXpathExpressions.schemaUniqueKeyExp.evaluate(document, XPathConstants.NODE);
       if (tnode==null) {
         log.warn("no {} specified in schema.", UNIQUE_KEY);
       } else {
@@ -746,7 +747,7 @@ public class IndexSchema {
     //                  /schema/field | /schema/dynamicField | /schema/fields/field | /schema/fields/dynamicField
     ArrayList<DynamicField> dFields = new ArrayList<>();
     Map<String,SchemaField> fields = new HashMap<>();
-    ArrayList<NodeInfo> nodes = (ArrayList) loader.xpathOrExp.evaluate(document, XPathConstants.NODESET);
+    ArrayList<NodeInfo> nodes = (ArrayList) loader.configXpathExpressions.xpathOrExp.evaluate(document, XPathConstants.NODESET);
 
     for (int i=0; i<nodes.size(); i++) {
       NodeInfo node = nodes.get(i);
@@ -829,8 +830,7 @@ public class IndexSchema {
    * Loads the copy fields
    */
   protected void loadCopyFields(NodeInfo document) throws XPathExpressionException {
-    String expression = "//" + COPY_FIELD;
-    ArrayList<NodeInfo> nodes = (ArrayList) loader.copyFieldsExp.evaluate(document, XPathConstants.NODESET);
+    ArrayList<NodeInfo> nodes = (ArrayList) loader.configXpathExpressions.copyFieldsExp.evaluate(document, XPathConstants.NODESET);
 
     for (int i=0; i<nodes.size(); i++) {
       NodeInfo node = nodes.get(i);
diff --git a/solr/core/src/java/org/apache/solr/search/CacheConfig.java b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
index b82e589..82b3a25 100644
--- a/solr/core/src/java/org/apache/solr/search/CacheConfig.java
+++ b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
@@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathExpression;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -96,22 +96,16 @@ public class CacheConfig implements MapSerializable{
 
 
   @SuppressWarnings({"unchecked"})
-  public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
-    // nocomit look at precompile
-    NodeInfo node = null;
-    try {
-      String path = IndexSchema.normalize(xpath, "/config/");
-      node = solrConfig.getNode(solrConfig.getResourceLoader().getXPath().compile(path), path, false);
-    } catch (XPathExpressionException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
-    }
-    if(node == null || !"true".equals(DOMUtil.getAttrOrDefault(node, "enabled", "true"))) {
-      Map<String, String> m = solrConfig.getOverlay().getEditableSubProperties(xpath);
-      if(m==null) return null;
-      List<String> parts = StrUtils.splitSmart(xpath, '/');
-      return getConfig(solrConfig,parts.get(parts.size()-1) , Collections.EMPTY_MAP,xpath);
+  public static CacheConfig getConfig(SolrConfig solrConfig, String xpath, XPathExpression xpathExpression) {
+    String path = IndexSchema.normalize(xpath, "/config/");
+    NodeInfo node = solrConfig.getNode(xpathExpression, path, false);
+    if (node == null || !"true".equals(DOMUtil.getAttrOrDefault(node, "enabled", "true"))) {
+      Map<String,String> m = solrConfig.getOverlay().getEditableSubProperties(path);
+      if (m == null) return null;
+      List<String> parts = StrUtils.splitSmart(path, '/');
+      return getConfig(solrConfig, parts.get(parts.size() - 1), Collections.EMPTY_MAP, path);
     }
-    return getConfig(solrConfig, node.getDisplayName(),DOMUtil.toMap(node.attributes()), xpath);
+    return getConfig(solrConfig, node.getDisplayName(), DOMUtil.toMap(node.attributes()), path);
   }
 
 
diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java b/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
index ab94688..8b43449 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
@@ -33,6 +33,7 @@ import org.apache.lucene.search.Sort;
 import org.apache.lucene.util.InfoStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.ConfigXpathExpressions;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.common.MapSerializable;
 import org.apache.solr.core.PluginInfo;
@@ -49,9 +50,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.core.XmlConfigFile.assertWarnOrFail;
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathExpression;
-import javax.xml.xpath.XPathExpressionException;
 
 /**
  * This config object encapsulates IndexWriter config params,
@@ -60,25 +58,6 @@ import javax.xml.xpath.XPathExpressionException;
 public class SolrIndexConfig implements MapSerializable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private XPathExpression indexConfigExp;
-  private XPathExpression mergeSchedulerExp;
-  private XPathExpression mergePolicyExp;
-  private XPathExpression ramBufferSizeMBExp;
-  private XPathExpression checkIntegrityAtMergeExp;
-
-
-  static String indexConfigPath = "indexConfig";
-
-  static String mergeSchedulerPath = indexConfigPath + "/mergeScheduler";
-
-  static String mergePolicyPath = indexConfigPath + "/mergePolicy";
-
-
-  static String ramBufferSizeMBPath = indexConfigPath + "/ramBufferSizeMB";
-
-  static String checkIntegrityAtMergePath = indexConfigPath + "/checkIntegrityAtMerge";
-
-
   private static final String NO_SUB_PACKAGES[] = new String[0];
 
   private static final String DEFAULT_MERGE_POLICY_FACTORY_CLASSNAME = DefaultMergePolicyFactory.class.getName();
@@ -117,38 +96,6 @@ public class SolrIndexConfig implements MapSerializable {
     // enable coarse-grained metrics by default
     metricsInfo = new PluginInfo("metrics", Collections.emptyMap(), null, null);
 
-    initExpressions(solrConfig);
-  }
-
-  private void initExpressions(SolrConfig solrConfig) {
-    XPath xPath = solrConfig.getResourceLoader().getXPath();
-    try {
-
-      indexConfigExp = xPath.compile(indexConfigPath);
-    } catch (XPathExpressionException e) {
-      log.error("", e);
-    }
-
-    try {
-      mergeSchedulerExp = xPath.compile(mergeSchedulerPath);
-    } catch (XPathExpressionException e) {
-      log.error("", e);
-    }
-    try {
-      mergePolicyExp = xPath.compile(mergePolicyPath);
-    } catch (XPathExpressionException e) {
-      log.error("", e);
-    }
-    try {
-      ramBufferSizeMBExp = xPath.compile(ramBufferSizeMBPath);
-    } catch (XPathExpressionException e) {
-      log.error("", e);
-    }
-    try {
-      checkIntegrityAtMergeExp = xPath.compile(checkIntegrityAtMergePath);
-    } catch (XPathExpressionException e) {
-      log.error("", e);
-    }
   }
 
   /**
@@ -162,37 +109,36 @@ public class SolrIndexConfig implements MapSerializable {
       prefix = "indexConfig";
       log.debug("Defaulting to prefix '{}' for index configuration", prefix);
     }
-
-    initExpressions(solrConfig);
     
     if (def == null) {
       def = new SolrIndexConfig(solrConfig);
     }
 
+    SolrResourceLoader loader = solrConfig.getResourceLoader();
     // sanity check: this will throw an error for us if there is more then one
     // config section
-   // Object unused = solrConfig.getNode(indexConfigExp, indexConfigPath, false);
+    Object unused = solrConfig.getNode(loader.configXpathExpressions.indexConfigExp, ConfigXpathExpressions.indexConfigPath, false);
 
     // Assert that end-of-life parameters or syntax is not in our config.
     // Warn for luceneMatchVersion's before LUCENE_3_6, fail fast above
     assertWarnOrFail("The <mergeScheduler>myclass</mergeScheduler> syntax is no longer supported in solrconfig.xml. Please use syntax <mergeScheduler class=\"myclass\"/> instead.",
-        !((solrConfig.getNode(mergeSchedulerExp, mergeSchedulerPath, false) != null) && (solrConfig.get(prefix + "/mergeScheduler/@class", null) == null)),
+        !((solrConfig.getNode(loader.configXpathExpressions.mergeSchedulerExp, ConfigXpathExpressions.mergeSchedulerPath, false) != null) && (solrConfig.get(prefix + "/mergeScheduler/@class", null) == null)),
         true);
     assertWarnOrFail("Beginning with Solr 7.0, <mergePolicy>myclass</mergePolicy> is no longer supported, use <mergePolicyFactory> instead.",
-        !((solrConfig.getNode(mergePolicyExp,  mergePolicyPath, false) != null) && (solrConfig.get(prefix + "/mergePolicy/@class", null) == null)),
+        !((solrConfig.getNode(loader.configXpathExpressions.mergePolicyExp,  ConfigXpathExpressions.mergePolicyPath, false) != null) && (solrConfig.get(prefix + "/mergePolicy/@class", null) == null)),
         true);
     assertWarnOrFail("The <luceneAutoCommit>true|false</luceneAutoCommit> parameter is no longer valid in solrconfig.xml.",
         solrConfig.get(prefix + "/luceneAutoCommit", null) == null,
         true);
 
-    useCompoundFile = solrConfig.getBool(prefix+"/useCompoundFile", def.useCompoundFile);
-    maxBufferedDocs=solrConfig.getInt(prefix+"/maxBufferedDocs",def.maxBufferedDocs);
-    ramBufferSizeMB = solrConfig.getDouble(ramBufferSizeMBExp, ramBufferSizeMBPath, def.ramBufferSizeMB);
+    useCompoundFile = solrConfig.getBool(loader.configXpathExpressions.useCompoundFileExp, ConfigXpathExpressions.useCompoundFilePath, def.useCompoundFile);
+    maxBufferedDocs = solrConfig.getInt(loader.configXpathExpressions.maxBufferedDocsExp, ConfigXpathExpressions.maxBufferedDocsPath, def.maxBufferedDocs);
+    ramBufferSizeMB = solrConfig.getDouble(loader.configXpathExpressions.ramBufferSizeMBExp, ConfigXpathExpressions.ramBufferSizeMBPath, def.ramBufferSizeMB);
 
     // how do we validate the value??
-    ramPerThreadHardLimitMB = solrConfig.getInt(prefix+"/ramPerThreadHardLimitMB", def.ramPerThreadHardLimitMB);
+    ramPerThreadHardLimitMB = solrConfig.getInt(loader.configXpathExpressions.ramPerThreadHardLimitMBExp, ConfigXpathExpressions.ramPerThreadHardLimitMBPath, def.ramPerThreadHardLimitMB);
 
-    writeLockTimeout=solrConfig.getInt(prefix+"/writeLockTimeout", def.writeLockTimeout);
+    writeLockTimeout=solrConfig.getInt(loader.configXpathExpressions.writeLockTimeoutExp, ConfigXpathExpressions.writeLockTimeoutPath, def.writeLockTimeout);
     lockType=solrConfig.get(prefix+"/lockType", def.lockType);
 
     List<PluginInfo> infos = solrConfig.readPluginInfos(prefix + "/metrics", false, false);
@@ -208,10 +154,10 @@ public class SolrIndexConfig implements MapSerializable {
         getPluginInfo(prefix + "/mergePolicy", solrConfig, null) == null,
         true);
     assertWarnOrFail("Beginning with Solr 7.0, <maxMergeDocs> is no longer supported, configure it on the relevant <mergePolicyFactory> instead.",
-        solrConfig.getInt(prefix+"/maxMergeDocs", 0) == 0,
+        solrConfig.getInt(loader.configXpathExpressions.maxMergeDocsExp, ConfigXpathExpressions.maxMergeDocPath, 0) == 0,
         true);
     assertWarnOrFail("Beginning with Solr 7.0, <mergeFactor> is no longer supported, configure it on the relevant <mergePolicyFactory> instead.",
-        solrConfig.getInt(prefix+"/mergeFactor", 0) == 0,
+        solrConfig.getInt(loader.configXpathExpressions.mergeFactorExp, ConfigXpathExpressions.mergeFactorPath, 0) == 0,
         true);
 
     String val = solrConfig.get(prefix + "/termIndexInterval", null);
@@ -219,7 +165,7 @@ public class SolrIndexConfig implements MapSerializable {
       throw new IllegalArgumentException("Illegal parameter 'termIndexInterval'");
     }
 
-    boolean infoStreamEnabled = solrConfig.getBool(prefix + "/infoStream", false);
+    boolean infoStreamEnabled = solrConfig.getBool(loader.configXpathExpressions.infoStreamExp, ConfigXpathExpressions.infoStreamPath, false);
     if(infoStreamEnabled) {
       String infoStreamFile = solrConfig.get(prefix + "/infoStream/@file", null);
       if (infoStreamFile == null) {
@@ -232,7 +178,7 @@ public class SolrIndexConfig implements MapSerializable {
     mergedSegmentWarmerInfo = getPluginInfo(prefix + "/mergedSegmentWarmer", solrConfig, def.mergedSegmentWarmerInfo);
 
     assertWarnOrFail("Beginning with Solr 5.0, <checkIntegrityAtMerge> option is no longer supported and should be removed from solrconfig.xml (these integrity checks are now automatic)",
-        (null == solrConfig.getNode(checkIntegrityAtMergeExp, checkIntegrityAtMergePath, false)),
+        (null == solrConfig.getNode(loader.configXpathExpressions.checkIntegrityAtMergeExp, ConfigXpathExpressions.checkIntegrityAtMergePath, false)),
         true);
   }
 
diff --git a/solr/core/src/java/org/apache/solr/update/VersionInfo.java b/solr/core/src/java/org/apache/solr/update/VersionInfo.java
index 2b9f2ec..61a785d 100644
--- a/solr/core/src/java/org/apache/solr/update/VersionInfo.java
+++ b/solr/core/src/java/org/apache/solr/update/VersionInfo.java
@@ -30,6 +30,8 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.Terms;
 import org.apache.solr.common.ParWork;
+import org.apache.solr.core.ConfigXpathExpressions;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.legacy.LegacyNumericUtils;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -101,7 +103,8 @@ public class VersionInfo {
     this.ulog = ulog;
     IndexSchema schema = ulog.uhandler.core.getLatestSchema(); 
     versionField = getAndCheckVersionField(schema);
-    versionBucketLockTimeoutMs = ulog.uhandler.core.getSolrConfig().getInt("updateHandler/versionBucketLockTimeoutMs",
+    versionBucketLockTimeoutMs = ulog.uhandler.core.getSolrConfig().getInt(ulog.uhandler.core.getSolrConfig().getResourceLoader().configXpathExpressions
+            .mergeFactorExp, ConfigXpathExpressions.mergeFactorPath,
         Integer.parseInt(System.getProperty(SYS_PROP_BUCKET_VERSION_LOCK_TIMEOUT_MS, "0")));
     buckets = new VersionBucket[ BitUtil.nextHighestPowerOfTwo(nBuckets) ];
 //    for (int i=0; i<buckets.length; i++) {
diff --git a/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java b/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
index f6f1815..8010314 100644
--- a/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
+++ b/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
@@ -29,6 +29,22 @@ import java.util.Properties;
  * the DOM (they came from DomUtils) and it's really confusing to see them in something labeled DOM
  */
 public class PropertiesUtil {
+  public final static ThreadLocal<StringBuilder> THREAD_LOCAL_StringBuilder = new ThreadLocal<>(){
+    protected StringBuilder initialValue() {
+      return new StringBuilder();
+    }
+  };
+  public final static ThreadLocal<ArrayList> THREAD_LOCAL_fragments = new ThreadLocal<>(){
+    protected ArrayList initialValue() {
+      return new ArrayList();
+    }
+  };
+  public final static ThreadLocal<ArrayList> THREAD_LOCAL_propertyRefs = new ThreadLocal<>(){
+    protected ArrayList initialValue() {
+      return new ArrayList();
+    }
+  };
+
   /*
   * 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
@@ -38,11 +54,15 @@ public class PropertiesUtil {
       return value;
     }
 
-    List<String> fragments = new ArrayList<>();
-    List<String> propertyRefs = new ArrayList<>();
+    List<String> fragments = THREAD_LOCAL_fragments.get();
+    List<String> propertyRefs = THREAD_LOCAL_propertyRefs.get();
+    fragments.clear();
+    propertyRefs.clear();
     parsePropertyString(value, fragments, propertyRefs);
 
-    StringBuilder sb = new StringBuilder();
+    StringBuilder sb = THREAD_LOCAL_StringBuilder.get();
+    sb.setLength(0);
+
     Iterator<String> i = fragments.iterator();
     Iterator<String> j = propertyRefs.iterator();
 
@@ -71,7 +91,12 @@ public class PropertiesUtil {
       }
       sb.append(fragment);
     }
-    return sb.toString();
+    String returnString = sb.toString();
+
+    sb.setLength(0);
+    fragments.clear();
+    propertyRefs.clear();
+    return  returnString;
   }
 
   /*
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
index 95abbb0..f71d0a5 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
@@ -164,6 +164,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         " }";
     runConfigCommand(harness, "/config", payload);
 
+
     MapWriter m = getRespMap("/config/overlay", harness);
     MapWriter props =null;
     assertEquals("100", m._getStr("overlay/props/updateHandler/autoCommit/maxDocs", null));
@@ -177,7 +178,7 @@ public class TestSolrConfigHandler extends RestTestBase {
     m =  getRespMap("/config", harness);
     assertNotNull(m);
 
-    assertEquals("100", m._getStr("config/updateHandler/autoCommit/maxDocs",null));
+    assertEquals(m.toString(), "100", m._getStr("config/updateHandler/autoCommit/maxDocs",null));
     assertEquals("10", m._getStr("config/updateHandler/autoCommit/maxTime",null));
     assertEquals("true", m._getStr("config/requestDispatcher/requestParsers/addHttpRequestToContext",null));
     payload = "{\n" +
diff --git a/solr/core/src/test/org/apache/solr/util/TestSystemIdResolver.java b/solr/core/src/test/org/apache/solr/util/TestSystemIdResolver.java
index 6ce35ea..268a79d 100644
--- a/solr/core/src/test/org/apache/solr/util/TestSystemIdResolver.java
+++ b/solr/core/src/test/org/apache/solr/util/TestSystemIdResolver.java
@@ -46,7 +46,7 @@ public class TestSystemIdResolver extends SolrTestCaseJ4 {
   
   public void testResolving() throws Exception {
     final Path testHome = SolrTestUtil.getFile("solr/collection1").getParentFile().toPath();
-    final SolrResourceLoader loader = new SolrResourceLoader(testHome.resolve("collection1"), this.getClass().getClassLoader());
+    final SolrResourceLoader loader = new SolrResourceLoader(testHome.resolve("collection1"));
     final SystemIdResolver resolver = new SystemIdResolver(loader);
     final String fileUri = new File(testHome+"/crazy-path-to-config.xml").toURI().toASCIIString();
     
@@ -98,7 +98,7 @@ public class TestSystemIdResolver extends SolrTestCaseJ4 {
     System.setProperty("solr.allow.unsafe.resourceloading", "true");
     
     final Path testHome = SolrTestUtil.getFile("solr/collection1").getParentFile().toPath();
-    final SolrResourceLoader loader = new SolrResourceLoader(testHome.resolve("collection1"), this.getClass().getClassLoader());
+    final SolrResourceLoader loader = new SolrResourceLoader(testHome.resolve("collection1"));
     final SystemIdResolver resolver = new SystemIdResolver(loader);
     
     assertEntityResolving(resolver, SystemIdResolver.createSystemIdFromResourceName(testHome+"/crazy-path-to-schema.xml"),