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/02/26 06:58:05 UTC

[lucene-solr] 01/01: @1404 After trying to keep it around forever, goodbye xinclude - I guess for those that like to eat the slow, you could allow it to be enabled.

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 d42aaecc7e592e2ad55578f1c1899c3f45945292
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Fri Feb 26 00:57:41 2021 -0600

    @1404 After trying to keep it around forever, goodbye xinclude - I guess for those that like to eat the slow, you could allow it to be enabled.
    
    Took 3 hours 12 minutes
    
    Took 9 minutes
---
 .../conf/solrconfig-opennlp-extract.xml            |  29 ++-
 .../solr/collection1/conf/solrconfig-analytics.xml |  29 ++-
 .../org/apache/solr/core/ConfigSetService.java     |  18 +-
 .../java/org/apache/solr/core/CoreContainer.java   |  27 +--
 .../src/java/org/apache/solr/core/SolrConfig.java  |  90 ++++++++-
 .../org/apache/solr/core/SolrResourceLoader.java   | 224 +++++++++++----------
 .../java/org/apache/solr/core/SolrXmlConfig.java   |   2 +-
 .../java/org/apache/solr/core/XmlConfigFile.java   |   6 +-
 .../apache/solr/rest/ManagedResourceStorage.java   |   6 +-
 .../solr/rest/schema/FieldTypeXmlAdapter.java      |   2 +
 .../apache/solr/schema/FieldTypePluginLoader.java  |  70 +------
 .../java/org/apache/solr/schema/IndexSchema.java   |   6 +-
 .../org/apache/solr/search/QueryResultKey.java     |   8 +-
 .../org/apache/solr/search/SolrIndexSearcher.java  |  72 +------
 .../java/org/apache/solr/servlet/HttpSolrCall.java |   2 +-
 .../src/java/org/apache/solr/update/UpdateLog.java |  31 +--
 .../conf/bad-solrconfig-multiple-dirfactory.xml    |  29 ++-
 .../collection1/conf/bad-solrconfig-nrtmode.xml    |  29 ++-
 .../solr/collection1/conf/solrconfig-SOLR-749.xml  |  28 ++-
 .../collection1/conf/solrconfig-altdirectory.xml   |  29 ++-
 .../conf/solrconfig-analytics-query.xml            |  29 ++-
 .../solr/collection1/conf/solrconfig-basic.xml     |  29 ++-
 .../conf/solrconfig-cache-enable-disable.xml       |  29 ++-
 .../solr/collection1/conf/solrconfig-caching.xml   |  29 ++-
 .../collection1/conf/solrconfig-classification.xml |  29 ++-
 .../conf/solrconfig-collapseqparser.xml            |  29 ++-
 .../conf/solrconfig-components-name.xml            |  29 ++-
 .../conf/solrconfig-configurerecoverystrategy.xml  |  29 ++-
 .../conf/solrconfig-customrecoverystrategy.xml     |  29 ++-
 .../collection1/conf/solrconfig-deeppaging.xml     |  29 ++-
 .../conf/solrconfig-delaying-component.xml         |  29 ++-
 .../solrconfig-distrib-update-processor-chains.xml |  29 ++-
 .../solrconfig-doc-expire-update-processor.xml     |  29 ++-
 .../conf/solrconfig-externalversionconstraint.xml  |  29 ++-
 .../collection1/conf/solrconfig-functionquery.xml  |  31 ++-
 .../solr/collection1/conf/solrconfig-headers.xml   |  29 ++-
 .../solr/collection1/conf/solrconfig-highlight.xml |  29 ++-
 .../conf/solrconfig-implicitproperties.xml         |  29 ++-
 .../conf/solrconfig-infixsuggesters.xml            |  29 ++-
 .../conf/solrconfig-managed-schema-test.xml        |  29 ++-
 .../collection1/conf/solrconfig-managed-schema.xml |  29 ++-
 .../conf/solrconfig-master-throttled.xml           |  29 ++-
 .../solr/collection1/conf/solrconfig-master.xml    |  29 ++-
 .../conf/solrconfig-master1-keepOneBackup.xml      |  29 ++-
 .../solr/collection1/conf/solrconfig-master1.xml   |  29 ++-
 .../solr/collection1/conf/solrconfig-master2.xml   |  29 ++-
 .../solr/collection1/conf/solrconfig-master3.xml   |  28 ++-
 .../solr/collection1/conf/solrconfig-minhash.xml   |  28 ++-
 .../solr/collection1/conf/solrconfig-minimal.xml   |  28 ++-
 .../solr/collection1/conf/solrconfig-nocache.xml   |  28 ++-
 .../solr/collection1/conf/solrconfig-noopregen.xml |  28 ++-
 .../solr/collection1/conf/solrconfig-paramset.xml  |  28 ++-
 .../conf/solrconfig-phrases-identification.xml     |  28 ++-
 .../collection1/conf/solrconfig-phrasesuggest.xml  |  28 ++-
 .../collection1/conf/solrconfig-plugcollector.xml  |  28 ++-
 .../conf/solrconfig-postingshighlight.xml          |  28 ++-
 .../conf/solrconfig-query-parser-init.xml          |  28 ++-
 .../conf/solrconfig-querysender-noquery.xml        |  28 ++-
 .../collection1/conf/solrconfig-querysender.xml    |  28 ++-
 .../solr/collection1/conf/solrconfig-repeater.xml  |  28 ++-
 .../conf/solrconfig-response-log-component.xml     |  28 ++-
 .../collection1/conf/solrconfig-schemaless.xml     |  28 ++-
 .../conf/solrconfig-script-updateprocessor.xml     |  28 ++-
 .../conf/solrconfig-searcher-listeners1.xml        |  28 ++-
 .../solr/collection1/conf/solrconfig-slave.xml     |  28 ++-
 .../solr/collection1/conf/solrconfig-slave1.xml    |  28 ++-
 .../conf/solrconfig-solcoreproperties.xml          |  28 ++-
 .../conf/solrconfig-sortingresponse.xml            |  28 ++-
 .../solr/collection1/conf/solrconfig-spatial.xml   |  28 ++-
 .../collection1/conf/solrconfig-spellchecker.xml   |  28 ++-
 ...fig-suggestercomponent-context-filter-query.xml |  28 ++-
 .../conf/solrconfig-suggestercomponent.xml         |  28 ++-
 .../solr/collection1/conf/solrconfig-test-misc.xml |  29 ++-
 .../collection1/conf/solrconfig-testxmlparser.xml  |  29 ++-
 .../solr/collection1/conf/solrconfig-tlog.xml      |  29 ++-
 .../conf/solrconfig-tolerant-search.xml            |  29 ++-
 .../conf/solrconfig-tolerant-update-minimal.xml    |  28 ++-
 .../collection1/conf/solrconfig-transformers.xml   |  28 ++-
 .../conf/solrconfig-update-processor-chains.xml    |  28 ++-
 .../solr/collection1/conf/solrconfig-xinclude.xml  |  28 ++-
 .../solr/collection1/conf/solrconfig.xml           |  28 ++-
 .../conf/solrconfig_SimpleTextCodec.xml            |  28 ++-
 .../solr/collection1/conf/solrconfig_codec.xml     |  28 ++-
 .../solr/collection1/conf/solrconfig_codec2.xml    |  28 ++-
 .../solr/collection1/conf/solrconfig_perf.xml      |  28 ++-
 .../src/test-files/solr/crazy-path-to-config.xml   |  29 ++-
 .../src/test/org/apache/solr/CursorPagingTest.java |   8 +-
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |  44 ++--
 .../org/apache/solr/core/ResourceLoaderTest.java   |   1 -
 .../org/apache/solr/core/TestCodecSupport.java     |   2 +-
 .../handler/component/SuggestComponentTest.java    |   2 +-
 .../org/apache/solr/request/SimpleFacetsTest.java  |   4 +-
 .../test/org/apache/solr/search/TestRecovery.java  |   1 -
 .../src/java/org/apache/solr/JSONTestUtil.java     |   5 +-
 .../src/java/org/apache/solr/SolrTestCase.java     |   4 +-
 .../java/org/apache/solr/util/BaseTestHarness.java |  20 +-
 96 files changed, 2299 insertions(+), 410 deletions(-)

diff --git a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml
index 8d718f3..cf81c8c 100644
--- a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml
+++ b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml
@@ -19,7 +19,34 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
diff --git a/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig-analytics.xml b/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig-analytics.xml
index 4c359a6..5dd7ce8 100644
--- a/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig-analytics.xml
+++ b/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig-analytics.xml
@@ -20,7 +20,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
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 3766e72..30e92b4 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
@@ -34,6 +34,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StopWatch;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.IndexSchemaFactory;
+import org.apache.solr.schema.ManagedIndexSchemaFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -85,12 +86,7 @@ public abstract class ConfigSetService {
       }
       timeLoadConfigSetFlags.done();
 
-      boolean trusted =
-          (coreLoader instanceof ZkSolrResourceLoader
-              && flags != null
-              && flags.get("trusted") != null
-              && !flags.getBooleanArg("trusted")
-              ) ? false: true;
+      boolean trusted = !(coreLoader instanceof ZkSolrResourceLoader) || flags == null || flags.get("trusted") == null || flags.getBooleanArg("trusted");
 
       if (log.isDebugEnabled()) log.debug("Trusted configset={} {}", trusted, flags);
       StopWatch timeCreateSolrConfig = new StopWatch(dcore.getName() + "-createSolrConfig");
@@ -149,7 +145,15 @@ public abstract class ConfigSetService {
     //  we don't know for sure without examining what files exists in the configSet, and we don't
     //  want to pay the overhead of that at this juncture.  If we guess wrong, no schema sharing.
     //  The fix is usually to name your schema managed-schema instead of schema.xml.
-    IndexSchemaFactory indexSchemaFactory = IndexSchemaFactory.newIndexSchemaFactory(solrConfig);
+
+    PluginInfo info = solrConfig.getPluginInfo(IndexSchemaFactory.class.getName());
+    IndexSchemaFactory indexSchemaFactory;
+    if (null != info) {
+      indexSchemaFactory = solrConfig.getResourceLoader().newInstance(info.className, IndexSchemaFactory.class, "schema.");
+      indexSchemaFactory.init(info.initArgs);
+    } else {
+      indexSchemaFactory = solrConfig.getResourceLoader().newInstance(ManagedIndexSchemaFactory.class.getName(), IndexSchemaFactory.class);
+    }
 
     String configSet = cd.getConfigSet();
     if (configSet != null && schemaCache != null) {
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 408752a..298b21c 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -34,9 +34,7 @@ import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder;
 import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.cloud.CloudDescriptor;
-import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
-import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
@@ -45,7 +43,6 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
 import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.CloseTracker;
 import org.apache.solr.common.util.ExecutorUtil;
@@ -1518,27 +1515,25 @@ public class CoreContainer implements Closeable {
       try {
         if (core != null) {
           if (!registered) {
-            if (core != null) {
-
-              SolrCore finalCore1 = core;
-              try {
-                solrCoreExecutor.submit(() -> {
-                  finalCore1.closeAndWait();
-                });
-              } catch (RejectedExecutionException e) {
+            SolrCore finalCore1 = core;
+            try {
+              solrCoreExecutor.submit(() -> {
                 finalCore1.closeAndWait();
-              }
-              SolrCore finalOld = old;
+              });
+            } catch (RejectedExecutionException e) {
+              finalCore1.closeAndWait();
+            }
+            SolrCore finalOld = old;
+            if (finalOld != null) {
               try {
                 solrCoreExecutor.submit(() -> {
-                  if (finalOld != null) {
-                    finalOld.closeAndWait();
-                  }
+                  finalOld.closeAndWait();
                 });
               } catch (RejectedExecutionException e) {
                 finalOld.closeAndWait();
               }
             }
+
           }
           if (isShutDown) {
             SolrCore finalCore1 = core;
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 b22a5e5..fb52f37 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -54,6 +54,7 @@ import net.sf.saxon.om.NodeInfo;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.util.Version;
 import org.apache.solr.client.solrj.io.stream.expr.Expressible;
 import org.apache.solr.cloud.RecoveryStrategy;
@@ -71,10 +72,17 @@ import org.apache.solr.rest.RestManager;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.search.CacheConfig;
+import org.apache.solr.search.CacheRegenerator;
 import org.apache.solr.search.CaffeineCache;
+import org.apache.solr.search.DocList;
 import org.apache.solr.search.QParserPlugin;
+import org.apache.solr.search.QueryCommand;
+import org.apache.solr.search.QueryResult;
+import org.apache.solr.search.QueryResultKey;
 import org.apache.solr.search.SolrCache;
+import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.ValueSourceParser;
+import org.apache.solr.search.facet.UnInvertedField;
 import org.apache.solr.search.stats.StatsCache;
 import org.apache.solr.servlet.SolrRequestParsers;
 import org.apache.solr.spelling.QueryConverter;
@@ -255,7 +263,7 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     if (dataDir != null && dataDir.length() == 0) dataDir = null;
 
 
-    org.apache.solr.search.SolrIndexSearcher.initRegenerators(this);
+    initRegenerators(this);
 
     if (get("jmx", null) != null) {
       log.warn("solrconfig.xml: <jmx> is no longer supported, use solr.xml:/metrics/reporter section instead");
@@ -313,6 +321,23 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     log.debug("Loaded SolrConfig: {}", name);
   }
 
+  private void initRegenerators(SolrConfig solrConfig) {
+    {
+      if (solrConfig.fieldValueCacheConfig != null && solrConfig.fieldValueCacheConfig.getRegenerator() == null) {
+        solrConfig.fieldValueCacheConfig.setRegenerator(new SolrFieldCacheRegenerator());
+      }
+
+      if (solrConfig.filterCacheConfig != null && solrConfig.filterCacheConfig.getRegenerator() == null) {
+        solrConfig.filterCacheConfig.setRegenerator(new SolrFilterCacheRegenerator());
+      }
+
+      if (solrConfig.queryResultCacheConfig != null && solrConfig.queryResultCacheConfig.getRegenerator() == null) {
+        final int queryResultWindowSize = solrConfig.queryResultWindowSize;
+        solrConfig.queryResultCacheConfig.setRegenerator(new SolrQueryCacheRegenerator(queryResultWindowSize));
+      }
+    }
+  }
+
   private static final AtomicBoolean versionWarningAlreadyLogged = new AtomicBoolean(false);
 
   public static final Version parseLuceneVersionString(final String matchVersion) {
@@ -1012,4 +1037,67 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     return requestParams;
   }
 
+  private static class SolrQueryCacheRegenerator implements CacheRegenerator {
+    private final int queryResultWindowSize;
+
+    public SolrQueryCacheRegenerator(int queryResultWindowSize) {
+      this.queryResultWindowSize = queryResultWindowSize;
+    }
+
+    @Override
+    @SuppressWarnings({"rawtypes"})
+    public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache,
+        Object oldKey, Object oldVal) throws IOException {
+      QueryResultKey key = (QueryResultKey) oldKey;
+      int nDocs = 1;
+      // request 1 doc and let caching round up to the next window size...
+      // unless the window size is <=1, in which case we will pick
+      // the minimum of the number of documents requested last time and
+      // a reasonable number such as 40.
+      // TODO: make more configurable later...
+
+      if (queryResultWindowSize <= 1) {
+        DocList oldList = (DocList) oldVal;
+        int oldnDocs = oldList.offset() + oldList.size();
+        // 40 has factors of 2,4,5,10,20
+        nDocs = Math.min(oldnDocs, 40);
+      }
+
+      int flags = SolrIndexSearcher.NO_CHECK_QCACHE | key.nc_flags;
+      QueryCommand qc = new QueryCommand();
+      qc.setQuery(key.query)
+          .setFilterList(key.filters)
+          .setSort(key.sort)
+          .setLen(nDocs)
+          .setSupersetMaxDoc(nDocs)
+          .setFlags(flags);
+      QueryResult qr = new QueryResult();
+      newSearcher.getDocListC(qr, qc);
+      return true;
+    }
+  }
+
+  private static class SolrFieldCacheRegenerator implements CacheRegenerator {
+    @Override
+    public boolean regenerateItem(SolrIndexSearcher newSearcher,
+        @SuppressWarnings({"rawtypes"}) SolrCache newCache,
+        @SuppressWarnings({"rawtypes"})SolrCache oldCache,
+        Object oldKey, Object oldVal) throws IOException {
+      if (oldVal instanceof UnInvertedField) {
+        UnInvertedField.getUnInvertedField((String) oldKey, newSearcher);
+      }
+      return true;
+    }
+  }
+
+  private static class SolrFilterCacheRegenerator implements CacheRegenerator {
+    @Override
+    @SuppressWarnings({"rawtypes"})public boolean regenerateItem(SolrIndexSearcher newSearcher
+        , @SuppressWarnings({"rawtypes"}) SolrCache newCache
+        , @SuppressWarnings({"rawtypes"})SolrCache oldCache,
+        Object oldKey, Object oldVal) throws IOException {
+      newSearcher.cacheDocSet((Query) oldKey, null, false);
+      return true;
+    }
+  }
 }
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 e5b14d6..65ccda6 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
@@ -46,7 +46,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import net.sf.saxon.Configuration;
 import net.sf.saxon.xpath.XPathFactoryImpl;
@@ -70,7 +69,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.FieldTypePluginLoader;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.ManagedIndexSchemaFactory;
 import org.apache.solr.schema.SimilarityFactory;
@@ -80,9 +78,11 @@ import org.apache.solr.util.SystemIdResolver;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 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,10 +98,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 static XPathFactoryImpl xpathFactory;
+  private XPathFactoryImpl xpathFactory;
   private final SystemIdResolver sysIdResolver;
 
-  public static XPathFactoryImpl getXpathFactory() {
+  public XPathFactoryImpl getXpathFactory() {
     return xpathFactory;
   }
 
@@ -145,72 +145,74 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
   public static String schemaUniqueKeyPath = IndexSchema.stepsToPath(IndexSchema.SCHEMA, IndexSchema.UNIQUE_KEY, IndexSchema.TEXT_FUNCTION);
 
 
-  public static XPathExpression shardHandlerFactoryExp;
-  public static XPathExpression counterExp;
-  public static XPathExpression meterExp;
-  public static XPathExpression timerExp;
-  public static XPathExpression histoExp;
-  public static XPathExpression historyExp;
-  public static XPathExpression transientCoreCacheFactoryExp;
-  public static XPathExpression tracerConfigExp;
-
-  public static XPathExpression coreLoadThreadsExp;
-  public static XPathExpression persistentExp;
-  public static XPathExpression sharedLibExp;
-  public static XPathExpression zkHostExp;
-  public static XPathExpression coresExp;
-
-  public static XPathExpression xpathOrExp;
-  public static XPathExpression schemaNameExp;
-  public static XPathExpression schemaVersionExp;
-  public static XPathExpression schemaSimExp;
-  public static XPathExpression defaultSearchFieldExp;
-  public static XPathExpression solrQueryParserDefaultOpExp;
-  public static XPathExpression schemaUniqueKeyExp;
-  public static XPathExpression fieldTypeXPathExpressionsExp;
-  public static XPathExpression copyFieldsExp;
-
-  public static XPathExpression luceneMatchVersionExp;
-  public static XPathExpression indexDefaultsExp;
-  public static XPathExpression mainIndexExp;
-  public static XPathExpression nrtModeExp;
-  public static XPathExpression unlockOnStartupExp;
-
-  public static XPathExpression metricsReporterExp;
-
-  private final Configuration ourConf;
+  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();
+
 
   {
-    Configuration conf = Configuration.newConfiguration();
-    conf.setNamePool(this.conf.getNamePool());
-    conf.setDocumentNumberAllocator(this.conf.getDocumentNumberAllocator());
-    conf.setXIncludeAware(true);
+
+    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);
-    ourConf = conf;
-  }
 
-  static volatile Configuration conf;
-  static {
+    xpathFactory = new XPathFactoryImpl(conf);;
+
     refreshConf();
   }
 
-  public static void refreshConf() {
-    try {
-      if (conf != null) {
-        try {
-          conf.close();
-        } catch (Exception e) {
-          log.info("Exception closing Configuration " + e.getClass().getName() + " " + e.getMessage());
-        }
-      }
-      conf = Configuration.newConfiguration();
 
-      conf.setValidation(false);
-      conf.setXIncludeAware(true);
-      conf.setExpandAttributeDefaults(true);
 
-      xpathFactory = new XPathFactoryImpl(conf);
+  public void refreshConf() {
+    try {
 
       XPath xpath = xpathFactory.newXPath();
 
@@ -274,7 +276,44 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
 
       metricsReporterExp = xpath.compile(metricsReporterPath);
 
-      FieldTypePluginLoader.refreshConf();
+      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);
+      }
 
     } catch (Exception e) {
       log.error("", e);
@@ -403,7 +442,11 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
   }
 
   public Configuration getConf() {
-    return ourConf;
+    return conf;
+  }
+
+  public XMLReader getXmlReader() {
+    return (XMLReader) parser.newSAXParser();
   }
 
   /**
@@ -626,16 +669,6 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
     }
   }
 
-  /*
-   * A static map of short class name to fully qualified class name
-   */
-  private final Map<String, String> classNameCache = new ConcurrentHashMap<>(256, 0.75f, 24);
-
-  @VisibleForTesting
-   void clearCache() {
-    classNameCache.clear();
-  }
-
   // Using this pattern, legacy analysis components from previous Solr versions are identified and delegated to SPI loader:
   private static final Pattern legacyAnalysisPattern =
       Pattern.compile("((\\Q" + base + ".analysis.\\E)|(\\Qsolr.\\E))([\\p{L}_$][\\p{L}\\p{N}_$]+?)(TokenFilter|Filter|Tokenizer|CharFilter)Factory");
@@ -686,7 +719,7 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
     if (!cname.startsWith("solr.") && cname.contains(".")) {
       //this is the fully qualified class name
       try {
-        return Class.forName(cname, true, classLoader).asSubclass(expectedType);
+        return Class.forName(cname, false, classLoader).asSubclass(expectedType);
       } catch (ClassNotFoundException e) {
 
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, name +" Error loading class '" + cname + "'", e);
@@ -697,27 +730,13 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
     if (trans != null) {
       //A short name is provided
       try {
-        return Class.forName(trans, true, classLoader).asSubclass(expectedType);
+        return Class.forName(trans, false, classLoader).asSubclass(expectedType);
       } catch (ClassNotFoundException e) {
 
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, name +" Error loading class '" + cname + "'", e);
       }
     }
 
-    if (subpackages == null || subpackages.length == 0 || subpackages == packages) {
-      subpackages = packages;
-      String c = classNameCache.get(cname);
-      if (c != null) {
-        try {
-          return Class.forName(c, true, classLoader).asSubclass(expectedType);
-        } catch (ClassNotFoundException | ClassCastException e) {
-          // this can happen if the legacyAnalysisPattern below caches the wrong thing
-          log.warn("{} Unable to load cached class, attempting lookup. name={} shortname={} reason={}", name , c, cname, e);
-          classNameCache.remove(cname);
-        }
-      }
-    }
-
     String deprecation = DEPRECATIONS.get(cname);
     if (deprecation != null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not load class=" + cname +
@@ -751,22 +770,11 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
 //      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, name + " Error loading class '" + cname + "'" + " subpackages=" + Arrays.asList(subpackages));
 
     } finally {
-      if (clazz != null) {
-        //cache the shortname vs FQN if it is loaded by the webapp classloader  and it is loaded
-        // using a shortname
-        if (clazz.getClassLoader() == SolrResourceLoader.class.getClassLoader() &&
-            !cname.equals(clazz.getName()) &&
-            (subpackages.length == 0 || subpackages == packages)) {
-          //store in the cache
-          classNameCache.put(cname, clazz.getName());
-        }
-
         // print warning if class is deprecated
-        if (clazz.isAnnotationPresent(Deprecated.class)) {
-          log.warn("Solr loaded a deprecated plugin/analysis class [{}]. Please consult documentation how to replace it accordingly.",
-              cname);
-        }
-      }
+//        if (clazz.isAnnotationPresent(Deprecated.class)) {
+//          log.warn("Solr loaded a deprecated plugin/analysis class [{}]. Please consult documentation how to replace it accordingly.",
+//              cname);
+//        }
     }
   }
 
@@ -996,10 +1004,12 @@ public class SolrResourceLoader implements ResourceLoader, Closeable {
 
   @Override
   public void close() throws IOException {
-    try {
-      ourConf.close();
-    } catch (Exception e) {
-      log.info("Exception closing Configuration " + e.getClass().getName() + " " + e.getMessage());
+    if (conf != null) {
+      try {
+        conf.close();
+      } catch (Exception e) {
+        log.info("Exception closing Configuration " + e.getClass().getName() + " " + e.getMessage());
+      }
     }
     IOUtils.close(classLoader);
     IOUtils.close(resourceClassLoader);
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 42cc84d..65d49bb 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -516,7 +516,7 @@ public class SolrXmlConfig {
   }
 
   private PluginInfo[] getMetricReporterPluginInfos(XmlConfigFile config) throws XPathExpressionException {
-    ArrayList<NodeInfo> nodes = (ArrayList) SolrResourceLoader.metricsReporterExp.evaluate(config.tree, XPathConstants.NODESET);
+    ArrayList<NodeInfo> nodes = (ArrayList<NodeInfo>) config.loader.metricsReporterExp.evaluate(config.tree, XPathConstants.NODESET);
     List<PluginInfo> configs = new ArrayList<>();
     boolean hasJmxReporter = false;
     if (nodes != null && nodes.size() > 0) {
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 43b0aea..b51f860 100644
--- a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
+++ b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
@@ -130,7 +130,7 @@ public class XmlConfigFile { // formerly simply "Config"
 
     try {
       SAXSource source = new SAXSource(is);
-
+      source.setXMLReader(loader.getXmlReader());
       PipelineConfiguration plc = getResourceLoader().getConf().makePipelineConfiguration();
       //      if (is.getSystemId() != null) {
       //     plc.setURIResolver(loader.getSysIdResolver().asURIResolver());
@@ -159,8 +159,8 @@ public class XmlConfigFile { // formerly simply "Config"
         log.error("Exception handling xml doc", e);
         throw e;
       }  finally {
-        builder.close();
-        builder.reset();
+        //builder.close();
+        //builder.reset();
       }
 
       this.tree = docTree;
diff --git a/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java b/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java
index 4f5a17a..c7c6f8c 100644
--- a/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java
+++ b/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java
@@ -244,9 +244,9 @@ public abstract class ManagedResourceStorage {
       final String znodePath = getZnodeForResource(storedResourceId);
       byte[] znodeData = null;
       try {
-        if (zkClient.exists(znodePath)) {
-          znodeData = zkClient.getData(znodePath, null, null);
-        }
+        znodeData = zkClient.getData(znodePath, null, null);
+      } catch (KeeperException.NoNodeException e) {
+
       } catch (Exception e) {
         if (e instanceof IOException) {
           throw (IOException)e;
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
index e79049e..9e3a4e8 100644
--- a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
+++ b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
@@ -116,8 +116,10 @@ public class FieldTypeXmlAdapter {
     try {
 
       PipelineConfiguration plc = loader.getConf().makePipelineConfiguration();
+
       ParseOptions po = plc.getParseOptions();
       po.setEntityResolver(loader.getSysIdResolver());
+      po.setXMLReader(loader.getXmlReader());
       // Set via conf already
       // po.setXIncludeAware(true);
       //  po.setExpandAttributeDefaults(true);
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 860917d..0562094 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
@@ -37,7 +37,6 @@ import org.slf4j.LoggerFactory;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpression;
 import javax.xml.xpath.XPathExpressionException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
@@ -53,61 +52,6 @@ public final class FieldTypePluginLoader
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private static XPathExpression analyzerQueryExp;
-  private static XPathExpression analyzerMultiTermExp;
-
-  private static XPathExpression analyzerIndexExp;
-  private static XPathExpression similarityExp;
-  private static XPathExpression charFilterExp;
-  private static XPathExpression tokenizerExp;
-  private static XPathExpression filterExp;
-
-  static {
-    refreshConf();
-  }
-
-  public static void refreshConf() {
-    XPath xpath = SolrResourceLoader.getXpathFactory().newXPath();
-    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);
-    }
-  }
-
   /**
    * @param schema The schema that will be used to initialize the FieldTypes
    * @param fieldTypes All FieldTypes that are instantiated by
@@ -138,22 +82,22 @@ public final class FieldTypePluginLoader
     FieldType ft = loader.newInstance(className, FieldType.class, "schema.");
     ft.setTypeName(name);
 
-    TinyElementImpl anode = (TinyElementImpl) analyzerQueryExp.evaluate(node, XPathConstants.NODE);
+    TinyElementImpl anode = (TinyElementImpl) loader.analyzerQueryExp.evaluate(node, XPathConstants.NODE);
     Analyzer queryAnalyzer = readAnalyzer(anode);
 
-    anode = (TinyElementImpl)analyzerMultiTermExp.evaluate(node, XPathConstants.NODE);
+    anode = (TinyElementImpl) loader.analyzerMultiTermExp.evaluate(node, XPathConstants.NODE);
     Analyzer multiAnalyzer = readAnalyzer(anode);
 
     // An analyzer without a type specified, or with type="index"
     Analyzer analyzer;
-    Object object = analyzerIndexExp
+    Object object = loader.analyzerIndexExp
         .evaluate(node, XPathConstants.NODE);
 
     anode = (TinyElementImpl) object;
     analyzer = readAnalyzer(anode);
 
     // a custom similarity[Factory]
-    object = similarityExp.evaluate(node, XPathConstants.NODE);
+    object = loader.similarityExp.evaluate(node, XPathConstants.NODE);
     SimilarityFactory simFactory;
     if (object instanceof TinyElementImpl) {
       anode = (TinyElementImpl) object;
@@ -260,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)charFilterExp.evaluate
+    ArrayList<NodeInfo> charFilterNodes = (ArrayList) loader.charFilterExp.evaluate
       (node, XPathConstants.NODESET);
-    ArrayList<NodeInfo> tokenizerNodes = (ArrayList)tokenizerExp.evaluate
+    ArrayList<NodeInfo> tokenizerNodes = (ArrayList) loader.tokenizerExp.evaluate
       (node, XPathConstants.NODESET);
-    ArrayList<NodeInfo> tokenFilterNodes = (ArrayList)filterExp.evaluate
+    ArrayList<NodeInfo> tokenFilterNodes = (ArrayList) loader.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 8f1c071..0b9cea1 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -171,6 +171,10 @@ public class IndexSchema {
 
   private Map<FieldType, PayloadDecoder> decoders = new HashMap<>();  // cache to avoid scanning token filters repeatedly, unnecessarily
 
+  static {
+    DynamicReplacement.DynamicPattern.createPattern(""); // early init
+  }
+
   /**
    * keys are all fields copied to, count is num of copyField
    * directives that target them.
@@ -1101,7 +1105,7 @@ public class IndexSchema {
 
 
   public static abstract class DynamicReplacement implements Comparable<DynamicReplacement> {
-    abstract protected static class DynamicPattern {
+    abstract public static class DynamicPattern {
       protected final String regex;
       protected final String fixedStr;
 
diff --git a/solr/core/src/java/org/apache/solr/search/QueryResultKey.java b/solr/core/src/java/org/apache/solr/search/QueryResultKey.java
index eba36ae..3f2f992 100644
--- a/solr/core/src/java/org/apache/solr/search/QueryResultKey.java
+++ b/solr/core/src/java/org/apache/solr/search/QueryResultKey.java
@@ -32,11 +32,11 @@ public final class QueryResultKey implements Accountable {
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(QueryResultKey.class);
   private static final long BASE_SF_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(SortField.class);
 
-  final Query query;
-  final Sort sort;
+  public final Query query;
+  public final Sort sort;
   final SortField[] sfields;
-  final List<Query> filters;
-  final int nc_flags;  // non-comparable flags... ignored by hashCode and equals
+  public final List<Query> filters;
+  public final int nc_flags;  // non-comparable flags... ignored by hashCode and equals
   final int minExactCount;
 
   private final int hc;  // cached hashCode
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 7acb5df..695c4e3 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -535,76 +535,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     return filterCache;
   }
 
-  //
-  // Set default regenerators on filter and query caches if they don't have any
-  //
-  public static void initRegenerators(SolrConfig solrConfig) {
-    if (solrConfig.fieldValueCacheConfig != null && solrConfig.fieldValueCacheConfig.getRegenerator() == null) {
-      solrConfig.fieldValueCacheConfig.setRegenerator(new CacheRegenerator() {
-        @Override
-        public boolean regenerateItem(SolrIndexSearcher newSearcher,
-                                      @SuppressWarnings({"rawtypes"})SolrCache newCache,
-                                      @SuppressWarnings({"rawtypes"})SolrCache oldCache,
-            Object oldKey, Object oldVal) throws IOException {
-          if (oldVal instanceof UnInvertedField) {
-            UnInvertedField.getUnInvertedField((String) oldKey, newSearcher);
-          }
-          return true;
-        }
-      });
-    }
-
-    if (solrConfig.filterCacheConfig != null && solrConfig.filterCacheConfig.getRegenerator() == null) {
-      solrConfig.filterCacheConfig.setRegenerator(new CacheRegenerator() {
-        @Override
-        @SuppressWarnings({"rawtypes"})public boolean regenerateItem(SolrIndexSearcher newSearcher
-                , @SuppressWarnings({"rawtypes"})SolrCache newCache
-                , @SuppressWarnings({"rawtypes"})SolrCache oldCache,
-            Object oldKey, Object oldVal) throws IOException {
-          newSearcher.cacheDocSet((Query) oldKey, null, false);
-          return true;
-        }
-      });
-    }
-
-    if (solrConfig.queryResultCacheConfig != null && solrConfig.queryResultCacheConfig.getRegenerator() == null) {
-      final int queryResultWindowSize = solrConfig.queryResultWindowSize;
-      solrConfig.queryResultCacheConfig.setRegenerator(new CacheRegenerator() {
-        @Override
-        @SuppressWarnings({"rawtypes"})
-        public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache,
-            Object oldKey, Object oldVal) throws IOException {
-          QueryResultKey key = (QueryResultKey) oldKey;
-          int nDocs = 1;
-          // request 1 doc and let caching round up to the next window size...
-          // unless the window size is <=1, in which case we will pick
-          // the minimum of the number of documents requested last time and
-          // a reasonable number such as 40.
-          // TODO: make more configurable later...
-
-          if (queryResultWindowSize <= 1) {
-            DocList oldList = (DocList) oldVal;
-            int oldnDocs = oldList.offset() + oldList.size();
-            // 40 has factors of 2,4,5,10,20
-            nDocs = Math.min(oldnDocs, 40);
-          }
-
-          int flags = NO_CHECK_QCACHE | key.nc_flags;
-          QueryCommand qc = new QueryCommand();
-          qc.setQuery(key.query)
-              .setFilterList(key.filters)
-              .setSort(key.sort)
-              .setLen(nDocs)
-              .setSupersetMaxDoc(nDocs)
-              .setFlags(flags);
-          QueryResult qr = new QueryResult();
-          newSearcher.getDocListC(qr, qc);
-          return true;
-        }
-      });
-    }
-  }
-
   public QueryResult search(QueryResult qr, QueryCommand cmd) throws IOException {
     getDocListC(qr, cmd);
     return qr;
@@ -1305,7 +1235,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
    * getDocList version that uses+populates query and filter caches. In the event of a timeout, the cache is not
    * populated.
    */
-  private void getDocListC(QueryResult qr, QueryCommand cmd) throws IOException {
+  public void getDocListC(QueryResult qr, QueryCommand cmd) throws IOException {
     DocListAndSet out = new DocListAndSet();
     qr.setDocListAndSet(out);
     QueryResultKey key = null;
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index b0df051..8baf933 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -260,7 +260,7 @@ public class HttpSolrCall {
       core = cores.getCore(origCorename);
 
       if (core == null && (!cores.isZooKeeperAware() || QoSParams.INTERNAL.equals(req.getHeader(QoSParams.REQUEST_SOURCE))) && cores.isCoreLoading(origCorename)) {
-        cores.waitForLoadingCore(origCorename, 2500);
+        cores.waitForLoadingCore(origCorename, 10000);
         core = cores.getCore(origCorename);
       }
 
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 34213f9..dfb3679 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -1628,11 +1628,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
       for (List<Update> singleList : updateList) {
         for (Update ptr : singleList) {
           if(Math.abs(ptr.version) > Math.abs(maxVersion)) continue;
-          if (ptr.version != 0) {
-            ret.add(ptr.version);
-          } else {
-            log.warn("Found version of 0 {} {} {}", ptr.pointer, ptr.previousVersion, ptr.log);
-          }
+          ret.add(ptr.version);
           if (--n <= 0) return ret;
         }
       }
@@ -1775,7 +1771,6 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
   public RecentUpdates getRecentUpdates() {
     Deque<TransactionLog> logList;
     tlogLock.lock();
-    RecentUpdates recentUpdates;
     try {
       logList = new LinkedList<>(logs);
       for (TransactionLog log : logList) {
@@ -1794,14 +1789,14 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
         logList.addFirst(bufferTlog);
       }
 
-      recentUpdates = new RecentUpdates(logList, numRecordsToKeep);
+
     } finally {
       tlogLock.unlock();
     }
 
     // TODO: what if I hand out a list of updates, then do an update, then hand out another list (and
     // one of the updates I originally handed out fell off the list).  Over-request?
-    return recentUpdates;
+    return new RecentUpdates(logList, numRecordsToKeep);
   }
 
   public void bufferUpdates() {
@@ -2025,7 +2020,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
         int operationAndFlags = 0;
         long nextCount = 0;
 
-        AtomicReference<SolrException> exceptionOnExecuteUpdate = null;
+        AtomicReference<SolrException> exceptionOnExecuteUpdate = new AtomicReference<>();
         for (; ; ) {
           Object o = null;
           if (cancelApplyBufferUpdate) break;
@@ -2079,12 +2074,13 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
 
           if (o == null) break;
           // fail fast
-         // if (exceptionOnExecuteUpdate.get() != null) throw exceptionOnExecuteUpdate.get();
+          if (exceptionOnExecuteUpdate.get() != null) throw exceptionOnExecuteUpdate.get();
 
           try {
 
             // should currently be a List<Oper,Ver,Doc/Id>
-            @SuppressWarnings({"rawtypes"}) List entry = (List) o;
+            @SuppressWarnings({"rawtypes"})
+            List entry = (List) o;
             operationAndFlags = (Integer) entry.get(UpdateLog.FLAGS_IDX);
             int oper = operationAndFlags & OPERATION_MASK;
             long version = (Long) entry.get(UpdateLog.VERSION_IDX);
@@ -2157,7 +2153,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
         }
 
         waitForAllUpdatesGetExecuted(ogexecutor, pendingTasks);
-       // if (exceptionOnExecuteUpdate.get() != null) throw exceptionOnExecuteUpdate.get();
+        if (exceptionOnExecuteUpdate.get() != null) throw exceptionOnExecuteUpdate.get();
 
         CommitUpdateCommand cmd = new CommitUpdateCommand(req, false);
         cmd.setVersion(commitVersion);
@@ -2229,7 +2225,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
         executor.submit(getBucketHash(cmd), () -> {
           try {
             // fail fast
-         //   if (exceptionHolder.get() != null) return;
+            if (exceptionHolder.get() != null) return;
             if (cmd instanceof AddUpdateCommand) {
               proc.processAdd((AddUpdateCommand) cmd);
             } else {
@@ -2239,6 +2235,13 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
             recoveryInfo.errors++;
             loglog.warn("REPLAY_ERR: IOException reading log", e);
             // could be caused by an incomplete flush if recovering from log
+          } catch (SolrException e) {
+            if (e.code() == ErrorCode.SERVICE_UNAVAILABLE.code) {
+              exceptionHolder.compareAndSet(null, e);
+              return;
+            }
+            recoveryInfo.errors++;
+            loglog.warn("REPLAY_ERR: SolrException reading log", e);
           } catch (Exception e) {
             recoveryInfo.errors++;
             loglog.warn("REPLAY_ERR: Exception reading log", e);
@@ -2404,7 +2407,6 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
 
   public void seedBucketsWithHighestVersion(SolrIndexSearcher newSearcher) {
     if (debug) log.debug("Looking up max value of version field to seed version buckets");
-    if (versionInfo != null) {
       versionInfo.blockUpdates();
       try {
         maxVersionFromIndex = seedBucketsWithHighestVersion(newSearcher, versionInfo);
@@ -2412,6 +2414,5 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
         versionInfo.unblockUpdates();
       }
     }
-  }
 }
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-multiple-dirfactory.xml b/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-multiple-dirfactory.xml
index 89b9451..0c66841 100644
--- a/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-multiple-dirfactory.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-multiple-dirfactory.xml
@@ -21,7 +21,34 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <directoryFactory name="DirectoryFactory" class="solr.RAMDirectoryFactory"/>
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-nrtmode.xml b/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-nrtmode.xml
index 0e961d6..8e252b2 100644
--- a/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-nrtmode.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-nrtmode.xml
@@ -21,7 +21,34 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <directoryFactory name="DirectoryFactory" class="NRTCachingDirectoryFactory"/>
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml
index 9c2411b..28f17cb 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml
@@ -19,7 +19,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <queryParser name="foo" class="FooQParserPlugin"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml
index 0c62584..41e2ee2 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml
@@ -19,7 +19,34 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <directoryFactory name="DirectoryFactory" class="org.apache.solr.core.AlternateDirectoryTest$TestFSDirectoryFactory"></directoryFactory>
   <indexReaderFactory name="IndexReaderFactory" class="org.apache.solr.core.AlternateDirectoryTest$TestIndexReaderFactory"></indexReaderFactory >
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-analytics-query.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-analytics-query.xml
index 740a282..9e35483 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-analytics-query.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-analytics-query.xml
@@ -51,7 +51,34 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml
index d98ecac..6df59c8 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml
@@ -22,7 +22,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler" />
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml
index 29e1799..afabbc8 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml
@@ -20,7 +20,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler" />
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml
index f7d941c..18704b0 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml
@@ -16,7 +16,34 @@
   -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <query>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-classification.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-classification.xml
index 3370600..5567faa 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-classification.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-classification.xml
@@ -22,7 +22,34 @@
   -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="solrconfig.snippet.randomindexconfig.xml"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml
index 1233cd1..206ab01 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml
@@ -51,7 +51,34 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml
index 4c54a6e..1d8f16d 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml
@@ -21,7 +21,34 @@
 
   <jmx />
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <!-- Used to specify an alternate directory to hold all index data.
        It defaults to "index" if not present, and should probably
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-configurerecoverystrategy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-configurerecoverystrategy.xml
index d86a6de..f8361db 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-configurerecoverystrategy.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-configurerecoverystrategy.xml
@@ -19,7 +19,34 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <recoveryStrategy>
     <int name="maxRetries">250</int>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategy.xml
index 4e8effa..e11fb52 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategy.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategy.xml
@@ -19,7 +19,34 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <!--
     The RecoveryStrategy and RecoveryStrategy.Builder classes may change in future and customisations
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml
index 4f8c7e4..54630e3 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml
@@ -28,7 +28,34 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
     <updateLog enable="${enable.update.log:true}">
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delaying-component.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delaying-component.xml
index fcb0140..fdbbbd4 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delaying-component.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delaying-component.xml
@@ -20,7 +20,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
 
   <schemaFactory class="ClassicIndexSchemaFactory"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml
index dbb8031..60a40c7 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml
@@ -33,7 +33,34 @@
   
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+    <indexConfig>
+        <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+             the RandomMergePolicy in all tests - but some tests expect very specific
+             Merge behavior, so those tests can set it as needed.
+        -->
+        <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+        <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+        <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+        <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+        <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+        <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+        <writeLockTimeout>1000</writeLockTimeout>
+        <commitLockTimeout>10000</commitLockTimeout>
+
+        <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+             use the single process lockType for speed - but tests that explicitly need
+             to vary the lockType canset it as needed.
+        -->
+        <lockType>${solr.tests.lockType:single}</lockType>
+
+        <infoStream>${solr.tests.infostream:false}</infoStream>
+
+    </indexConfig>
+
 
   <!-- an update processor the explicitly excludes distrib to test
        clean errors when people attempt atomic updates w/o it
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-doc-expire-update-processor.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-doc-expire-update-processor.xml
index 0b61fa5..1a24f07 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-doc-expire-update-processor.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-doc-expire-update-processor.xml
@@ -31,7 +31,34 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
     <updateLog enable="${enable.update.log:true}">
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml
index e1a788d..34d3f5f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml
@@ -33,7 +33,34 @@
   
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
     <updateLog enable="${enable.update.log:true}">
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml
index ef0c39a..ec14458 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml
@@ -19,8 +19,35 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
- 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-headers.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-headers.xml
index 328fc9b..c06a0eb 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-headers.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-headers.xml
@@ -20,7 +20,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <searchComponent name="componentThatAddsHeader" class="org.apache.solr.servlet.ResponseHeaderTest$ComponentThatAddsHeader"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml
index c714a41..a18fe97 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml
@@ -20,7 +20,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-implicitproperties.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-implicitproperties.xml
index 5ef9d72..1b1dccd 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-implicitproperties.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-implicitproperties.xml
@@ -29,7 +29,34 @@
                     class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
-  <xi:include href="./solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <jmx/>
   <updateHandler class="solr.DirectUpdateHandler2">
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-infixsuggesters.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-infixsuggesters.xml
index a30186d..125a753 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-infixsuggesters.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-infixsuggesters.xml
@@ -17,7 +17,34 @@
 -->
 
 <config>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="solr.NRTCachingDirectoryFactory"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema-test.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema-test.xml
index 666132f..c2a45a5 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema-test.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema-test.xml
@@ -21,7 +21,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <requestHandler name="/select" class="solr.SearchHandler" />
 </config>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
index 46cfc9f..c23c46d 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
@@ -20,7 +20,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <schemaFactory class="ManagedIndexSchemaFactory">
     <bool name="mutable">${managed.schema.mutable:true}</bool>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master-throttled.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master-throttled.xml
index 1a37f3f..4bf3d44 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master-throttled.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master-throttled.xml
@@ -23,7 +23,34 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml
index e501af2..7899bf0 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml
@@ -23,7 +23,34 @@
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml
index bcd7874..fe77e3f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml
@@ -22,7 +22,34 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml
index 9271686..59b0cf8 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml
@@ -23,7 +23,34 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml
index 55301c2..db5c752 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml
@@ -22,7 +22,34 @@
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml
index 1c1dd40..2d30d59 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml
@@ -22,7 +22,33 @@
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-minhash.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-minhash.xml
index 05871ff..a2270c9 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-minhash.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-minhash.xml
@@ -59,7 +59,33 @@
 
   <statsCache class="${solr.statsCache:}"/>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+    <indexConfig>
+        <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+             the RandomMergePolicy in all tests - but some tests expect very specific
+             Merge behavior, so those tests can set it as needed.
+        -->
+        <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+        <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+        <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+        <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+        <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+        <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+        <writeLockTimeout>1000</writeLockTimeout>
+        <commitLockTimeout>10000</commitLockTimeout>
+
+        <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+             use the single process lockType for speed - but tests that explicitly need
+             to vary the lockType canset it as needed.
+        -->
+        <lockType>${solr.tests.lockType:single}</lockType>
+
+        <infoStream>${solr.tests.infostream:false}</infoStream>
+
+    </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-minimal.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-minimal.xml
index aedb683..21c40fd 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-minimal.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-minimal.xml
@@ -38,7 +38,33 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
     <commitWithin>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml
index 137c519..5cc8e01 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml
@@ -20,7 +20,33 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-noopregen.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-noopregen.xml
index e2a20a3..ada02e5 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-noopregen.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-noopregen.xml
@@ -21,7 +21,33 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler" />
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-paramset.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-paramset.xml
index a9e71f6..67d629d 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-paramset.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-paramset.xml
@@ -22,7 +22,33 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrases-identification.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrases-identification.xml
index 2830e2f..343c301 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrases-identification.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrases-identification.xml
@@ -19,7 +19,33 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
     <updateLog enable="${enable.update.log:true}">
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml
index 0f79d7c..9ddc507 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml
@@ -20,7 +20,33 @@
 <!-- solrconfig.xml for a WFST phrase suggester -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-plugcollector.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-plugcollector.xml
index abaa2c4..b2440c7 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-plugcollector.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-plugcollector.xml
@@ -50,7 +50,33 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml
index a578563..16d1c0e 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml
@@ -21,7 +21,33 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler">
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-query-parser-init.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-query-parser-init.xml
index eb538fa..2f39560 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-query-parser-init.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-query-parser-init.xml
@@ -22,7 +22,33 @@
   -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml
index 9d4f83d..6dc9b36 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml
@@ -19,7 +19,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
     <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender.xml
index 1404f8a..19c0550 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender.xml
@@ -19,7 +19,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
     <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml
index 761d45d..d51dc08 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml
@@ -23,7 +23,33 @@
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-response-log-component.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-response-log-component.xml
index 643d9a6..3f42dea 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-response-log-component.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-response-log-component.xml
@@ -19,7 +19,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <requestHandler name="/select"
      class="solr.SearchHandler"></requestHandler>
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
index 105176f..c951feb 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
@@ -25,7 +25,33 @@
     </updateLog>
   </updateHandler>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <!-- This is intentionally commented out, see SOLR-8131 -->
   <!--<schemaFactory class="ManagedIndexSchemaFactory">-->
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml
index 74f00fd..478a8db 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml
@@ -23,7 +23,33 @@
   -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-searcher-listeners1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-searcher-listeners1.xml
index c71f8ba..da13953 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-searcher-listeners1.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-searcher-listeners1.xml
@@ -36,7 +36,33 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   
   <queryResponseWriter name="xml" default="true"
                        class="solr.XMLResponseWriter" />
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml
index 39a7870..585d254 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml
@@ -23,7 +23,33 @@
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave1.xml
index 26821b4..a2a6e2d 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave1.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave1.xml
@@ -23,7 +23,33 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-solcoreproperties.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-solcoreproperties.xml
index fc707d4..c096da9 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-solcoreproperties.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-solcoreproperties.xml
@@ -24,7 +24,33 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <requestHandler name="/select" class="solr.SearchHandler">
     <lst name="defaults">
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingresponse.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingresponse.xml
index 1bb3c97..6bd8150 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingresponse.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-sortingresponse.xml
@@ -22,7 +22,33 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler" />
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spatial.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spatial.xml
index 889d1c2..d47387c 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spatial.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spatial.xml
@@ -21,7 +21,33 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler" />
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellchecker.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellchecker.xml
index a876fb6..b3b30e7 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellchecker.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellchecker.xml
@@ -17,7 +17,33 @@
 -->
 
 <config>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-suggestercomponent-context-filter-query.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-suggestercomponent-context-filter-query.xml
index 36b6755..7ce869f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-suggestercomponent-context-filter-query.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-suggestercomponent-context-filter-query.xml
@@ -16,7 +16,33 @@
  limitations under the License.
 -->
 <config>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
    <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-suggestercomponent.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-suggestercomponent.xml
index 857fb11..e9da644 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-suggestercomponent.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-suggestercomponent.xml
@@ -17,7 +17,33 @@
 -->
 
 <config>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
    <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-test-misc.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-test-misc.xml
index afd5954..68c094d 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-test-misc.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-test-misc.xml
@@ -24,7 +24,34 @@
 -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-testxmlparser.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-testxmlparser.xml
index 401710e..ebf9e05 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-testxmlparser.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-testxmlparser.xml
@@ -21,7 +21,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler" />
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
index 228080e..34c755f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
@@ -37,7 +37,34 @@
   
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+    <indexConfig>
+        <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+             the RandomMergePolicy in all tests - but some tests expect very specific
+             Merge behavior, so those tests can set it as needed.
+        -->
+        <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+        <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+        <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+        <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+        <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+        <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+        <writeLockTimeout>1000</writeLockTimeout>
+        <commitLockTimeout>10000</commitLockTimeout>
+
+        <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+             use the single process lockType for speed - but tests that explicitly need
+             to vary the lockType canset it as needed.
+        -->
+        <lockType>${solr.tests.lockType:single}</lockType>
+
+        <infoStream>${solr.tests.infostream:false}</infoStream>
+
+    </indexConfig>
+
 
   <!-- an update processor the explicitly excludes distrib to test
        clean errors when people attempt atomic updates w/o it
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-search.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-search.xml
index d9a81fb..130f328 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-search.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-search.xml
@@ -29,8 +29,33 @@
 
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml"
-    xmlns:xi="http://www.w3.org/2001/XInclude" />
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
     <commitWithin>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-update-minimal.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-update-minimal.xml
index d3b90db..4107625 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-update-minimal.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-update-minimal.xml
@@ -22,7 +22,33 @@
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   
   <schemaFactory class="ClassicIndexSchemaFactory"/>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <requestHandler name="/select" class="solr.SearchHandler">
     <lst name="defaults">
       <str name="echoParams">explicit</str>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-transformers.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-transformers.xml
index ef38e80..efea0b7 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-transformers.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-transformers.xml
@@ -19,7 +19,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+    <indexConfig>
+        <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+             the RandomMergePolicy in all tests - but some tests expect very specific
+             Merge behavior, so those tests can set it as needed.
+        -->
+        <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+        <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+        <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+        <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+        <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+        <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+        <writeLockTimeout>1000</writeLockTimeout>
+        <commitLockTimeout>10000</commitLockTimeout>
+
+        <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+             use the single process lockType for speed - but tests that explicitly need
+             to vary the lockType canset it as needed.
+        -->
+        <lockType>${solr.tests.lockType:single}</lockType>
+
+        <infoStream>${solr.tests.infostream:false}</infoStream>
+
+    </indexConfig>
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
index ef521a4..5da41d6 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
@@ -24,7 +24,33 @@
   -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-xinclude.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-xinclude.xml
index 17df214..c125319 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-xinclude.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-xinclude.xml
@@ -19,7 +19,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml
index 8536308..e7d8bc9 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml
@@ -59,7 +59,33 @@
 
   <statsCache class="${solr.statsCache:}"/>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+    <indexConfig>
+        <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+             the RandomMergePolicy in all tests - but some tests expect very specific
+             Merge behavior, so those tests can set it as needed.
+        -->
+        <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+        <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+        <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+        <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+        <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+        <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+        <writeLockTimeout>1000</writeLockTimeout>
+        <commitLockTimeout>10000</commitLockTimeout>
+
+        <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+             use the single process lockType for speed - but tests that explicitly need
+             to vary the lockType canset it as needed.
+        -->
+        <lockType>${solr.tests.lockType:single}</lockType>
+
+        <infoStream>${solr.tests.infostream:false}</infoStream>
+
+    </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_SimpleTextCodec.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_SimpleTextCodec.xml
index 7b0c3e3..478af06 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_SimpleTextCodec.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_SimpleTextCodec.xml
@@ -18,7 +18,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec.xml
index ad08096..30d2ee5 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec.xml
@@ -18,7 +18,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec2.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec2.xml
index c4a8ae7..6a6fef9 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec2.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_codec2.xml
@@ -18,7 +18,33 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
index 29fbd3d..075fa35 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
@@ -23,7 +23,33 @@
        If replication is in use, this should match the replication configuration. -->
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
 
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
diff --git a/solr/core/src/test-files/solr/crazy-path-to-config.xml b/solr/core/src/test-files/solr/crazy-path-to-config.xml
index 1c6f64f..7e55f1a 100644
--- a/solr/core/src/test-files/solr/crazy-path-to-config.xml
+++ b/solr/core/src/test-files/solr/crazy-path-to-config.xml
@@ -24,7 +24,34 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
-  <xi:include href="collection1/conf/solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <indexConfig>
+    <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+         the RandomMergePolicy in all tests - but some tests expect very specific
+         Merge behavior, so those tests can set it as needed.
+    -->
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    <ramPerThreadHardLimitMB>${solr.tests.ramPerThreadHardLimitMB}</ramPerThreadHardLimitMB>
+
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+    <writeLockTimeout>1000</writeLockTimeout>
+    <commitLockTimeout>10000</commitLockTimeout>
+
+    <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+         use the single process lockType for speed - but tests that explicitly need
+         to vary the lockType canset it as needed.
+    -->
+    <lockType>${solr.tests.lockType:single}</lockType>
+
+    <infoStream>${solr.tests.infostream:false}</infoStream>
+
+  </indexConfig>
+
 
   <updateHandler class="solr.DirectUpdateHandler2"/>
 
diff --git a/solr/core/src/test/org/apache/solr/CursorPagingTest.java b/solr/core/src/test/org/apache/solr/CursorPagingTest.java
index 86c6c27..ed799f3 100644
--- a/solr/core/src/test/org/apache/solr/CursorPagingTest.java
+++ b/solr/core/src/test/org/apache/solr/CursorPagingTest.java
@@ -32,6 +32,7 @@ import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.CursorMark;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
 import static org.apache.solr.common.params.CursorMarkParams.CURSOR_MARK_NEXT;
@@ -62,7 +63,7 @@ public class CursorPagingTest extends SolrTestCaseJ4 {
       { "Not Available", "Low", "Medium", "High", "Critical" };
 
   @BeforeClass
-  public static void beforeTests() throws Exception {
+  public static void beforeCursorPagingTest() throws Exception {
     // we need DVs on point fields to compute stats & facetsew
     System.setProperty(NUMERIC_POINTS_SYSPROP, "true");
     randomizeNumericTypesProperties();
@@ -72,6 +73,11 @@ public class CursorPagingTest extends SolrTestCaseJ4 {
     initCore(TEST_SOLRCONFIG_NAME, TEST_SCHEMAXML_NAME);
   }
 
+  @AfterClass
+  public static void afterCursorPagingTest() throws Exception {
+    deleteCore();
+  }
+
   @After
   public void cleanup() throws Exception {
     assertU(delQ("*:*"));
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 9e8acb9..9847f1c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -86,21 +86,23 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
 
   @BeforeClass
   public static void beforeCollectionsAPISolrJTest() throws Exception {
-    System.setProperty("solr.zkclienttimeout", "4000");
-    System.setProperty("zkClientTimeout", "4000");
-
-    System.setProperty("solr.http2solrclient.default.idletimeout", "60000");
-    System.setProperty("distribUpdateSoTimeout", "60000");
-    System.setProperty("socketTimeout", "60000");
-    System.setProperty("connTimeout", "60000");
-    System.setProperty("solr.test.socketTimeout.default", "60000");
-    System.setProperty("solr.connect_timeout.default", "60000");
-    System.setProperty("solr.so_commit_timeout.default", "60000");
-    System.setProperty("solr.httpclient.defaultConnectTimeout", "60000");
-    System.setProperty("solr.httpclient.defaultSoTimeout", "60000");
-    System.setProperty("solr.default.collection_op_timeout", "60000");
-
-    System.setProperty("solr.createCollectionTimeout", "60000");
+    System.setProperty("solr.zkclienttimeout", "15000");
+    System.setProperty("zkClientTimeout", "15000");
+
+
+    String timeout = "640000";
+    System.setProperty("solr.http2solrclient.default.idletimeout", timeout);
+    System.setProperty("distribUpdateSoTimeout", timeout);
+    System.setProperty("socketTimeout", timeout);
+    System.setProperty("connTimeout", timeout);
+    System.setProperty("solr.test.socketTimeout.default", timeout);
+    System.setProperty("solr.connect_timeout.default", timeout);
+    System.setProperty("solr.so_commit_timeout.default", timeout);
+    System.setProperty("solr.httpclient.defaultConnectTimeout", timeout);
+    System.setProperty("solr.httpclient.defaultSoTimeout", timeout);
+    System.setProperty("solr.default.collection_op_timeout", timeout);
+    System.setProperty("solr.enableMetrics", "false");
+    System.setProperty("solr.createCollectionTimeout", timeout);
 
     System.setProperty("solr.suppressDefaultConfigBootstrap", "false");
     configureCluster( TEST_NIGHTLY ? 4 : 2).formatZk(true)
@@ -242,7 +244,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
   @Ignore // MRM TODO: - testing large numbers
   public void testCreateAndDeleteCollection() throws Exception {
     String collectionName = "solrj_test";
-    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 36, 18)
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 12, 12)
             .process(cluster.getSolrClient());
 
 
@@ -257,11 +259,11 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
 //      assertTrue(status.get("QTime") > 0);
 //    }
 
-    response = CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
-
-    assertEquals(0, response.getStatus());
-
-    assertFalse(zkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName));
+//    response = CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+//
+//    assertEquals(0, response.getStatus());
+//
+//    assertFalse(zkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName));
     // MRM TODO: what happened to success?
 //    assertTrue(response.toString(), response.isSuccess());
 //    Map<String,NamedList<Integer>> nodesStatus = response.getCollectionNodesStatus();
diff --git a/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java b/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java
index f65f8c3..dacd2d5 100644
--- a/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java
+++ b/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java
@@ -214,7 +214,6 @@ public class ResourceLoaderTest extends SolrTestCaseJ4 {
 
   public void testCacheWrongType() throws Exception {
     SolrResourceLoader loader = new SolrResourceLoader();
-    loader.clearCache();
     Class[] params = { Map.class };
     Map<String,String> args = Map.of("minGramSize", "1", "maxGramSize", "2");
     final String className = "solr.NGramTokenizerFactory";
diff --git a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
index 37e83e0..8167d6f 100644
--- a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
+++ b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
@@ -231,7 +231,7 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
     SolrConfig config = TestHarness.createConfig(testSolrHome, previousCoreName, "solrconfig_codec2.xml", loader);
     assertEquals("Unexpected codec factory for this test.", "solr.SchemaCodecFactory", config.get("codecFactory/@class"));
     String path = IndexSchema.normalize("codecFactory", config.getPrefix());
-    LuceneTestCase.expectThrows(NoSuchElementException.class, () -> config.getNode(h.getXpath().compile(path), path, false).children().iterator().next());
+    LuceneTestCase.expectThrows(SolrException.class, () -> config.getNode(h.getXpath().compile(path), path, false).children().iterator().next());
     IndexSchema schema = IndexSchemaFactory.buildIndexSchema("schema_codec.xml", config);
 
     CoreContainer coreContainer = h.getCoreContainer();
diff --git a/solr/core/src/test/org/apache/solr/handler/component/SuggestComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/SuggestComponentTest.java
index 2899624..304094e 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/SuggestComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/SuggestComponentTest.java
@@ -44,7 +44,7 @@ public class SuggestComponentTest extends SolrTestCaseJ4 {
   }
 
   @AfterClass
-  public void afterSuggestComponentTest() {
+  public static void afterSuggestComponentTest() {
     deleteCore();
   }
 
diff --git a/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java b/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
index 14d6947..35a7616 100644
--- a/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
+++ b/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
@@ -57,6 +57,7 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeSimpleFacetsTest() throws Exception {
+    pendingDocs = new ArrayList<>();
     // we need DVs on point fields to compute stats & facets
     if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
     initCore("solrconfig.xml","schema.xml");
@@ -67,6 +68,7 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
   public static void afterSimpleFacetsTest() throws Exception {
     deleteCore();
     pendingDocs.clear();
+    pendingDocs = null;
   }
 
   static int random_commit_percent = 30;
@@ -77,7 +79,7 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
       assertU(commit());
   }
 
-  static ArrayList<String[]> pendingDocs = new ArrayList<>();
+  static ArrayList<String[]> pendingDocs;
 
   // committing randomly gives different looking segments each time
   static void add_doc(String... fieldsAndValues) {
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index 39b1b1b..0077011 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -55,7 +55,6 @@ import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/solr/test-framework/src/java/org/apache/solr/JSONTestUtil.java b/solr/test-framework/src/java/org/apache/solr/JSONTestUtil.java
index 7ff3224..7304e55 100644
--- a/solr/test-framework/src/java/org/apache/solr/JSONTestUtil.java
+++ b/solr/test-framework/src/java/org/apache/solr/JSONTestUtil.java
@@ -80,6 +80,7 @@ public class JSONTestUtil {
     String path = pos>=0 ? pathAndExpected.substring(0,pos) : null;
     String expected = pos>=0 ? pathAndExpected.substring(pos+2) : pathAndExpected;
     Object expectObj = failRepeatedKeys ? new NoDupsObjectBuilder(new JSONParser(expected)).getVal() : ObjectBuilder.fromJSON(expected);
+    assert path != null;
     return matchObj(path, input, expectObj, delta);
   }
 
@@ -101,9 +102,9 @@ public class JSONTestUtil {
     }
 
     @Override
-    public void addKeyVal(Object map, Object key, Object val) throws IOException {
+    public void addKeyVal(Object map, Object key, Object val) {
       Object prev = ((Map<Object, Object>) map).put(key, val);
-      if (prev != null && !key.equals("requestMethod")) { // TODO: how does this get duped? see TestJsonFacets
+      if (prev != null) {
         throw new RuntimeException("REPEATED JSON OBJECT KEY: key=" + key + " prevValue=" + prev + " thisValue" + val);
       }
     }
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCase.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCase.java
index 17341f2..b640e52 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCase.java
@@ -53,7 +53,6 @@ import org.apache.solr.common.util.CloseTracker;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.SolrQueuedThreadPool;
 import org.apache.solr.common.util.SysStats;
-import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.security.PublicKeyHandler;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.util.CryptoKeys;
@@ -577,7 +576,6 @@ public class SolrTestCase extends Assert {
       ParWork.shutdownParWorkExecutor();
 
     } finally {
-      SolrResourceLoader.refreshConf();
       ObjectReleaseTracker.clear();
       TestInjection.reset();
     }
@@ -585,7 +583,7 @@ public class SolrTestCase extends Assert {
       HttpClientUtil.resetHttpClientBuilder();
       Http2SolrClient.resetSslContextFactory();
       TestInjection.reset();
-
+      JSONTestUtil.failRepeatedKeys = false;
       random = null;
       reusedKeys = null;
       sslConfig = null;
diff --git a/solr/test-framework/src/java/org/apache/solr/util/BaseTestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/BaseTestHarness.java
index e2c34a4..853e87b 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/BaseTestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/BaseTestHarness.java
@@ -19,7 +19,6 @@ package org.apache.solr.util;
 import net.sf.saxon.event.PipelineConfiguration;
 import net.sf.saxon.event.Sender;
 import net.sf.saxon.lib.ParseOptions;
-import net.sf.saxon.lib.Validation;
 import net.sf.saxon.trans.XPathException;
 import net.sf.saxon.tree.tiny.TinyDocumentImpl;
 import org.apache.solr.common.SolrException;
@@ -90,28 +89,27 @@ abstract public class BaseTestHarness {
     try {
       builder.open();
       SAXSource source = new SAXSource(new InputSource(new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8))));
+      source.setXMLReader(resourceLoader.getXmlReader());
       //  source.getInputSource().setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
       ParseOptions po = plc.getParseOptions();
       if (resourceLoader != null) {
         po.setEntityResolver(resourceLoader.getSysIdResolver());
       }
       // Set via conf already
-      // po.setXIncludeAware(true);
-      // po.setCheckEntityReferences(false);
-      // po.setExpandAttributeDefaults(false);
-      po.setDTDValidationMode(Validation.STRIP);
+      po.setCheckEntityReferences(false);
+      po.setExpandAttributeDefaults(false);
       po.setPleaseCloseAfterUse(true);
       Sender.send(source, builder, po);
       docTree = (TinyDocumentImpl) builder.getCurrentRoot();
     } catch (XPathException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     } finally {
-      try {
-        builder.close();
-        builder.reset();
-      } catch (XPathException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-      }
+//      try {
+//        builder.close();
+//        builder.reset();
+//      } catch (XPathException e) {
+//        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+//      }
     }
     return docTree;
   }