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/01 23:52:47 UTC

[lucene-solr] 02/04: @1297 Finish test using core ref counting correctly.

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 6a54ff4c754592e7adb4dd36190f8127f962bda5
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Mon Feb 1 11:08:27 2021 -0600

    @1297 Finish test using core ref counting correctly.
---
 .../function/field/AbstractAnalyticsFieldTest.java |  10 +-
 .../value/CastingBooleanValueStreamTest.java       |   2 +
 .../AbstractDataImportHandlerTestCase.java         |   3 +-
 .../src/java/org/apache/solr/api/AnnotatedApi.java |   4 +-
 .../java/org/apache/solr/cloud/StatePublisher.java |  24 +-
 .../java/org/apache/solr/cloud/ZkController.java   |  21 +-
 .../apache/solr/cloud/api/collections/Assign.java  |   8 +-
 .../java/org/apache/solr/core/CoreContainer.java   |   7 +-
 .../java/org/apache/solr/core/CoreDescriptor.java  |   5 +-
 .../src/java/org/apache/solr/core/PluginBag.java   |  10 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |  55 ++--
 .../src/java/org/apache/solr/core/SolrCores.java   |   6 +-
 .../java/org/apache/solr/core/SolrTinyBuilder.java |   7 +-
 .../java/org/apache/solr/core/SolrXmlConfig.java   |   3 +-
 .../solr/handler/admin/CoreAdminOperation.java     |   3 +-
 .../apache/solr/metrics/SolrCoreMetricManager.java |   2 +-
 .../java/org/apache/solr/schema/IndexSchema.java   |   2 +-
 .../apache/solr/servlet/SolrDispatchFilter.java    |   8 +-
 .../java/org/apache/solr/util/PropertiesUtil.java  |   8 +-
 .../test/org/apache/solr/MinimalSchemaTest.java    |   9 +-
 .../src/test/org/apache/solr/OutputWriterTest.java |  20 +-
 .../src/test/org/apache/solr/SolrInfoBeanTest.java | 112 ++++----
 solr/core/src/test/org/apache/solr/TestJoin.java   |   4 +-
 solr/core/src/test/org/apache/solr/TestTrie.java   |  11 +-
 .../apache/solr/cloud/TestCloudDeleteByQuery.java  |   4 +
 .../apache/solr/core/AlternateDirectoryTest.java   |  10 +-
 .../solr/core/ByteBuffersDirectoryFactoryTest.java |  14 +-
 .../solr/core/ConfigureRecoveryStrategyTest.java   |  33 ++-
 .../org/apache/solr/core/RequestHandlersTest.java  | 116 ++++-----
 .../src/test/org/apache/solr/core/SOLR749Test.java |   1 +
 .../test/org/apache/solr/core/SolrCoreTest.java    |   3 +
 .../org/apache/solr/core/TestCodecSupport.java     |  92 +++----
 .../org/apache/solr/core/TestCoreContainer.java    | 281 +++++++++++----------
 .../apache/solr/core/TestInfoStreamLogging.java    |   6 +-
 .../test/org/apache/solr/core/TestInitParams.java  | 132 +++++-----
 .../org/apache/solr/core/TestJmxIntegration.java   | 103 ++++----
 .../apache/solr/core/TestMergePolicyConfig.java    | 167 ++++++------
 .../apache/solr/core/TestQuerySenderListener.java  |  70 +++--
 .../apache/solr/core/TestQuerySenderNoQuery.java   |  61 +++--
 .../org/apache/solr/core/TestSimpleTextCodec.java  |  52 ++--
 .../apache/solr/core/TestSolrDeletionPolicy1.java  |  63 +++--
 .../apache/solr/core/TestSolrDeletionPolicy2.java  |  33 +--
 .../org/apache/solr/core/TestSolrIndexConfig.java  |  25 +-
 .../DocumentAnalysisRequestHandlerTest.java        |  19 +-
 .../org/apache/solr/handler/TestCoreBackup.java    |  21 +-
 .../admin/SegmentsInfoRequestHandlerTest.java      | 117 ++++-----
 .../PhrasesIdentificationComponentTest.java        |  29 ++-
 .../component/QueryElevationComponentTest.java     | 168 ++++--------
 .../solr/handler/export/TestExportWriter.java      |   1 +
 .../org/apache/solr/highlight/HighlighterTest.java |  19 +-
 .../highlight/TestPostingsSolrHighlighter.java     |   9 +-
 .../solr/metrics/SolrCoreMetricManagerTest.java    |  13 +-
 .../org/apache/solr/request/TestWriterPerf.java    |  46 ++--
 .../apache/solr/schema/ChangedSchemaMergeTest.java |   2 +-
 .../test/org/apache/solr/schema/CopyFieldTest.java |  32 +--
 .../test/org/apache/solr/schema/EnumFieldTest.java |  15 +-
 .../solr/schema/IndexSchemaRuntimeFieldTest.java   |   1 +
 .../test/org/apache/solr/schema/PolyFieldTest.java |   7 +-
 .../apache/solr/schema/PreAnalyzedFieldTest.java   | 135 +++++-----
 .../org/apache/solr/schema/RequiredFieldsTest.java |   4 +-
 .../solr/schema/ResolveAnalyzerByNameTest.java     |   5 +-
 .../org/apache/solr/schema/TestSchemaField.java    |   4 +-
 .../apache/solr/schema/TestSortableTextField.java  |  76 +++---
 .../org/apache/solr/search/LargeFieldTest.java     |  27 +-
 .../solr/search/TestAddFieldRealTimeGet.java       |  54 ++--
 .../solr/search/TestExtendedDismaxParser.java      |  17 +-
 .../apache/solr/search/TestHashQParserPlugin.java  |  12 +-
 .../org/apache/solr/search/TestIndexSearcher.java  |  52 ++--
 .../apache/solr/search/TestMultiWordSynonyms.java  |  13 +-
 .../apache/solr/search/TestNoOpRegenerator.java    |  48 ++--
 .../apache/solr/search/TestPseudoReturnFields.java |  13 +-
 .../solr/search/TestReRankQParserPlugin.java       |   6 +-
 .../apache/solr/search/TestSolrQueryParser.java    |  79 +++---
 .../test/org/apache/solr/search/TestTrieFacet.java | 103 ++++----
 .../org/apache/solr/search/join/BJQParserTest.java |   9 +-
 .../solr/search/join/TestScoreJoinQPScore.java     |   5 +-
 .../similarities/BaseSimilarityTestCase.java       |   5 +-
 .../solr/servlet/DirectSolrConnectionTest.java     |  15 +-
 .../solr/spelling/DirectSolrSpellCheckerTest.java  |   6 +-
 .../solr/spelling/FileBasedSpellCheckerTest.java   |  10 +-
 .../org/apache/solr/update/AddBlockUpdateTest.java |   5 +-
 .../apache/solr/update/DocumentBuilderTest.java    |   5 +
 .../test/org/apache/solr/update/UpdateLogTest.java |  16 +-
 .../AddSchemaFieldsUpdateProcessorFactoryTest.java |  77 ++++--
 .../processor/DefaultValueUpdateProcessorTest.java |   2 +-
 .../processor/DistributedUpdateProcessorTest.java  |   9 +-
 ...reCommitOptimizeUpdateProcessorFactoryTest.java |   2 +-
 .../processor/UUIDUpdateProcessorFallbackTest.java |   8 +-
 .../UniqFieldsUpdateProcessorFactoryTest.java      |   4 +-
 .../solr/client/solrj/impl/Http2SolrClient.java    |  39 ++-
 .../apache/solr/common/PerThreadExecService.java   |  13 +-
 .../solr/common/util/SolrQueuedThreadPool.java     |   2 +-
 .../apache/solr/common/util/ValidatingJsonMap.java |   7 +-
 .../solr/client/solrj/request/Schema2Test.java     | 267 ++++++++++++++++++++
 .../solr/client/solrj/request/SchemaTest.java      | 174 -------------
 .../src/java/org/apache/solr/SolrTestCaseHS.java   |  14 +-
 96 files changed, 1868 insertions(+), 1583 deletions(-)

diff --git a/solr/contrib/analytics/src/test/org/apache/solr/analytics/function/field/AbstractAnalyticsFieldTest.java b/solr/contrib/analytics/src/test/org/apache/solr/analytics/function/field/AbstractAnalyticsFieldTest.java
index c4479ea..4eb7e46 100644
--- a/solr/contrib/analytics/src/test/org/apache/solr/analytics/function/field/AbstractAnalyticsFieldTest.java
+++ b/solr/contrib/analytics/src/test/org/apache/solr/analytics/function/field/AbstractAnalyticsFieldTest.java
@@ -30,6 +30,7 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.Bits;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.analytics.ExpressionFactory;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.RefCounted;
@@ -209,11 +210,12 @@ public class AbstractAnalyticsFieldTest extends SolrTestCaseJ4 {
       multiBooleans.put(""+i, booleans);
     }
     assertU(commit());
+    try (SolrCore core = h.getCore()) {
+      ref = core.getSearcher();
+      searcher = ref.get();
 
-    ref = h.getCore().getSearcher();
-    searcher = ref.get();
-
-    indexSchema = h.getCore().getLatestSchema();
+      indexSchema = core.getLatestSchema();
+    }
   }
 
   protected ExpressionFactory getExpressionFactory() {
diff --git a/solr/contrib/analytics/src/test/org/apache/solr/analytics/value/CastingBooleanValueStreamTest.java b/solr/contrib/analytics/src/test/org/apache/solr/analytics/value/CastingBooleanValueStreamTest.java
index b498dbe..48d4e55 100644
--- a/solr/contrib/analytics/src/test/org/apache/solr/analytics/value/CastingBooleanValueStreamTest.java
+++ b/solr/contrib/analytics/src/test/org/apache/solr/analytics/value/CastingBooleanValueStreamTest.java
@@ -19,10 +19,12 @@ package org.apache.solr.analytics.value;
 import java.util.Arrays;
 import java.util.Iterator;
 
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.analytics.value.FillableTestValue.TestBooleanValueStream;
 import org.junit.Test;
 
+@LuceneTestCase.Nightly
 public class CastingBooleanValueStreamTest extends SolrTestCaseJ4 {
 
   @Test
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java
index bcbc7ec..ff8cbdc 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java
@@ -67,8 +67,7 @@ public abstract class AbstractDataImportHandlerTestCase extends
   }
 
   protected String loadDataConfig(String dataConfigFileName) {
-    try {
-      SolrCore core = h.getCore();
+    try (SolrCore core = h.getCore()) {
       return SolrWriter.getResourceAsString(core.getResourceLoader()
               .openResource(dataConfigFileName));
     } catch (IOException e) {
diff --git a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
index fcf51eb..8df9e39 100644
--- a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
+++ b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
@@ -62,6 +62,8 @@ import org.slf4j.LoggerFactory;
 public class AnnotatedApi extends Api implements PermissionNameProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  public final static ObjectMapper MAPPER = SolrJacksonAnnotationInspector.INSTANCE.createObjectMapper();
+
   public static final String ERR = "Error executing commands :";
   private EndPoint endPoint;
   private final Map<String, Cmd> commands;
@@ -193,7 +195,7 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
     final String command;
     final Method method;
     final Object obj;
-    ObjectMapper mapper = SolrJacksonAnnotationInspector.INSTANCE.createObjectMapper();
+    ObjectMapper mapper = MAPPER;
     int paramsCount;
     Class c;
     boolean isWrappedInPayloadObj = false;
diff --git a/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java b/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
index 773c116..131c00b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
+++ b/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
@@ -54,7 +54,7 @@ public class StatePublisher implements Closeable {
 
   public static final NoOpMessage TERMINATE_OP = new NoOpMessage();
 
-  private final ArrayBlockingQueue<ZkNodeProps> workQueue = new ArrayBlockingQueue(300, true);
+  private final ArrayBlockingQueue<ZkNodeProps> workQueue = new ArrayBlockingQueue(32, true);
   private final ZkDistributedQueue overseerJobQueue;
   private volatile Worker worker;
   private volatile Future<?> workerFuture;
@@ -68,7 +68,19 @@ public class StatePublisher implements Closeable {
 
     @Override
     public void run() {
-      while (!terminated) {
+      ActionThrottle throttle = new ActionThrottle("StatePublisherWorker", 50);
+
+      while (!terminated && !zkStateReader.getZkClient().isClosed()) {
+        if (!zkStateReader.getZkClient().isConnected()) {
+          try {
+            Thread.sleep(250);
+          } catch (InterruptedException e) {
+
+          }
+          continue;
+        }
+        throttle.minimumWaitBetweenActions();
+        throttle.markAttemptingAction();
         ZkNodeProps message = null;
         ZkNodeProps bulkMessage = new ZkNodeProps();
         bulkMessage.getProperties().put("operation", "state");
@@ -91,9 +103,9 @@ public class StatePublisher implements Closeable {
               try {
                 message = workQueue.poll(30, TimeUnit.MILLISECONDS);
               } catch (InterruptedException e) {
-              return;
-            }
-            if (log.isDebugEnabled()) log.debug("Got state message " + message);
+
+              }
+              if (log.isDebugEnabled()) log.debug("Got state message " + message);
               if (message != null) {
                 if (message == TERMINATE_OP) {
                   terminated = true;
@@ -238,7 +250,7 @@ public class StatePublisher implements Closeable {
   public void close() {
     this.terminated = true;
     try {
-      workerFuture.cancel(true);
+      workerFuture.cancel(false);
     } catch (Exception e) {
       log.error("Exception waiting for close", e);
     }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index ce719ea..0aa92a7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -107,7 +107,6 @@ import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashMap;
@@ -712,21 +711,17 @@ public class ZkController implements Closeable, Runnable {
         }
       });
 
-      closer.collect(overseerElector);
-
-      if (overseer != null) {
-        closer.collect("", () -> {
-          try {
-            overseer.closeAndDone();
-          } catch (Exception e) {
-            log.warn("Exception closing Overseer", e);
-          }
-        });
-      }
-
       collectionToTerms.forEach((s, zkCollectionTerms) -> closer.collect(zkCollectionTerms));
 
     } finally {
+      IOUtils.closeQuietly(overseerElector);
+      if (overseer != null) {
+        try {
+          overseer.closeAndDone();
+        } catch (Exception e) {
+          log.warn("Exception closing Overseer", e);
+        }
+      }
       IOUtils.closeQuietly(zkStateReader);
 
       if (closeZkClient && zkClient != null) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
index b4c91d9..6efadb8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
@@ -23,7 +23,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -196,7 +195,7 @@ public class Assign {
   public static List<ReplicaPosition> getNodesForNewReplicas(ClusterState clusterState, DocCollection collection,
                                                           String shard, int nrtReplicas, int tlogReplicas, int pullReplicas,
                                                           Object createNodeSet, SolrCloudManager cloudManager) throws IOException, InterruptedException, AssignmentException {
-    log.debug("getNodesForNewReplicas() shard: {} , nrtReplicas : {} , tlogReplicas: {} , pullReplicas: {} , createNodeSet {}"
+    if (log.isDebugEnabled()) log.debug("getNodesForNewReplicas() shard: {} , nrtReplicas : {} , tlogReplicas: {} , pullReplicas: {} , createNodeSet {}"
         , shard, nrtReplicas, tlogReplicas, pullReplicas, createNodeSet);
 
     //int maxShardsPerNode = collection.getMaxShardsPerNode() == -1 ? Integer.MAX_VALUE : collection.getMaxShardsPerNode();
@@ -206,11 +205,10 @@ public class Assign {
       createNodeList = (List<String>) createNodeSet;
     } else {
       // deduplicate
-      createNodeList = createNodeSet == null ? null : new ArrayList<>(new LinkedHashSet<>(StrUtils.splitSmart((String) createNodeSet, ",", true)));
+      createNodeList = createNodeSet == null ? null : new ArrayList<>(StrUtils.splitSmart((String) createNodeSet, ",", true));
     }
     String collectionName = collection.getName();
-    HashMap<String, ReplicaCount> nodeNameVsShardCount = getNodeNameVsShardCount(collectionName, clusterState, cloudManager.getClusterStateProvider().getLiveNodes(), createNodeList);
-
+    //HashMap<String, ReplicaCount> nodeNameVsShardCount = getNodeNameVsShardCount(collectionName, clusterState, cloudManager.getClusterStateProvider().getLiveNodes(), createNodeList);
 
     AssignRequest assignRequest = new AssignRequestBuilder()
         .forCollection(collectionName)
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 d699380..ed63a06 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -191,6 +191,9 @@ public class CoreContainer implements Closeable {
 
   public volatile ExecutorService solrCoreExecutor;
 
+  public final ExecutorService coreContainerExecutor = ParWork.getParExecutorService("Core",
+      2, Math.max(6, SysStats.PROC_COUNT), 1000, new LinkedBlockingQueue<>(16));
+
   private final OrderedExecutor replayUpdatesExecutor;
 
   @SuppressWarnings({"rawtypes"})
@@ -400,7 +403,7 @@ public class CoreContainer implements Closeable {
     containerProperties.putAll(cfg.getSolrProperties());
 
     solrCoreExecutor = ParWork.getParExecutorService("Core",
-        4, Math.max(6, SysStats.PROC_COUNT * 2), 1000, new LinkedBlockingQueue<>(1024));
+        4, Math.max(6, SysStats.PROC_COUNT * 2), 1000, new LinkedBlockingQueue<>(64));
   }
 
   @SuppressWarnings({"unchecked"})
@@ -1192,7 +1195,7 @@ public class CoreContainer implements Closeable {
       closer.collect(loader);
 
       closer.collect();
-
+      closer.collect(coreContainerExecutor);
       closer.collect(solrCoreExecutor);
       closer.collect(zkSys);
     }
diff --git a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
index d622734..42613b3 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
@@ -186,8 +186,9 @@ public class CoreDescriptor {
 
     originalCoreProperties.setProperty(CORE_NAME, name);
 
+    HashMap sysProperties = new HashMap(System.getProperties());
     name = PropertiesUtil.substituteProperty(checkPropertyIsNotEmpty(name, CORE_NAME),
-                                             containerProperties);
+                                             containerProperties, sysProperties);
 
     coreProperties.putAll(defaultProperties);
     coreProperties.put(CORE_NAME, name);
@@ -203,7 +204,7 @@ public class CoreDescriptor {
 
       if (!requiredProperties.contains(propname))   // Required props are already dealt with
         coreProperties.setProperty(propname,
-            PropertiesUtil.substituteProperty(propvalue, containerProperties));
+            PropertiesUtil.substituteProperty(propvalue, containerProperties, sysProperties));
     }
 
     loadExtraProperties();
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index 7dace6c..c72db66 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -75,7 +75,7 @@ public class PluginBag<T> implements AutoCloseable {
     // TODO: since reads will dominate writes, we could also think about creating a new instance of a map each time it changes.
     // Not sure how much benefit this would have over ConcurrentHashMap though
     // We could also perhaps make this constructor into a factory method to return different implementations depending on thread safety needs.
-    this.registry = new ConcurrentHashMap<>(64, 0.75f, 3);
+    this.registry = new ConcurrentHashMap<>(32, 0.75f, 6);
     this.immutableRegistry = Collections.unmodifiableMap(registry);
     meta = SolrConfig.classVsSolrPluginInfo.get(klass.getName());
     if (meta == null) {
@@ -233,7 +233,9 @@ public class PluginBag<T> implements AutoCloseable {
     PluginHolder<T> old = null;
     if (!disableHandler) old = registry.put(name, plugin);
     if (plugin.pluginInfo != null && plugin.pluginInfo.isDefault()) setDefault(name);
-    if (plugin.isLoaded()) registerMBean(plugin.get(), core, name);
+    if (plugin.isLoaded()) {
+      registerMBean(plugin.get(), core, name);
+    }
     // old instance has been replaced - close it to prevent mem leaks
     if (old != null && old != plugin) {
       closeQuietly(old);
@@ -308,7 +310,7 @@ public class PluginBag<T> implements AutoCloseable {
     return result.isLoaded();
   }
 
-  private void registerMBean(Object inst, SolrCore core, String pluginKey) {
+  private static void registerMBean(Object inst, SolrCore core, String pluginKey) {
     if (core == null) return;
     if (inst instanceof SolrInfoBean) {
       SolrInfoBean mBean = (SolrInfoBean) inst;
@@ -401,7 +403,7 @@ public class PluginBag<T> implements AutoCloseable {
    * A class that loads plugins Lazily. When the get() method is invoked
    * the Plugin is initialized and returned.
    */
-  public class LazyPluginHolder<T> extends PluginHolder<T> {
+  public static class LazyPluginHolder<T> extends PluginHolder<T> {
     private volatile T lazyInst;
     private final SolrConfig.SolrPluginInfo pluginMeta;
     protected SolrException solrException;
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 193cbb7..26724db 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -242,7 +242,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
 
   private final Set<String> metricNames = ConcurrentHashMap.newKeySet(64);
   private final String metricTag = SolrMetricProducer.getUniqueMetricTag(this, null);
-  private final SolrMetricsContext solrMetricsContext;
+  private volatile SolrMetricsContext solrMetricsContext;
 
   public volatile boolean searchEnabled = true;
   public volatile boolean indexEnabled = true;
@@ -737,7 +737,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
         log.info("Wait for reload lock");
 
         while (!(lock.tryLock() || lock.tryLock(250, TimeUnit.MILLISECONDS))) {
-          if (coreContainer.isShutDown() || isClosed() || closing) {
+          if (closing) {
             log.warn("Skipping reload because we are closed");
             reloadyWaiting.decrementAndGet();
             throw new AlreadyClosedException();
@@ -755,9 +755,6 @@ public final class SolrCore implements SolrInfoBean, Closeable {
         }
       }
 
-      if (coreContainer.isShutDown()) {
-        throw new AlreadyClosedException();
-      }
       final SolrCore currentCore;
       if (!getNewIndexDir().equals(getIndexDir())) {
         // the directory is changing, don't pass on state
@@ -1063,11 +1060,15 @@ public final class SolrCore implements SolrInfoBean, Closeable {
       // Initialize the RestManager
       restManager = initRestManager(cd);
 
-      // Initialize the metrics manager
       this.coreMetricManager = initCoreMetricManager(solrConfig);
-      solrMetricsContext = coreMetricManager.getSolrMetricsContext();
-      this.coreMetricManager.loadReporters();
 
+      newSearcherCounter = coreMetricManager.getSolrMetricsContext().counter("new", Category.SEARCHER.toString());
+      newSearcherTimer = coreMetricManager.getSolrMetricsContext().timer("time", Category.SEARCHER.toString(), "new");
+      newSearcherWarmupTimer = coreMetricManager.getSolrMetricsContext().timer("warmup", Category.SEARCHER.toString(), "new");
+      newSearcherMaxReachedCounter = coreMetricManager.getSolrMetricsContext().counter("maxReached", Category.SEARCHER.toString(), "new");
+      newSearcherOtherErrorsCounter = coreMetricManager.getSolrMetricsContext().counter("errors", Category.SEARCHER.toString(), "new");
+
+      this.coreMetricManager.loadReporters();
 
       if (updateHandler == null) {
         directoryFactory = initDirectoryFactory();
@@ -1090,13 +1091,9 @@ public final class SolrCore implements SolrInfoBean, Closeable {
       checkVersionFieldExistsInSchema(schema, coreDescriptor);
       setLatestSchema(schema);
 
-      // initialize core metrics
-      initializeMetrics(solrMetricsContext, null);
-
       SolrFieldCacheBean solrFieldCacheBean = new SolrFieldCacheBean();
       // this is registered at the CONTAINER level because it's not core-specific - for now we
       // also register it here for back-compat
-      solrFieldCacheBean.initializeMetrics(solrMetricsContext, "core");
       infoRegistry.put("fieldCache", solrFieldCacheBean);
 
       this.maxWarmingSearchers = solrConfig.maxWarmingSearchers;
@@ -1143,11 +1140,6 @@ public final class SolrCore implements SolrInfoBean, Closeable {
 
       infoRegistry.put("core", this);
 
-      // Allow the directory factory to report metrics
-      if (directoryFactory instanceof SolrMetricProducer) {
-        ((SolrMetricProducer) directoryFactory).initializeMetrics(solrMetricsContext, "directoryFactory");
-      }
-
       bufferUpdatesIfConstructing(coreDescriptor);
 
 
@@ -1336,12 +1328,6 @@ public final class SolrCore implements SolrInfoBean, Closeable {
 
   @Override
   public void initializeMetrics(SolrMetricsContext parentContext, String scope) {
-    newSearcherCounter = parentContext.counter("new", Category.SEARCHER.toString());
-    newSearcherTimer = parentContext.timer("time", Category.SEARCHER.toString(), "new");
-    newSearcherWarmupTimer = parentContext.timer("warmup", Category.SEARCHER.toString(), "new");
-    newSearcherMaxReachedCounter = parentContext.counter("maxReached", Category.SEARCHER.toString(), "new");
-    newSearcherOtherErrorsCounter = parentContext.counter("errors", Category.SEARCHER.toString(), "new");
-
     parentContext.gauge(() -> name == null ? "(null)" : name, true, "coreName", Category.CORE.toString());
     parentContext.gauge(() -> startTime, true, "startTime", Category.CORE.toString());
     parentContext.gauge(() -> getOpenCount(), true, "refCount", Category.CORE.toString());
@@ -1392,6 +1378,21 @@ public final class SolrCore implements SolrInfoBean, Closeable {
 
   @Override
   public SolrMetricsContext getSolrMetricsContext() {
+    if (solrMetricsContext == null) {
+      // Initialize the metrics manager
+
+      solrMetricsContext = coreMetricManager.getSolrMetricsContext();
+      // initialize core metrics
+      initializeMetrics(solrMetricsContext, null);
+      SolrFieldCacheBean solrFieldCacheBean = (SolrFieldCacheBean) infoRegistry.get("fieldcache");
+      if (solrFieldCacheBean != null) {
+        solrFieldCacheBean.initializeMetrics(solrMetricsContext, "core");
+      }
+      // Allow the directory factory to report metrics
+      if (directoryFactory instanceof SolrMetricProducer) {
+        ((SolrMetricProducer) directoryFactory).initializeMetrics(solrMetricsContext, "directoryFactory");
+      }
+    }
     return solrMetricsContext;
   }
 
@@ -1710,7 +1711,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
 //      log.debug("close refcount after {} {}", this, count);
 //    }
 
-    if (count == 0) {
+    if (count == 0 || coreContainer.solrCores.isClosed() && count == 1) {
       try {
         doClose();
       } catch (Exception e1) {
@@ -1738,8 +1739,10 @@ public final class SolrCore implements SolrInfoBean, Closeable {
     while (!canBeClosed() || refCount.get() != -1) {
       cnt++;
       try {
-        synchronized (closeAndWait) {
-          closeAndWait.wait(250);
+        if (!closing) {
+          synchronized (closeAndWait) {
+            closeAndWait.wait(250);
+          }
         }
         if (cnt >= 2 && !closing) {
           close();
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCores.java b/solr/core/src/java/org/apache/solr/core/SolrCores.java
index beb31c2..530bb46 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCores.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCores.java
@@ -223,7 +223,7 @@ class SolrCores implements Closeable {
 //  }
 
   protected void swap(String n0, String n1) {
-    if (isClosed()) {
+    if (isClosed() || container.isShutDown()) {
       throw new AlreadyClosedException();
     }
     synchronized (cores) {
@@ -258,8 +258,8 @@ class SolrCores implements Closeable {
 
   }
 
-  private boolean isClosed() {
-    return closed || container.isShutDown();
+  boolean isClosed() {
+    return closed;
   }
 
   protected SolrCore remove(String name) {
diff --git a/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java b/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
index 33b325e..baa3a36 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
@@ -21,10 +21,12 @@ import net.sf.saxon.om.AttributeInfo;
 import net.sf.saxon.tree.tiny.TinyBuilder;
 import org.apache.solr.util.PropertiesUtil;
 
+import java.util.HashMap;
 import java.util.Properties;
 
 public class SolrTinyBuilder extends TinyBuilder  {
   private final Properties substituteProps;
+  private final HashMap sysProperties;
 
   /**
    * Create a TinyTree builder
@@ -35,12 +37,13 @@ public class SolrTinyBuilder extends TinyBuilder  {
   public SolrTinyBuilder(PipelineConfiguration pipe, Properties substituteProps) {
     super(pipe);
     this.substituteProps = substituteProps;
+    this.sysProperties = new HashMap(System.getProperties());
   }
 
   protected int makeTextNode(CharSequence chars, int len) {
     String sub = PropertiesUtil
         .substituteProperty(chars.subSequence(0, len).toString(),
-            substituteProps);
+            substituteProps, sysProperties);
 
     return super.makeTextNode(sub, sub.length());
   }
@@ -48,7 +51,7 @@ public class SolrTinyBuilder extends TinyBuilder  {
   protected String getAttValue(AttributeInfo att) {
     String sub = PropertiesUtil
         .substituteProperty(att.getValue(),
-            substituteProps);
+            substituteProps, sysProperties);
     return sub;
   }
 
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 7d7b2d8..fe92433 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -195,6 +195,7 @@ public class SolrXmlConfig {
 
   private static Properties loadProperties(XmlConfigFile config) {
     try {
+      HashMap sysProperties = new HashMap(System.getProperties());
       NodeInfo node = (NodeInfo) ((ArrayList) config.evaluate(config.tree, "solr", XPathConstants.NODESET)).get(0);
 
       ArrayList<NodeInfo> props = (ArrayList) config.getResourceLoader().getXPath().evaluate("property", node, XPathConstants.NODESET);
@@ -202,7 +203,7 @@ public class SolrXmlConfig {
       for (int i = 0; i < props.size(); i++) {
         NodeInfo prop = props.get(i);
         properties.setProperty(DOMUtil.getAttr(prop, NAME),
-            PropertiesUtil.substituteProperty(DOMUtil.getAttr(prop, "value"), null));
+            PropertiesUtil.substituteProperty(DOMUtil.getAttr(prop, "value"), null, sysProperties));
       }
       return properties;
     }
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
index 708172b..44986b3 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
@@ -19,6 +19,7 @@ package org.apache.solr.handler.admin;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
+import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Optional;
@@ -90,7 +91,7 @@ enum CoreAdminOperation implements CoreAdminOp {
       String instanceDir = it.req.getParams().get(CoreAdminParams.INSTANCE_DIR);
       if (instanceDir == null) instanceDir = it.req.getParams().get("property.instanceDir");
       if (instanceDir != null) {
-        instanceDir = PropertiesUtil.substituteProperty(instanceDir, coreContainer.getContainerProperties());
+        instanceDir = PropertiesUtil.substituteProperty(instanceDir, coreContainer.getContainerProperties(), new HashMap(System.getProperties()));
         instancePath = coreContainer.getCoreRootDirectory().resolve(instanceDir).normalize();
       } else {
         instancePath = coreContainer.getCoreRootDirectory().resolve(coreName);
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
index a1fe253..f073102 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -36,7 +36,7 @@ import org.apache.solr.core.SolrInfoBean;
 public class SolrCoreMetricManager implements Closeable {
 
   private final SolrCore core;
-  private SolrMetricsContext solrMetricsContext;
+  private volatile SolrMetricsContext solrMetricsContext;
   private SolrMetricManager metricManager;
   private String collectionName;
   private String shardName;
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 e7c192a..fb9ce95 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -959,7 +959,7 @@ public class IndexSchema {
       throw new SolrException(ErrorCode.SERVER_ERROR, msg);
     }
     if (null == destSchemaField) {
-      String msg = "copyField dest :'" + dest + "' is not an explicit field and doesn't match a dynamicField.";
+      String msg = "copyField dest :'" + dest + "' is not an explicit field and doesn't match a dynamicField. " + dynamicFields;
       throw new SolrException(ErrorCode.SERVER_ERROR, msg);
     }
     if (sourceIsGlob) {
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index af6eb02..73dea40 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -63,6 +63,7 @@ import io.opentracing.tag.Tags;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.http.HttpHeaders;
 import org.apache.lucene.util.Version;
+import org.apache.solr.api.AnnotatedApi;
 import org.apache.solr.api.V2HttpCall;
 import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.common.ParWork;
@@ -89,7 +90,6 @@ import org.apache.solr.security.AuditEvent;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
 import org.apache.solr.security.PublicKeyHandler;
-import org.apache.solr.util.SolrJacksonAnnotationInspector;
 import org.apache.solr.util.StartupLoggingUtils;
 import org.apache.solr.util.configuration.SSLConfigurationsFactory;
 import org.apache.solr.util.tracing.GlobalTracer;
@@ -97,7 +97,6 @@ import org.apache.zookeeper.KeeperException;
 import org.eclipse.jetty.client.HttpClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.slf4j.MDC;
 
 import static org.apache.solr.security.AuditEvent.EventType;
 
@@ -110,8 +109,9 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   static {
-    log.warn("expected pre init of factories {} {} {} {}",
-        FieldTypeXmlAdapter.dbf, XMLResponseParser.inputFactory, XMLResponseParser.saxFactory, SolrJacksonAnnotationInspector.INSTANCE);
+    log.warn("expected pre init of factories {} {} {} {} {} {}",
+        FieldTypeXmlAdapter.dbf, XMLResponseParser.inputFactory, XMLResponseParser.saxFactory,
+        AnnotatedApi.MAPPER, org.apache.http.conn.util.PublicSuffixMatcherLoader.getDefault());
   }
 
   private volatile StopRunnable stopRunnable;
diff --git a/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java b/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
index 569239a..eb7cd4c 100644
--- a/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
+++ b/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
@@ -21,6 +21,7 @@ import org.apache.solr.common.SolrException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 /**
@@ -32,7 +33,7 @@ public class PropertiesUtil {
   * This method borrowed from Ant's PropertyHelper.replaceProperties:
   *   http://svn.apache.org/repos/asf/ant/core/trunk/src/main/org/apache/tools/ant/PropertyHelper.java
   */
-  public static String substituteProperty(String value, Properties coreProperties) {
+  public static String substituteProperty(String value, Properties coreProperties, Map systemProperties) {
     if (value == null || value.indexOf('$') == -1) {
       return value;
     }
@@ -59,7 +60,10 @@ public class PropertiesUtil {
           fragment = coreProperties.getProperty(propertyName);
         }
         if (fragment == null) {
-          fragment = System.getProperty(propertyName, defaultValue);
+          fragment = (String) systemProperties.get(propertyName);
+          if (fragment == null) {
+            fragment = defaultValue;
+          }
         }
         if (fragment == null) {
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No system property or default value specified for " + propertyName + " value:" + value);
diff --git a/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java b/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
index 1383467..89cf893 100644
--- a/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
+++ b/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
@@ -17,6 +17,7 @@
 package org.apache.solr;
 
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.core.SolrCore;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -43,8 +44,10 @@ public class MinimalSchemaTest extends SolrTestCaseJ4 {
     /* make sure some misguided soul doesn't inadvertently give us 
        a uniqueKey field and defeat the point of the tests
     */
+    SolrCore core = h.getCore();
     assertNull("UniqueKey Field isn't null", 
-               h.getCore().getLatestSchema().getUniqueKeyField());
+               core.getLatestSchema().getUniqueKeyField());
+    core.close();
 
     lrf.args.put(CommonParams.VERSION,"2.2");
 
@@ -107,7 +110,9 @@ public class MinimalSchemaTest extends SolrTestCaseJ4 {
    */
   @Test
   public void testAllConfiguredHandlers() {
-    Set<String> handlerNames = h.getCore().getRequestHandlers().keySet();
+    SolrCore core = h.getCore();
+    Set<String> handlerNames = core.getRequestHandlers().keySet();
+    core.close();
     for (String handler : handlerNames) {
       try {
 
diff --git a/solr/core/src/test/org/apache/solr/OutputWriterTest.java b/solr/core/src/test/org/apache/solr/OutputWriterTest.java
index 585dd9a..b073bbd 100644
--- a/solr/core/src/test/org/apache/solr/OutputWriterTest.java
+++ b/solr/core/src/test/org/apache/solr/OutputWriterTest.java
@@ -22,6 +22,7 @@ import java.io.Writer;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.PluginBag;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
@@ -68,7 +69,7 @@ public class OutputWriterTest extends SolrTestCaseJ4 {
     @Test
     public void testUselessWriter() throws Exception {
         lrf.args.put("wt", "useless");
-        String out = h.query(req("foo"));
+        String out = query(req("foo"));
         assertEquals(USELESS_OUTPUT, out);
     }
     
@@ -76,7 +77,7 @@ public class OutputWriterTest extends SolrTestCaseJ4 {
     public void testTrivialXsltWriter() throws Exception {
         lrf.args.put("wt", "xslt");
         lrf.args.put("tr", "dummy.xsl");
-        String out = h.query(req("foo"));
+        String out = query(req("foo"));
         // System.out.println(out);
         assertTrue(out.contains("DUMMY"));
     }
@@ -85,19 +86,20 @@ public class OutputWriterTest extends SolrTestCaseJ4 {
     public void testTrivialXsltWriterInclude() throws Exception {
         lrf.args.put("wt", "xslt");
         lrf.args.put("tr", "dummy-using-include.xsl");
-        String out = h.query(req("foo"));
+        String out = query(req("foo"));
         // System.out.println(out);
         assertTrue(out.contains("DUMMY"));
     }
 
     public void testLazy() {
-        PluginBag.PluginHolder<QueryResponseWriter> qrw = h.getCore().getResponseWriters().getRegistry().get("useless");
-        assertTrue("Should be a lazy class", qrw instanceof PluginBag.LazyPluginHolder);
-
-        qrw = h.getCore().getResponseWriters().getRegistry().get("xml");
-        assertTrue("Should not be a lazy class", qrw.isLoaded());
-        assertTrue("Should not be a lazy class", qrw.getClass() == PluginBag.PluginHolder.class);
+        try (SolrCore core = h.getCore()) {
+            PluginBag.PluginHolder<QueryResponseWriter> qrw = core.getResponseWriters().getRegistry().get("useless");
+            assertTrue("Should be a lazy class", qrw instanceof PluginBag.LazyPluginHolder);
 
+            qrw = core.getResponseWriters().getRegistry().get("xml");
+            assertTrue("Should not be a lazy class", qrw.isLoaded());
+            assertTrue("Should not be a lazy class", qrw.getClass() == PluginBag.PluginHolder.class);
+        }
     }
     
     ////////////////////////////////////////////////////////////////////////////
diff --git a/solr/core/src/test/org/apache/solr/SolrInfoBeanTest.java b/solr/core/src/test/org/apache/solr/SolrInfoBeanTest.java
index cc4a102..58a083f 100644
--- a/solr/core/src/test/org/apache/solr/SolrInfoBeanTest.java
+++ b/solr/core/src/test/org/apache/solr/SolrInfoBeanTest.java
@@ -17,6 +17,7 @@
 package org.apache.solr;
 
 import org.apache.lucene.util.TestUtil;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.handler.admin.LukeRequestHandler;
 import org.apache.solr.handler.component.SearchComponent;
@@ -56,71 +57,72 @@ public class SolrInfoBeanTest extends SolrTestCaseJ4
     classes.addAll(getClassesForPackage(DefaultSolrHighlighter.class.getPackage().getName()));
     classes.addAll(getClassesForPackage(CaffeineCache.class.getPackage().getName()));
    // System.out.println(classes);
-    
-    int checked = 0;
-    SolrMetricManager metricManager = h.getCoreContainer().getMetricManager();
-    String registry = h.getCore().getCoreMetricManager().getRegistryName();
-    SolrMetricsContext solrMetricsContext = new SolrMetricsContext(metricManager, registry, "foo");
-    String scope = TestUtil.randomSimpleString(random(), 2, 10);
-    for( Class clazz : classes ) {
-      if( SolrInfoBean.class.isAssignableFrom( clazz ) ) {
-        try {
-          SolrInfoBean info = (SolrInfoBean)clazz.getConstructor().newInstance();
-          if (info instanceof SolrMetricProducer) {
-            ((SolrMetricProducer)info).initializeMetrics(solrMetricsContext, scope);
-          }
-          
-          //System.out.println( info.getClass() );
-          assertNotNull( info.getClass().getCanonicalName(), info.getName() );
-          assertNotNull( info.getClass().getCanonicalName(), info.getDescription() );
-          assertNotNull( info.getClass().getCanonicalName(), info.getCategory() );
-          
-          if( info instanceof CaffeineCache ) {
-            continue;
+    try (SolrCore core = h.getCore()) {
+      int checked = 0;
+      SolrMetricManager metricManager = h.getCoreContainer().getMetricManager();
+      String registry = core.getCoreMetricManager().getRegistryName();
+      SolrMetricsContext solrMetricsContext = new SolrMetricsContext(metricManager, registry, "foo");
+      String scope = TestUtil.randomSimpleString(random(), 2, 10);
+      for (Class clazz : classes) {
+        if (SolrInfoBean.class.isAssignableFrom(clazz)) {
+          try {
+            SolrInfoBean info = (SolrInfoBean) clazz.getConstructor().newInstance();
+            if (info instanceof SolrMetricProducer) {
+              ((SolrMetricProducer) info).initializeMetrics(solrMetricsContext, scope);
+            }
+
+            //System.out.println( info.getClass() );
+            assertNotNull(info.getClass().getCanonicalName(), info.getName());
+            assertNotNull(info.getClass().getCanonicalName(), info.getDescription());
+            assertNotNull(info.getClass().getCanonicalName(), info.getCategory());
+
+            if (info instanceof CaffeineCache) {
+              continue;
+            }
+
+            assertNotNull(info.toString());
+            checked++;
+          } catch (ReflectiveOperationException ex) {
+            // expected...
+            //System.out.println( "unable to initialize: "+clazz );
           }
-          
-          assertNotNull( info.toString() );
-          checked++;
-        }
-        catch( ReflectiveOperationException ex ) {
-          // expected...
-          //System.out.println( "unable to initialize: "+clazz );
         }
       }
+      assertTrue("there are at least 10 SolrInfoBean that should be found in the classpath, found " + checked, checked > 10);
     }
-    assertTrue( "there are at least 10 SolrInfoBean that should be found in the classpath, found " + checked, checked > 10 );
   }
   
   private static List<Class> getClassesForPackage(String pckgname) throws Exception {
-    ArrayList<File> directories = new ArrayList<>();
-    ClassLoader cld = h.getCore().getResourceLoader().getClassLoader();
-    String path = pckgname.replace('.', '/');
-    Enumeration<URL> resources = cld.getResources(path);
-    while (resources.hasMoreElements()) {
-      final URI uri = resources.nextElement().toURI();
-      if (!"file".equalsIgnoreCase(uri.getScheme()))
-        continue;
-      final File f = new File(uri);
-      directories.add(f);
-    }
-      
-    ArrayList<Class> classes = new ArrayList<>();
-    for (File directory : directories) {
-      if (directory.exists()) {
-        String[] files = directory.list();
-        for (String file : files) {
-          if (file.endsWith(".class")) {
-             String clazzName = file.substring(0, file.length() - 6);
-             // exclude Test classes that happen to be in these packages.
-             // class.ForName'ing some of them can cause trouble.
-             if (!clazzName.endsWith("Test") && !clazzName.startsWith("Test")) {
-               classes.add(Class.forName(pckgname + '.' + clazzName));
-             }
+    try (SolrCore core = h.getCore()) {
+      ArrayList<File> directories = new ArrayList<>();
+      ClassLoader cld = core.getResourceLoader().getClassLoader();
+      String path = pckgname.replace('.', '/');
+      Enumeration<URL> resources = cld.getResources(path);
+      while (resources.hasMoreElements()) {
+        final URI uri = resources.nextElement().toURI();
+        if (!"file".equalsIgnoreCase(uri.getScheme())) continue;
+        final File f = new File(uri);
+        directories.add(f);
+      }
+
+      ArrayList<Class> classes = new ArrayList<>();
+      for (File directory : directories) {
+        if (directory.exists()) {
+          String[] files = directory.list();
+          for (String file : files) {
+            if (file.endsWith(".class")) {
+              String clazzName = file.substring(0, file.length() - 6);
+              // exclude Test classes that happen to be in these packages.
+              // class.ForName'ing some of them can cause trouble.
+              if (!clazzName.endsWith("Test") && !clazzName.startsWith("Test")) {
+                classes.add(Class.forName(pckgname + '.' + clazzName));
+              }
+            }
           }
         }
       }
+      assertFalse("No classes found in package '" + pckgname + "'; maybe your test classes are packaged as JAR file?", classes.isEmpty());
+      return classes;
     }
-    assertFalse("No classes found in package '"+pckgname+"'; maybe your test classes are packaged as JAR file?", classes.isEmpty());
-    return classes;
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/TestJoin.java b/solr/core/src/test/org/apache/solr/TestJoin.java
index 40dbea0..29e0dd8 100644
--- a/solr/core/src/test/org/apache/solr/TestJoin.java
+++ b/solr/core/src/test/org/apache/solr/TestJoin.java
@@ -155,12 +155,12 @@ public class TestJoin extends SolrTestCaseJ4 {
 
     // "from" field missing docValues
     expectThrows(SolrException.class, () -> {
-      h.query(req(p, "q", "{!join from=nodocvalues_s to=dept_ss_dv method=topLevelDV}*:*", "fl","id"));
+      query(req(p, "q", "{!join from=nodocvalues_s to=dept_ss_dv method=topLevelDV}*:*", "fl","id"));
     });
 
     // "to" field missing docValues
     expectThrows(SolrException.class, () -> {
-      h.query(req(p, "q", "{!join from=dept_ss_dv to=nodocvalues_s method=topLevelDV}*:*", "fl","id"));
+      query(req(p, "q", "{!join from=dept_ss_dv to=nodocvalues_s method=topLevelDV}*:*", "fl","id"));
     });
   }
 
diff --git a/solr/core/src/test/org/apache/solr/TestTrie.java b/solr/core/src/test/org/apache/solr/TestTrie.java
index 5d71669..55d405e 100644
--- a/solr/core/src/test/org/apache/solr/TestTrie.java
+++ b/solr/core/src/test/org/apache/solr/TestTrie.java
@@ -20,6 +20,7 @@ import java.text.SimpleDateFormat;
 import java.util.Locale;
 import java.util.TimeZone;
 
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.TrieField;
@@ -275,10 +276,12 @@ public class TestTrie extends SolrTestCaseJ4 {
   }
 
   private void checkPrecisionSteps(String fieldType) {
-    FieldType type = h.getCore().getLatestSchema().getFieldType(fieldType);
-    if (type instanceof TrieField) {
-      TrieField field = (TrieField) type;
-      assertTrue(field.getPrecisionStep() > 0 && field.getPrecisionStep() < 64);
+    try (SolrCore core = h.getCore()) {
+      FieldType type = core.getLatestSchema().getFieldType(fieldType);
+      if (type instanceof TrieField) {
+        TrieField field = (TrieField) type;
+        assertTrue(field.getPrecisionStep() > 0 && field.getPrecisionStep() < 64);
+      }
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
index 38941a1..b20857d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
@@ -37,6 +37,7 @@ import org.apache.solr.common.params.SolrParams;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -235,14 +236,17 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
     testMalformedDBQ(S_TWO_LEADER_CLIENT);
   }
 
+  @Ignore // TEST-MRM TODO: flakey - are we returning the error right for a forward to leader?
   public void testMalformedDBQViaShard1NonLeaderClient() throws Exception {
     testMalformedDBQ(S_ONE_NON_LEADER_CLIENT);
   }
 
+  @Ignore // TEST-MRM TODO: flakey
   public void testMalformedDBQViaShard2NonLeaderClient() throws Exception {
     testMalformedDBQ(S_TWO_NON_LEADER_CLIENT);
   }
 
+  @Ignore // TEST-MRM TODO: flakey
   public void testMalformedDBQViaNoCollectionClient() throws Exception {
     testMalformedDBQ(NO_COLLECTION_CLIENT);
   }
diff --git a/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java b/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java
index 9bf08a5..a18743a 100644
--- a/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java
+++ b/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java
@@ -43,11 +43,11 @@ public class AlternateDirectoryTest extends SolrTestCaseJ4 {
   }
   
   public void testAltReaderUsed() throws Exception {
-    IndexReaderFactory readerFactory = h.getCore().getIndexReaderFactory();
-    assertNotNull("Factory is null", readerFactory);
-    assertEquals("readerFactory is wrong class",
-                 AlternateDirectoryTest.TestIndexReaderFactory.class.getName(), 
-                 readerFactory.getClass().getName());
+    try (SolrCore core = h.getCore()) {
+      IndexReaderFactory readerFactory = core.getIndexReaderFactory();
+      assertNotNull("Factory is null", readerFactory);
+      assertEquals("readerFactory is wrong class", AlternateDirectoryTest.TestIndexReaderFactory.class.getName(), readerFactory.getClass().getName());
+    }
   }
 
   static public class TestFSDirectoryFactory extends StandardDirectoryFactory {
diff --git a/solr/core/src/test/org/apache/solr/core/ByteBuffersDirectoryFactoryTest.java b/solr/core/src/test/org/apache/solr/core/ByteBuffersDirectoryFactoryTest.java
index 2d66969..89c36c9 100644
--- a/solr/core/src/test/org/apache/solr/core/ByteBuffersDirectoryFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/core/ByteBuffersDirectoryFactoryTest.java
@@ -55,13 +55,15 @@ public class ByteBuffersDirectoryFactoryTest extends SolrTestCaseJ4 {
   public void testIndexRetrieve() throws Exception {
     System.setProperty("solr.directoryFactory", "solr.ByteBuffersDirectoryFactory");
     initCore("solrconfig-minimal.xml","schema-minimal.xml");
-    DirectoryFactory factory = h.getCore().getDirectoryFactory();
-    assertTrue("Found: " + factory.getClass().getName(), factory instanceof ByteBuffersDirectoryFactory);
-    for (int i = 0 ; i < 5 ; ++i) {
-      assertU(adoc("id", "" + i, "a_s", "_" + i + "_"));
+    try (SolrCore core = h.getCore()) {
+      DirectoryFactory factory = core.getDirectoryFactory();
+      assertTrue("Found: " + factory.getClass().getName(), factory instanceof ByteBuffersDirectoryFactory);
+      for (int i = 0; i < 5; ++i) {
+        assertU(adoc("id", "" + i, "a_s", "_" + i + "_"));
+      }
+      assertU(commit());
+      assertQ(req("q", "a_s:_0_"), "//result[@numFound = '1']");
     }
-    assertU(commit());
-    assertQ(req("q", "a_s:_0_"), "//result[@numFound = '1']");
     deleteCore();
   }
 
diff --git a/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java b/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
index afad751..d22de0a 100644
--- a/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
+++ b/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
@@ -49,24 +49,23 @@ public class ConfigureRecoveryStrategyTest extends SolrTestCaseJ4 {
   }
 
   public void testBuilder() throws Exception {
-    final RecoveryStrategy.Builder recoveryStrategyBuilder =
-        h.getCore().getSolrCoreState().getRecoveryStrategyBuilder();
-    assertNotNull("recoveryStrategyBuilder is null", recoveryStrategyBuilder);
-
-    final String expectedClassName;
-
-    if (solrConfigFileName.equals(solrConfigFileNameConfigure)) {
-      expectedClassName = RecoveryStrategy.Builder.class.getName();
-    } else if (solrConfigFileName.equals(solrConfigFileNameCustom)) {
-      assertTrue("recoveryStrategyBuilder is wrong class (instanceof)",
-          recoveryStrategyBuilder instanceof CustomRecoveryStrategyBuilder);
-      expectedClassName = ConfigureRecoveryStrategyTest.CustomRecoveryStrategyBuilder.class.getName();
-    } else {
-      expectedClassName = null;
-    }
+    try (SolrCore core = h.getCore()) {
+      final RecoveryStrategy.Builder recoveryStrategyBuilder = core.getSolrCoreState().getRecoveryStrategyBuilder();
+      assertNotNull("recoveryStrategyBuilder is null", recoveryStrategyBuilder);
+
+      final String expectedClassName;
+
+      if (solrConfigFileName.equals(solrConfigFileNameConfigure)) {
+        expectedClassName = RecoveryStrategy.Builder.class.getName();
+      } else if (solrConfigFileName.equals(solrConfigFileNameCustom)) {
+        assertTrue("recoveryStrategyBuilder is wrong class (instanceof)", recoveryStrategyBuilder instanceof CustomRecoveryStrategyBuilder);
+        expectedClassName = ConfigureRecoveryStrategyTest.CustomRecoveryStrategyBuilder.class.getName();
+      } else {
+        expectedClassName = null;
+      }
 
-    assertEquals("recoveryStrategyBuilder is wrong class (name)",
-        expectedClassName, recoveryStrategyBuilder.getClass().getName());
+      assertEquals("recoveryStrategyBuilder is wrong class (name)", expectedClassName, recoveryStrategyBuilder.getClass().getName());
+    }
   }
 
   public void testAlmostAllMethodsAreFinal() throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/core/RequestHandlersTest.java b/solr/core/src/test/org/apache/solr/core/RequestHandlersTest.java
index b026574..87aed0e 100644
--- a/solr/core/src/test/org/apache/solr/core/RequestHandlersTest.java
+++ b/solr/core/src/test/org/apache/solr/core/RequestHandlersTest.java
@@ -34,89 +34,79 @@ public class RequestHandlersTest extends SolrTestCaseJ4 {
 
   @Test
   public void testInitCount() {
-    String registry = h.getCore().getCoreMetricManager().getRegistryName();
-    SolrMetricManager manager = h.getCoreContainer().getMetricManager();
-    Gauge<Number> g = (Gauge<Number>)manager.registry(registry).getMetrics().get("QUERY./mock.initCount");
-    assertEquals("Incorrect init count",
-                 1, g.getValue().intValue());
+    try (SolrCore core = h.getCore()) {
+      String registry = core.getCoreMetricManager().getRegistryName();
+      SolrMetricManager manager = h.getCoreContainer().getMetricManager();
+      Gauge<Number> g = (Gauge<Number>) manager.registry(registry).getMetrics().get("QUERY./mock.initCount");
+      assertEquals("Incorrect init count", 1, g.getValue().intValue());
+    }
   }
 
   @Test
   public void testImplicitRequestHandlers(){
-    SolrCore core = h.getCore();
-    assertNotNull(core.getRequestHandler( "/update/json"));
-    assertNotNull(core.getRequestHandler( "/update/json/docs"));
-    assertNotNull(core.getRequestHandler( "/update/csv"));
+    try (SolrCore core = h.getCore()) {
+      assertNotNull(core.getRequestHandler("/update/json"));
+      assertNotNull(core.getRequestHandler("/update/json/docs"));
+      assertNotNull(core.getRequestHandler("/update/csv"));
+    }
   }
 
   @Test
   public void testLazyLoading() {
-    SolrCore core = h.getCore();
-    PluginBag.PluginHolder<SolrRequestHandler> handler = core.getRequestHandlers().getRegistry().get("/lazy");
-    assertFalse(handler.isLoaded());
-    
-    assertU(adoc("id", "42",
-                 "name", "Zapp Brannigan"));
-    assertU(adoc("id", "43",
-                 "title", "Democratic Order of Planets"));
-    assertU(adoc("id", "44",
-                 "name", "The Zapper"));
-    assertU(adoc("id", "45",
-                 "title", "25 star General"));
-    assertU(adoc("id", "46",
-                 "subject", "Defeated the pacifists of the Gandhi nebula"));
-    assertU(adoc("id", "47",
-                 "text", "line up and fly directly at the enemy death cannons, clogging them with wreckage!"));
-    assertU(commit());
-
-    assertQ("lazy request handler returns all matches",
-            req("q","id:[42 TO 47]"),
-            "*[count(//doc)=6]");
-
-        // But it should behave just like the 'defaults' request handler above
-    assertQ("lazy handler returns fewer matches",
-            req("q", "id:[42 TO 47]", "qt","/lazy"),
-            "*[count(//doc)=4]"
-            );
-
-    assertQ("lazy handler includes highlighting",
-            req("q", "name:Zapp OR title:General", "qt","/lazy"),
-            "//lst[@name='highlighting']"
-            );
+    try (SolrCore core = h.getCore()) {
+      PluginBag.PluginHolder<SolrRequestHandler> handler = core.getRequestHandlers().getRegistry().get("/lazy");
+      assertFalse(handler.isLoaded());
+
+      assertU(adoc("id", "42", "name", "Zapp Brannigan"));
+      assertU(adoc("id", "43", "title", "Democratic Order of Planets"));
+      assertU(adoc("id", "44", "name", "The Zapper"));
+      assertU(adoc("id", "45", "title", "25 star General"));
+      assertU(adoc("id", "46", "subject", "Defeated the pacifists of the Gandhi nebula"));
+      assertU(adoc("id", "47", "text", "line up and fly directly at the enemy death cannons, clogging them with wreckage!"));
+      assertU(commit());
+
+      assertQ("lazy request handler returns all matches", req("q", "id:[42 TO 47]"), "*[count(//doc)=6]");
+
+      // But it should behave just like the 'defaults' request handler above
+      assertQ("lazy handler returns fewer matches", req("q", "id:[42 TO 47]", "qt", "/lazy"), "*[count(//doc)=4]");
+
+      assertQ("lazy handler includes highlighting", req("q", "name:Zapp OR title:General", "qt", "/lazy"), "//lst[@name='highlighting']");
+    }
   }
 
   @Test
   public void testPathNormalization()
   {
-    SolrCore core = h.getCore();
-    SolrRequestHandler h1 = core.getRequestHandler("/update" );
-    assertNotNull( h1 );
-
-    SolrRequestHandler h2 = core.getRequestHandler("/update/" );
-    assertNotNull( h2 );
-    
-    assertEquals( h1, h2 ); // the same object
-    
-    assertNull( core.getRequestHandler("/update/asdgadsgas" ) ); // prefix
+    try (SolrCore core = h.getCore()) {
+      SolrRequestHandler h1 = core.getRequestHandler("/update");
+      assertNotNull(h1);
+
+      SolrRequestHandler h2 = core.getRequestHandler("/update/");
+      assertNotNull(h2);
+
+      assertEquals(h1, h2); // the same object
+
+      assertNull(core.getRequestHandler("/update/asdgadsgas")); // prefix
+    }
   }
 
   @Test
   public void testStatistics() {
-    SolrCore core = h.getCore();
-    SolrRequestHandler updateHandler = core.getRequestHandler("/update");
-    SolrRequestHandler termHandler = core.getRequestHandler("/terms");
+    try (SolrCore core = h.getCore()) {
+      SolrRequestHandler updateHandler = core.getRequestHandler("/update");
+      SolrRequestHandler termHandler = core.getRequestHandler("/terms");
 
-    assertU(adoc("id", "47",
-        "text", "line up and fly directly at the enemy death cannons, clogging them with wreckage!"));
-    assertU(commit());
+      assertU(adoc("id", "47", "text", "line up and fly directly at the enemy death cannons, clogging them with wreckage!"));
+      assertU(commit());
 
-    Map<String,Object> updateStats = updateHandler.getSolrMetricsContext().getMetricsSnapshot();
-    Map<String,Object> termStats = termHandler.getSolrMetricsContext().getMetricsSnapshot();
+      Map<String,Object> updateStats = updateHandler.getSolrMetricsContext().getMetricsSnapshot();
+      Map<String,Object> termStats = termHandler.getSolrMetricsContext().getMetricsSnapshot();
 
-    Long updateTime = (Long) updateStats.get("UPDATE./update.totalTime");
-    Long termTime = (Long) termStats.get("QUERY./terms.totalTime");
+      Long updateTime = (Long) updateStats.get("UPDATE./update.totalTime");
+      Long termTime = (Long) termStats.get("QUERY./terms.totalTime");
 
-    // MRM TODO: these update stat's seemed a bit expensive to be enabled in such tight spots by default
-    // assertFalse("RequestHandlers should not share statistics!", updateTime.equals(termTime));
+      // MRM TODO: these update stat's seemed a bit expensive to be enabled in such tight spots by default
+      // assertFalse("RequestHandlers should not share statistics!", updateTime.equals(termTime));
+    }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/core/SOLR749Test.java b/solr/core/src/test/org/apache/solr/core/SOLR749Test.java
index d802899..9fb120f 100644
--- a/solr/core/src/test/org/apache/solr/core/SOLR749Test.java
+++ b/solr/core/src/test/org/apache/solr/core/SOLR749Test.java
@@ -46,6 +46,7 @@ public class SOLR749Test extends SolrTestCaseJ4 {
     ValueSourceParser vsp = core.getValueSourceParser("boost");
     assertTrue("vsp is null and it shouldn't be", vsp != null);
     assertTrue("vsp is not an instanceof " + DummyValueSourceParser.class, vsp instanceof DummyValueSourceParser);
+    core.close();
   }
 
   public void testHowManyDocsHaveBoostFunctionComputed() throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
index 92cdfa5..7ffd46d 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
@@ -70,6 +70,7 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
     old = core.registerRequestHandler( path, handler2 );
     assertEquals( old, handler1 ); // should pop out the old one
     assertEquals( core.getRequestHandlers().get( path ), handler2 );
+    core.close();
   }
 
   @Test
@@ -116,6 +117,7 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
       ++ihCount; assertEquals(pathToClassMap.get("update"), "solr.UpdateRequestHandlerApi");
     }
     assertEquals("wrong number of implicit handlers", ihCount, implicitHandlers.size());
+    core.close();
   }
 
   @Test
@@ -258,6 +260,7 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
     //try a Req Handler, which are stored by name, not clas
     bean = infoRegistry.get("/select");
     assertNotNull("bean not registered", bean);
+    core.close();
   }
 
   @Test
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 bccc506..236a01f 100644
--- a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
+++ b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
@@ -53,52 +53,59 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
   }
 
   public void testPostingsFormats() {
-    Codec codec = h.getCore().getCodec();
-    Map<String, SchemaField> fields = h.getCore().getLatestSchema().getFields();
-    SchemaField schemaField = fields.get("string_direct_f");
-    PerFieldPostingsFormat format = (PerFieldPostingsFormat) codec.postingsFormat();
-    assertEquals("Direct", format.getPostingsFormatForField(schemaField.getName()).getName());
-    schemaField = fields.get("string_standard_f");
-    assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField(schemaField.getName()).getName());
-    schemaField = fields.get("string_f");
-    assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField(schemaField.getName()).getName());
+    try (SolrCore core = h.getCore()) {
+      Codec codec = core.getCodec();
+      Map<String,SchemaField> fields = core.getLatestSchema().getFields();
+      SchemaField schemaField = fields.get("string_direct_f");
+      PerFieldPostingsFormat format = (PerFieldPostingsFormat) codec.postingsFormat();
+      assertEquals("Direct", format.getPostingsFormatForField(schemaField.getName()).getName());
+      schemaField = fields.get("string_standard_f");
+      assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField(schemaField.getName()).getName());
+      schemaField = fields.get("string_f");
+      assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField(schemaField.getName()).getName());
+    }
   }
 
   public void testDocValuesFormats() {
-    // NOTE: Direct (and Disk) DocValues formats were removed, so we use "Asserting" 
-    // as a way to vet that the configuration actually matters.
-    Codec codec = h.getCore().getCodec();
-    Map<String, SchemaField> fields = h.getCore().getLatestSchema().getFields();
-    SchemaField schemaField = fields.get("string_disk_f");
-    PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
-    assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField(schemaField.getName()).getName());
-    schemaField = fields.get("string_direct_f");
-    assertEquals("Asserting", format.getDocValuesFormatForField(schemaField.getName()).getName());
-    schemaField = fields.get("string_f");
-    assertEquals(TestUtil.getDefaultDocValuesFormat().getName(),
-        format.getDocValuesFormatForField(schemaField.getName()).getName());
+    try (SolrCore core = h.getCore()) {
+      // NOTE: Direct (and Disk) DocValues formats were removed, so we use "Asserting"
+      // as a way to vet that the configuration actually matters.
+      Codec codec = core.getCodec();
+      Map<String,SchemaField> fields = core.getLatestSchema().getFields();
+      SchemaField schemaField = fields.get("string_disk_f");
+      PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
+      assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField(schemaField.getName()).getName());
+      schemaField = fields.get("string_direct_f");
+      assertEquals("Asserting", format.getDocValuesFormatForField(schemaField.getName()).getName());
+      schemaField = fields.get("string_f");
+      assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField(schemaField.getName()).getName());
+    }
   }
 
   public void testDynamicFieldsPostingsFormats() {
-    Codec codec = h.getCore().getCodec();
-    PerFieldPostingsFormat format = (PerFieldPostingsFormat) codec.postingsFormat();
-
-    assertEquals("Direct", format.getPostingsFormatForField("foo_direct").getName());
-    assertEquals("Direct", format.getPostingsFormatForField("bar_direct").getName());
-    assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField("foo_standard").getName());
-    assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField("bar_standard").getName());
+    try (SolrCore core = h.getCore()) {
+      Codec codec = core.getCodec();
+      PerFieldPostingsFormat format = (PerFieldPostingsFormat) codec.postingsFormat();
+
+      assertEquals("Direct", format.getPostingsFormatForField("foo_direct").getName());
+      assertEquals("Direct", format.getPostingsFormatForField("bar_direct").getName());
+      assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField("foo_standard").getName());
+      assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField("bar_standard").getName());
+    }
   }
 
   public void testDynamicFieldsDocValuesFormats() {
     // NOTE: Direct (and Disk) DocValues formats were removed, so we use "Asserting" 
     // as a way to vet that the configuration actually matters.
-    Codec codec = h.getCore().getCodec();
-    PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
-
-    assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("foo_disk").getName());
-    assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("bar_disk").getName());
-    assertEquals("Asserting", format.getDocValuesFormatForField("foo_direct").getName());
-    assertEquals("Asserting", format.getDocValuesFormatForField("bar_direct").getName());
+    try (SolrCore core = h.getCore()) {
+      Codec codec = core.getCodec();
+      PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
+
+      assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("foo_disk").getName());
+      assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("bar_disk").getName());
+      assertEquals("Asserting", format.getDocValuesFormatForField("foo_direct").getName());
+      assertEquals("Asserting", format.getDocValuesFormatForField("bar_direct").getName());
+    }
   }
   
   private void reloadCoreAndRecreateIndex() {
@@ -114,22 +121,21 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
     }
     try {
       reloadCoreAndRecreateIndex();
-      assertCompressionMode(expectedModeString, h.getCore());  
+      try (SolrCore core = h.getCore()) {
+        assertCompressionMode(expectedModeString, core);
+      }
     } finally {
       System.clearProperty("tests.COMPRESSION_MODE");
     }
   }
 
   protected void assertCompressionMode(String expectedModeString, SolrCore core) throws IOException {
-    h.getCore().withSearcher(searcher -> {
+    core.withSearcher(searcher -> {
       SegmentInfos infos = SegmentInfos.readLatestCommit(searcher.getIndexReader().directory());
       SegmentInfo info = infos.info(infos.size() - 1).info;
-      assertEquals("Expecting compression mode string to be " + expectedModeString +
-              " but got: " + info.getAttribute(Lucene50StoredFieldsFormat.MODE_KEY) +
-              "\n SegmentInfo: " + info +
-              "\n SegmentInfos: " + infos +
-              "\n Codec: " + core.getCodec(),
-          expectedModeString, info.getAttribute(Lucene50StoredFieldsFormat.MODE_KEY));
+      assertEquals(
+          "Expecting compression mode string to be " + expectedModeString + " but got: " + info.getAttribute(Lucene50StoredFieldsFormat.MODE_KEY) + "\n SegmentInfo: " + info + "\n SegmentInfos: "
+              + infos + "\n Codec: " + core.getCodec(), expectedModeString, info.getAttribute(Lucene50StoredFieldsFormat.MODE_KEY));
       return null;
     });
   }
diff --git a/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java b/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
index 9c18cbc..d113be6 100644
--- a/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
+++ b/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
@@ -85,18 +85,20 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
   @Test
   public void testShareSchema() throws Exception {
     System.setProperty("shareSchema", "true");
-    SolrResourceLoader loader = h.getCore().getResourceLoader();
-    CoreContainer cores = init(loader, CONFIGSETS_SOLR_XML);
+    try (SolrCore core = h.getCore()) {
+      SolrResourceLoader loader = core.getResourceLoader();
+      CoreContainer cores = init(loader, CONFIGSETS_SOLR_XML);
 
-    try {
-      SolrCore core1 = cores.create("core1", ImmutableMap.of("configSet", "minimal"));
-      SolrCore core2 = cores.create("core2", ImmutableMap.of("configSet", "minimal"));
-      
-      assertSame(core1.getLatestSchema(), core2.getLatestSchema());
+      try {
+        SolrCore core1 = cores.create("core1", ImmutableMap.of("configSet", "minimal"));
+        SolrCore core2 = cores.create("core2", ImmutableMap.of("configSet", "minimal"));
 
-    } finally {
-      cores.shutdown();
-      System.clearProperty("shareSchema");
+        assertSame(core1.getLatestSchema(), core2.getLatestSchema());
+
+      } finally {
+        cores.shutdown();
+        System.clearProperty("shareSchema");
+      }
     }
   }
 
@@ -120,82 +122,85 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
 
   @Test
   public void testReloadThreaded() throws Exception {
-    SolrResourceLoader loader = h.getCore().getResourceLoader();
-    final CoreContainer cc = init(loader, CONFIGSETS_SOLR_XML);
-    SolrCore core = cc.create("core1", ImmutableMap.of("configSet", "minimal"));
+    try (SolrCore c = h.getCore()) {
+      SolrResourceLoader loader = c.getResourceLoader();
+      final CoreContainer cc = init(loader, CONFIGSETS_SOLR_XML);
+      SolrCore core = cc.create("core1", ImmutableMap.of("configSet", "minimal"));
 
-    class TestThread extends Thread {
-      @Override
-      public void run() {
-        cc.reload("core1");
+      class TestThread extends Thread {
+        @Override
+        public void run() {
+          cc.reload("core1");
+        }
       }
-    }
 
-    List<Thread> threads = new ArrayList<>();
-    int numThreads = 4;
-    for (int i = 0; i < numThreads; i++) {
-      threads.add(new TestThread());
-    }
+      List<Thread> threads = new ArrayList<>();
+      int numThreads = 4;
+      for (int i = 0; i < numThreads; i++) {
+        threads.add(new TestThread());
+      }
 
-    for (Thread thread : threads) {
-      thread.start();
-    }
+      for (Thread thread : threads) {
+        thread.start();
+      }
 
-    for (Thread thread : threads) {
-      thread.join();
-    }
+      for (Thread thread : threads) {
+        thread.join();
+      }
 
-    cc.shutdown();
-    core.closeAndWait();
+      cc.shutdown();
+    }
   }
 
   @Test
   public void testNoCores() throws Exception {
-    SolrResourceLoader loader = h.getCore().getResourceLoader();
-    CoreContainer cores = init(loader, CONFIGSETS_SOLR_XML);
+    try (SolrCore core = h.getCore()) {
+      SolrResourceLoader loader = core.getResourceLoader();
+      CoreContainer cores = init(loader, CONFIGSETS_SOLR_XML);
 
-    try {
-      //assert zero cores
-      assertEquals("There should not be cores", 0, cores.getCores().size());
-      
-      //add a new core
-      cores.create("core1", ImmutableMap.of("configSet", "minimal"));
+      try {
+        //assert zero cores
+        assertEquals("There should not be cores", 0, cores.getCores().size());
 
-      //assert one registered core
+        //add a new core
+        cores.create("core1", ImmutableMap.of("configSet", "minimal"));
 
-      assertEquals("There core registered", 1, cores.getCores().size());
+        //assert one registered core
 
-      cores.unload("core1");
-      //assert cero cores
-      assertEquals("There should not be cores", 0, cores.getCores().size());
-      
-      // try and remove a core that does not exist
-      SolrException thrown = expectThrows(SolrException.class, () -> {
-        cores.unload("non_existent_core");
-      });
-      assertThat(thrown.getMessage(), containsString("Cannot unload non-existent core [non_existent_core]"));
+        assertEquals("There core registered", 1, cores.getCores().size());
 
+        cores.unload("core1");
+        //assert cero cores
+        assertEquals("There should not be cores", 0, cores.getCores().size());
 
-      // try and remove a null core
-      thrown = expectThrows(SolrException.class, () -> {
-        cores.unload(null);
-      });
-      assertThat(thrown.getMessage(), containsString("Cannot unload non-existent core [null]"));
-    } finally {
-      cores.shutdown();
+        // try and remove a core that does not exist
+        SolrException thrown = expectThrows(SolrException.class, () -> {
+          cores.unload("non_existent_core");
+        });
+        assertThat(thrown.getMessage(), containsString("Cannot unload non-existent core [non_existent_core]"));
+
+        // try and remove a null core
+        thrown = expectThrows(SolrException.class, () -> {
+          cores.unload(null);
+        });
+        assertThat(thrown.getMessage(), containsString("Cannot unload non-existent core [null]"));
+      } finally {
+        cores.shutdown();
+      }
     }
 
   }
 
   @Test
   public void testLogWatcherEnabledByDefault() throws Exception {
-    SolrResourceLoader loader = h.getCore().getResourceLoader();
-    CoreContainer cc = init(loader, "<solr></solr>");
-    try {
-      assertNotNull(cc.getLogging());
-    }
-    finally {
-      cc.shutdown();
+    try (SolrCore core = h.getCore()) {
+      SolrResourceLoader loader = core.getResourceLoader();
+      CoreContainer cc = init(loader, "<solr></solr>");
+      try {
+        assertNotNull(cc.getLogging());
+      } finally {
+        cc.shutdown();
+      }
     }
   }
 
@@ -233,19 +238,21 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
 
   @Test
   public void testClassLoaderHierarchy() throws Exception {
-    SolrResourceLoader loader = h.getCore().getResourceLoader();
-    final CoreContainer cc = init(loader, CONFIGSETS_SOLR_XML);
-    try {
-      ClassLoader sharedLoader = cc.loader.getClassLoader();
-      ClassLoader baseLoader = SolrResourceLoader.class.getClassLoader();
-      assertSame(baseLoader, sharedLoader.getParent());
+    try (SolrCore core = h.getCore()) {
+      SolrResourceLoader loader = core.getResourceLoader();
+      final CoreContainer cc = init(loader, CONFIGSETS_SOLR_XML);
+      try {
+        ClassLoader sharedLoader = cc.loader.getClassLoader();
+        ClassLoader baseLoader = SolrResourceLoader.class.getClassLoader();
+        assertSame(baseLoader, sharedLoader.getParent());
 
-      SolrCore core1 = cc.create("core1", ImmutableMap.of("configSet", "minimal"));
-      ClassLoader coreLoader = core1.getResourceLoader().getClassLoader();
-      assertSame(sharedLoader, coreLoader.getParent());
+        SolrCore core1 = cc.create("core1", ImmutableMap.of("configSet", "minimal"));
+        ClassLoader coreLoader = core1.getResourceLoader().getClassLoader();
+        assertSame(sharedLoader, coreLoader.getParent());
 
-    } finally {
-      cc.shutdown();
+      } finally {
+        cc.shutdown();
+      }
     }
   }
 
@@ -276,12 +283,15 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
       jar3.putNextEntry(new JarEntry("jar3File"));
       jar3.closeEntry();
     }
-    SolrResourceLoader loader = h.getCore().getResourceLoader();
-    final CoreContainer cc1 = init(loader, tmpRoot, "<solr></solr>");
-    try {
-      cc1.loader.openResource("defaultSharedLibFile").close();
-    } finally {
-      cc1.shutdown();
+    SolrResourceLoader loader;
+    try (SolrCore core = h.getCore()) {
+      loader = core.getResourceLoader();
+      final CoreContainer cc1 = init(loader, tmpRoot, "<solr></solr>");
+      try {
+        cc1.loader.openResource("defaultSharedLibFile").close();
+      } finally {
+        cc1.shutdown();
+      }
     }
 
     // Explicitly declaring 'lib' makes no change compared to the default
@@ -352,17 +362,17 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
 
   @Test
   public void testCustomHandlers() throws Exception {
-    SolrResourceLoader loader = h.getCore().getResourceLoader();
-    CoreContainer cc = init(loader, CUSTOM_HANDLERS_SOLR_XML);
-    try {
-      assertThat(cc.getCollectionsHandler(), is(instanceOf(CustomCollectionsHandler.class)));
-      assertThat(cc.getInfoHandler(), is(instanceOf(CustomInfoHandler.class)));
-      assertThat(cc.getMultiCoreHandler(), is(instanceOf(CustomCoreAdminHandler.class)));
-    }
-    finally {
-      cc.shutdown();
+    try (SolrCore core = h.getCore()) {
+      SolrResourceLoader loader = core.getResourceLoader();
+      CoreContainer cc = init(loader, CUSTOM_HANDLERS_SOLR_XML);
+      try {
+        assertThat(cc.getCollectionsHandler(), is(instanceOf(CustomCollectionsHandler.class)));
+        assertThat(cc.getInfoHandler(), is(instanceOf(CustomInfoHandler.class)));
+        assertThat(cc.getMultiCoreHandler(), is(instanceOf(CustomCoreAdminHandler.class)));
+      } finally {
+        cc.shutdown();
+      }
     }
-
   }
 
   private static class MockCoresLocator implements CoresLocator {
@@ -414,54 +424,53 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
 
     // ----
     // init the CoreContainer
-    SolrResourceLoader loader = h.getCore().getResourceLoader();
-    CoreContainer cc = init(loader, CONFIGSETS_SOLR_XML);
-
-    // check that we have the cores we expect
-    cores = cc.getLoadedCoreNames();
-    assertNotNull("core names is null", cores);
-    assertEquals("wrong number of cores", 0, cores.size());
-
-    // check that we have the failures we expect
-    failures = cc.getCoreInitFailures();
-    assertNotNull("core failures is a null map", failures);
-    assertEquals("wrong number of core failures", 0, failures.size());
-
-    // -----
-    // try to add a collection with a configset that doesn't exist
-    ignoreException(Pattern.quote("bogus_path"));
-    SolrException thrown = expectThrows(SolrException.class, () -> {
-      cc.create("bogus", ImmutableMap.of("configSet", "bogus_path"));
-    });
-    Throwable rootCause = Throwables.getRootCause(thrown);
-    assertTrue("init exception doesn't mention bogus dir: " + rootCause.getMessage(),
-        0 < rootCause.getMessage().indexOf("bogus_path"));
-
-    // check that we have the cores we expect
-    cores = cc.getLoadedCoreNames();
-    assertNotNull("core names is null", cores);
-    assertEquals("wrong number of cores", 0, cores.size());
+    CoreContainer cc;
+    try (SolrCore core = h.getCore()) {
+      SolrResourceLoader loader = core.getResourceLoader();
+      cc = init(loader, CONFIGSETS_SOLR_XML);
 
-    // check that we have the failures we expect
-    failures = cc.getCoreInitFailures();
-    assertNotNull("core failures is a null map", failures);
-    assertEquals("wrong number of core failures", 1, failures.size());
-    fail = failures.get("bogus").exception;
-    assertNotNull("null failure for test core", fail);
-    assertTrue("init failure doesn't mention problem: " + fail.getMessage(),
-        0 < fail.getMessage().indexOf("bogus_path"));
+      // check that we have the cores we expect
+      cores = cc.getLoadedCoreNames();
+      assertNotNull("core names is null", cores);
+      assertEquals("wrong number of cores", 0, cores.size());
 
-    // check that we get null accessing a non-existent core
-    assertNull(cc.getCore("does_not_exist"));
-    // check that we get a 500 accessing the core with an init failure
-    thrown = expectThrows(SolrException.class, () -> {
-      SolrCore c = cc.getCore("bogus");
-    });
-    assertEquals(500, thrown.code());
-    String cause = Throwables.getRootCause(thrown).getMessage();
-    assertTrue("getCore() ex cause doesn't mention init fail: " + cause,
-        0 < cause.indexOf("bogus_path"));
+      // check that we have the failures we expect
+      failures = cc.getCoreInitFailures();
+      assertNotNull("core failures is a null map", failures);
+      assertEquals("wrong number of core failures", 0, failures.size());
 
+      // -----
+      // try to add a collection with a configset that doesn't exist
+      ignoreException(Pattern.quote("bogus_path"));
+      SolrException thrown = expectThrows(SolrException.class, () -> {
+        cc.create("bogus", ImmutableMap.of("configSet", "bogus_path"));
+      });
+      Throwable rootCause = Throwables.getRootCause(thrown);
+      assertTrue("init exception doesn't mention bogus dir: " + rootCause.getMessage(), 0 < rootCause.getMessage().indexOf("bogus_path"));
+
+      // check that we have the cores we expect
+      cores = cc.getLoadedCoreNames();
+      assertNotNull("core names is null", cores);
+      assertEquals("wrong number of cores", 0, cores.size());
+
+      // check that we have the failures we expect
+      failures = cc.getCoreInitFailures();
+      assertNotNull("core failures is a null map", failures);
+      assertEquals("wrong number of core failures", 1, failures.size());
+      fail = failures.get("bogus").exception;
+      assertNotNull("null failure for test core", fail);
+      assertTrue("init failure doesn't mention problem: " + fail.getMessage(), 0 < fail.getMessage().indexOf("bogus_path"));
+
+      // check that we get null accessing a non-existent core
+      assertNull(cc.getCore("does_not_exist"));
+      // check that we get a 500 accessing the core with an init failure
+      thrown = expectThrows(SolrException.class, () -> {
+        SolrCore c = cc.getCore("bogus");
+      });
+      assertEquals(500, thrown.code());
+      String cause = Throwables.getRootCause(thrown).getMessage();
+      assertTrue("getCore() ex cause doesn't mention init fail: " + cause, 0 < cause.indexOf("bogus_path"));
+    }
     cc.shutdown();
   }
 
diff --git a/solr/core/src/test/org/apache/solr/core/TestInfoStreamLogging.java b/solr/core/src/test/org/apache/solr/core/TestInfoStreamLogging.java
index c448583..bede66c 100644
--- a/solr/core/src/test/org/apache/solr/core/TestInfoStreamLogging.java
+++ b/solr/core/src/test/org/apache/solr/core/TestInfoStreamLogging.java
@@ -29,8 +29,10 @@ public class TestInfoStreamLogging extends SolrTestCaseJ4 {
   }
   
   public void testIndexConfig() throws Exception {
-    IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
+    try (SolrCore core = h.getCore()) {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
 
-    assertTrue(iwc.getInfoStream() instanceof LoggingInfoStream);
+      assertTrue(iwc.getInfoStream() instanceof LoggingInfoStream);
+    }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/core/TestInitParams.java b/solr/core/src/test/org/apache/solr/core/TestInitParams.java
index 1a9ce57..219c683 100644
--- a/solr/core/src/test/org/apache/solr/core/TestInitParams.java
+++ b/solr/core/src/test/org/apache/solr/core/TestInitParams.java
@@ -38,96 +38,100 @@ public class TestInitParams extends SolrTestCaseJ4 {
   }
   @Test
   public void testComponentWithInitParams(){
+    try (SolrCore core = h.getCore()) {
+      for (String s : Arrays.asList("/dump1", "/dump3", "/root/dump5", "/root1/anotherlevel/dump6")) {
+        SolrRequestHandler handler = core.getRequestHandler(s);
+        SolrQueryResponse rsp = new SolrQueryResponse();
+        SolrQueryRequest req = req("initArgs", "true");
+        handler.handleRequest(req, rsp);
+        NamedList nl = (NamedList) rsp.getValues().get("initArgs");
+        NamedList def = (NamedList) nl.get(PluginInfo.DEFAULTS);
+        assertEquals("A", def.get("a"));
+        def = (NamedList) nl.get(PluginInfo.INVARIANTS);
+        assertEquals("B", def.get("b"));
+        def = (NamedList) nl.get(PluginInfo.APPENDS);
+        assertEquals("C", def.get("c"));
+        IOUtils.closeQuietly(req);
+      }
 
-    for (String s : Arrays.asList("/dump1", "/dump3","/root/dump5" , "/root1/anotherlevel/dump6")) {
-      SolrRequestHandler handler = h.getCore().getRequestHandler(s);
+      InitParams initParams = core.getSolrConfig().getInitParams().get("a");
+
+      PluginInfo pluginInfo = new PluginInfo("requestHandler", new HashMap<String,String>(), new NamedList<>(singletonMap("defaults", new NamedList(Utils.makeMap("a", "A1")))), null);
+      initParams.apply(pluginInfo);
+      assertEquals("A", initParams.defaults.get("a"));
+    }
+  }
+
+  @Test
+  public void testMultiInitParams(){
+    try (SolrCore core = h.getCore()) {
+      SolrRequestHandler handler = core.getRequestHandler("/dump6");
       SolrQueryResponse rsp = new SolrQueryResponse();
       SolrQueryRequest req = req("initArgs", "true");
       handler.handleRequest(req, rsp);
-      req.close();
       NamedList nl = (NamedList) rsp.getValues().get("initArgs");
       NamedList def = (NamedList) nl.get(PluginInfo.DEFAULTS);
       assertEquals("A", def.get("a"));
+      assertEquals("P", def.get("p"));
       def = (NamedList) nl.get(PluginInfo.INVARIANTS);
       assertEquals("B", def.get("b"));
       def = (NamedList) nl.get(PluginInfo.APPENDS);
       assertEquals("C", def.get("c"));
       IOUtils.closeQuietly(req);
     }
-
-    InitParams initParams = h.getCore().getSolrConfig().getInitParams().get("a");
-
-    PluginInfo pluginInfo = new PluginInfo("requestHandler",
-        new HashMap<String, String>(),
-        new NamedList<>(singletonMap("defaults", new NamedList(Utils.makeMap("a", "A1")))), null);
-    initParams.apply(pluginInfo);
-    assertEquals( "A",initParams.defaults.get("a"));
-  }
-
-  @Test
-  public void testMultiInitParams(){
-    SolrRequestHandler handler = h.getCore().getRequestHandler("/dump6");
-    SolrQueryResponse rsp = new SolrQueryResponse();
-    SolrQueryRequest req = req("initArgs", "true");
-    handler.handleRequest(req, rsp);
-    req.close();
-    NamedList nl = (NamedList) rsp.getValues().get("initArgs");
-    NamedList def = (NamedList) nl.get(PluginInfo.DEFAULTS);
-    assertEquals("A", def.get("a"));
-    assertEquals("P", def.get("p"));
-    def = (NamedList) nl.get(PluginInfo.INVARIANTS);
-    assertEquals("B", def.get("b"));
-    def = (NamedList) nl.get(PluginInfo.APPENDS);
-    assertEquals("C", def.get("c"));
-    IOUtils.closeQuietly(req);
   }
 
 
   @Test
   public void testComponentWithConflictingInitParams(){
-    SolrRequestHandler handler = h.getCore().getRequestHandler("/dump2");
-    SolrQueryResponse rsp = new SolrQueryResponse();
-    SolrQueryRequest req = req("initArgs", "true");
-    handler.handleRequest(req, rsp);
-    req.close();
-    NamedList nl = (NamedList) rsp.getValues().get("initArgs");
-    NamedList def = (NamedList) nl.get(PluginInfo.DEFAULTS);
-    assertEquals("A1" ,def.get("a"));
-    def = (NamedList) nl.get(PluginInfo.INVARIANTS);
-    assertEquals("B1" ,def.get("b"));
-    def = (NamedList) nl.get(PluginInfo.APPENDS);
-    assertEquals(Arrays.asList("C1","C") ,def.getAll("c"));
-    IOUtils.closeQuietly(req);
+    try (SolrCore core = h.getCore()) {
+      SolrRequestHandler handler = core.getRequestHandler("/dump2");
+      SolrQueryResponse rsp = new SolrQueryResponse();
+      SolrQueryRequest req = req("initArgs", "true");
+      handler.handleRequest(req, rsp);
+      NamedList nl = (NamedList) rsp.getValues().get("initArgs");
+      NamedList def = (NamedList) nl.get(PluginInfo.DEFAULTS);
+      assertEquals("A1", def.get("a"));
+      def = (NamedList) nl.get(PluginInfo.INVARIANTS);
+      assertEquals("B1", def.get("b"));
+      def = (NamedList) nl.get(PluginInfo.APPENDS);
+      assertEquals(Arrays.asList("C1", "C"), def.getAll("c"));
+      IOUtils.closeQuietly(req);
+    }
   }
 
   public void testNestedRequestHandler() {
-    assertNotNull(h.getCore().getRequestHandler("/greedypath"));
-    assertNotNull(h.getCore().getRequestHandler("/greedypath/some/path"));
-    assertNotNull( h.getCore().getRequestHandler("/greedypath/some/other/path"));
-    assertNull(h.getCore().getRequestHandler("/greedypath/unknownpath"));
+    try (SolrCore core = h.getCore()) {
+      assertNotNull(core.getRequestHandler("/greedypath"));
+      assertNotNull(core.getRequestHandler("/greedypath/some/path"));
+      assertNotNull(core.getRequestHandler("/greedypath/some/other/path"));
+      assertNull(core.getRequestHandler("/greedypath/unknownpath"));
+    }
   }
 
-  public void testElevateExample(){
-    SolrRequestHandler handler = h.getCore().getRequestHandler("/elevate");
-    SolrQueryResponse rsp = new SolrQueryResponse();
-    SolrQueryRequest req = req("initArgs", "true");
-    handler.handleRequest(req, rsp);
-    req.close();
-    NamedList nl = (NamedList) rsp.getValues().get("initArgs");
-    NamedList def = (NamedList) nl.get(PluginInfo.DEFAULTS);
-    assertEquals("text" ,def.get("df"));
-    IOUtils.closeQuietly(req);
+  public void testElevateExample() {
+    try (SolrCore core = h.getCore()) {
+      SolrRequestHandler handler = core.getRequestHandler("/elevate");
+      SolrQueryResponse rsp = new SolrQueryResponse();
+      SolrQueryRequest req = req("initArgs", "true");
+      handler.handleRequest(req, rsp);
+      NamedList nl = (NamedList) rsp.getValues().get("initArgs");
+      NamedList def = (NamedList) nl.get(PluginInfo.DEFAULTS);
+      assertEquals("text", def.get("df"));
+      IOUtils.closeQuietly(req);
+    }
   }
 
   public void testArbitraryAttributes() {
-    SolrRequestHandler handler = h.getCore().getRequestHandler("/dump7");
-    SolrQueryResponse rsp = new SolrQueryResponse();
-    SolrQueryRequest req = req("initArgs", "true");
-    handler.handleRequest(req, rsp);
-    req.close();
-    NamedList nl = (NamedList) rsp.getValues().get("initArgs");
-    assertEquals("server-enabled.txt", nl.get("healthcheckFile"));
-    IOUtils.closeQuietly(req);
+    try (SolrCore core = h.getCore()) {
+      SolrRequestHandler handler = core.getRequestHandler("/dump7");
+      SolrQueryResponse rsp = new SolrQueryResponse();
+      SolrQueryRequest req = req("initArgs", "true");
+      handler.handleRequest(req, rsp);
+      NamedList nl = (NamedList) rsp.getValues().get("initArgs");
+      assertEquals("server-enabled.txt", nl.get("healthcheckFile"));
+      IOUtils.closeQuietly(req);
+    }
   }
 
   public void testMatchPath(){
diff --git a/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java b/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java
index b3f5b2e..b67aabf 100644
--- a/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java
+++ b/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java
@@ -71,17 +71,20 @@ public class TestJmxIntegration extends SolrTestCaseJ4 {
 
     initCore("solrconfig.xml", "schema.xml");
 
-    // we should be able to see that the core has JmxIntegration enabled
-    registryName = h.getCore().getCoreMetricManager().getRegistryName();
-    SolrMetricManager manager = h.getCoreContainer().getMetricManager();
-    Map<String,SolrMetricReporter> reporters = manager.getReporters(registryName);
-    assertEquals(1, reporters.size());
-    SolrMetricReporter reporter = reporters.values().iterator().next();
-    assertTrue(reporter instanceof SolrJmxReporter);
-    SolrJmxReporter jmx = (SolrJmxReporter)reporter;
-    assertTrue("JMX not enabled", jmx.isActive());
-    // and we should be able to see that the reporter
-    // refers to the JMX server we started
+    SolrJmxReporter jmx;
+    try (SolrCore core = h.getCore()) {
+      // we should be able to see that the core has JmxIntegration enabled
+      registryName = core.getCoreMetricManager().getRegistryName();
+      SolrMetricManager manager = h.getCoreContainer().getMetricManager();
+      Map<String,SolrMetricReporter> reporters = manager.getReporters(registryName);
+      assertEquals(1, reporters.size());
+      SolrMetricReporter reporter = reporters.values().iterator().next();
+      assertTrue(reporter instanceof SolrJmxReporter);
+      jmx = (SolrJmxReporter) reporter;
+      assertTrue("JMX not enabled", jmx.isActive());
+      // and we should be able to see that the reporter
+      // refers to the JMX server we started
+    }
 
     mbeanServer = jmx.getMBeanServer();
 
@@ -149,31 +152,31 @@ public class TestJmxIntegration extends SolrTestCaseJ4 {
 
   @Test
   public void testJmxUpdate() throws Exception {
+    try (SolrCore core = h.getCore()) {
+      SolrInfoBean bean = null;
+      // wait until searcher is registered
+      for (int i = 0; i < 100; i++) {
+        bean = core.getInfoRegistry().get("searcher");
+        if (bean != null) break;
+        Thread.sleep(250);
+      }
+      if (bean == null) throw new RuntimeException("searcher was never registered");
+      ObjectName searcher = nameFactory.createName("gauge", registryName, "SEARCHER.searcher.*");
 
-    SolrInfoBean bean = null;
-    // wait until searcher is registered
-    for (int i=0; i<100; i++) {
-      bean = h.getCore().getInfoRegistry().get("searcher");
-      if (bean != null) break;
-      Thread.sleep(250);
-    }
-    if (bean==null) throw new RuntimeException("searcher was never registered");
-    ObjectName searcher = nameFactory.createName("gauge", registryName, "SEARCHER.searcher.*");
-
-    if (log.isInfoEnabled()) {
-      log.info("Mbeans in server: {}", mbeanServer.queryNames(null, null));
-    }
+      if (log.isInfoEnabled()) {
+        log.info("Mbeans in server: {}", mbeanServer.queryNames(null, null));
+      }
 
-    Set<ObjectInstance> objects = mbeanServer.queryMBeans(searcher, null);
-    assertFalse("No mbean found for SolrIndexSearcher", mbeanServer.queryMBeans(searcher, null).isEmpty());
+      Set<ObjectInstance> objects = mbeanServer.queryMBeans(searcher, null);
+      assertFalse("No mbean found for SolrIndexSearcher", mbeanServer.queryMBeans(searcher, null).isEmpty());
 
-    ObjectName name = nameFactory.createName("gauge", registryName, "SEARCHER.searcher.numDocs");
-    int oldNumDocs =  (Integer)mbeanServer.getAttribute(name, "Value");
-    assertU(adoc("id", "1"));
-    assertU("commit", commit());
-    int numDocs = (Integer)mbeanServer.getAttribute(name, "Value");
-    assertTrue("New numDocs is same as old numDocs as reported by JMX",
-        numDocs > oldNumDocs);
+      ObjectName name = nameFactory.createName("gauge", registryName, "SEARCHER.searcher.numDocs");
+      int oldNumDocs = (Integer) mbeanServer.getAttribute(name, "Value");
+      assertU(adoc("id", "1"));
+      assertU("commit", commit());
+      int numDocs = (Integer) mbeanServer.getAttribute(name, "Value");
+      assertTrue("New numDocs is same as old numDocs as reported by JMX", numDocs > oldNumDocs);
+    }
   }
 
   @Test
@@ -182,21 +185,27 @@ public class TestJmxIntegration extends SolrTestCaseJ4 {
     // make sure searcher beans are registered
     assertQ(req("q", "*:*"), "//result[@numFound='0']");
 
-    SolrMetricManager mgr = h.getCoreContainer().getMetricManager();
-    String registryName = h.getCore().getCoreMetricManager().getRegistryName();
-    String coreName = h.getCore().getName();
-    String coreHashCode = Integer.toHexString(h.getCore().hashCode());
-    Map<String, SolrMetricReporter> reporters = mgr.getReporters(registryName);
-    // take first JMX reporter
+    SolrMetricManager mgr;
+    Object tag;
+    Map<String,SolrMetricReporter> reporters;
     SolrJmxReporter reporter = null;
-    for (Map.Entry<String, SolrMetricReporter> e : reporters.entrySet()) {
-      if (e.getKey().endsWith(coreHashCode) && e.getValue() instanceof SolrJmxReporter) {
-        reporter = (SolrJmxReporter)e.getValue();
-        break;
+    String coreHashCode;
+    try (SolrCore core = h.getCore()) {
+      mgr = h.getCoreContainer().getMetricManager();
+      String registryName = core.getCoreMetricManager().getRegistryName();
+      String coreName = core.getName();
+      coreHashCode = Integer.toHexString(core.hashCode());
+      reporters = mgr.getReporters(registryName);
+      // take first JMX reporter
+      for (Map.Entry<String,SolrMetricReporter> e : reporters.entrySet()) {
+        if (e.getKey().endsWith(coreHashCode) && e.getValue() instanceof SolrJmxReporter) {
+          reporter = (SolrJmxReporter) e.getValue();
+          break;
+        }
       }
+      assertNotNull("could not find JMX reporter for " + registryName, reporter);
+      tag = reporter.getInstanceTag();
     }
-    assertNotNull("could not find JMX reporter for " + registryName, reporter);
-    String tag = reporter.getInstanceTag();
 
     Set<ObjectInstance> oldBeans = mbeanServer.queryMBeans(null, null);
     int oldNumberOfObjects = 0;
@@ -217,7 +226,9 @@ public class TestJmxIntegration extends SolrTestCaseJ4 {
     assertQ(req("q", "*:*"), "//result[@numFound='0']");
 
     reporters = mgr.getReporters(registryName);
-    coreHashCode = Integer.toHexString(h.getCore().hashCode());
+    try (SolrCore core = h.getCore()) {
+      coreHashCode = Integer.toHexString(core.hashCode());
+    }
     // take first JMX reporter
     reporter = null;
     for (Map.Entry<String, SolrMetricReporter> e : reporters.entrySet()) {
diff --git a/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java b/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java
index 42b269b..b8e4e10 100644
--- a/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java
@@ -52,46 +52,48 @@ public class TestMergePolicyConfig extends SolrTestCaseJ4 {
   public void testSetNoCFSMergePolicyConfig() throws Exception {
     final boolean useCompoundFile = random().nextBoolean();
     System.setProperty("testSetNoCFSMergePolicyConfig.useCompoundFile", String.valueOf(useCompoundFile));
-    try {
+
       initCore("solrconfig-mergepolicyfactory-nocfs.xml","schema-minimal.xml");
-      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
+    try (SolrCore core = h.getCore())  {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
       assertEquals(useCompoundFile, iwc.getUseCompoundFile());
 
       TieredMergePolicy tieredMP = assertAndCast(TieredMergePolicy.class,
                                                  iwc.getMergePolicy());
       assertEquals(0.5D, tieredMP.getNoCFSRatio(), 0.0D);
-    } finally {
-      System.getProperties().remove("testSetNoCFSMergePolicyConfig.useCompoundFile");
     }
   }
 
   public void testDefaultMergePolicyConfig() throws Exception {
     initCore("solrconfig-mergepolicy-defaults.xml","schema-minimal.xml");
-    IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
-    assertEquals(false, iwc.getUseCompoundFile());
+    try (SolrCore core = h.getCore()) {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
+      assertEquals(false, iwc.getUseCompoundFile());
 
-    TieredMergePolicy tieredMP = assertAndCast(TieredMergePolicy.class,
-                                               iwc.getMergePolicy());
-    assertEquals(TieredMergePolicy.DEFAULT_NO_CFS_RATIO, tieredMP.getNoCFSRatio(), 0.0D);
+      TieredMergePolicy tieredMP = assertAndCast(TieredMergePolicy.class, iwc.getMergePolicy());
+      assertEquals(TieredMergePolicy.DEFAULT_NO_CFS_RATIO, tieredMP.getNoCFSRatio(), 0.0D);
 
-    assertCommitSomeNewDocs();
-    assertCompoundSegments(h.getCore(), false);
+      assertCommitSomeNewDocs();
+      assertCompoundSegments(core, false);
+    }
   }
 
   public void testLegacyMergePolicyConfig() throws Exception {
     final boolean expectCFS = Boolean.parseBoolean(System.getProperty("useCompoundFile"));
 
-    initCore("solrconfig-mergepolicy-legacy.xml","schema-minimal.xml");
-    IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
-    assertEquals(expectCFS, iwc.getUseCompoundFile());
+    initCore("solrconfig-mergepolicy-legacy.xml", "schema-minimal.xml");
+    try (SolrCore core = h.getCore()) {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
+      assertEquals(expectCFS, iwc.getUseCompoundFile());
 
-    TieredMergePolicy tieredMP = assertAndCast(TieredMergePolicy.class, iwc.getMergePolicy());
+      TieredMergePolicy tieredMP = assertAndCast(TieredMergePolicy.class, iwc.getMergePolicy());
 
-    assertEquals(10, tieredMP.getMaxMergeAtOnce());
-    assertEquals(10.0D, tieredMP.getSegmentsPerTier(), 0.0D);
+      assertEquals(10, tieredMP.getMaxMergeAtOnce());
+      assertEquals(10.0D, tieredMP.getSegmentsPerTier(), 0.0D);
 
-    assertCommitSomeNewDocs();
-    assertCompoundSegments(h.getCore(), expectCFS);
+      assertCommitSomeNewDocs();
+      assertCompoundSegments(core, expectCFS);
+    }
   }
   
   public void testTieredMergePolicyConfig() throws Exception {
@@ -99,66 +101,70 @@ public class TestMergePolicyConfig extends SolrTestCaseJ4 {
       = Boolean.parseBoolean(System.getProperty("useCompoundFile"));
 
     initCore("solrconfig-tieredmergepolicyfactory.xml","schema-minimal.xml");
-    IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
-    assertEquals(expectCFS, iwc.getUseCompoundFile());
-
-
-    TieredMergePolicy tieredMP = assertAndCast(TieredMergePolicy.class,
-                                               iwc.getMergePolicy());
-
-    // set by legacy <mergeFactor> setting
-    assertEquals(7, tieredMP.getMaxMergeAtOnce());
-    
-    // mp-specific setters
-    assertEquals(19, tieredMP.getMaxMergeAtOnceExplicit());
-    assertEquals(0.1D, tieredMP.getNoCFSRatio(), 0.0D);
-    // make sure we overrode segmentsPerTier 
-    // (split from maxMergeAtOnce out of mergeFactor)
-    assertEquals(9D, tieredMP.getSegmentsPerTier(), 0.001);
-    
-    assertCommitSomeNewDocs();
-    // even though we have a single segment (which is 100% of the size of 
-    // the index which is higher then our 0.6D threshold) the
-    // compound ratio doesn't matter because the segment was never merged
-    assertCompoundSegments(h.getCore(), expectCFS);
-
-    assertCommitSomeNewDocs();
-    assertNumSegments(h.getCore(), 2);
-    assertCompoundSegments(h.getCore(), expectCFS);
-
-    assertU(optimize("maxSegments", "1"));
-    assertNumSegments(h.getCore(), 1);
-    // we've now forced a merge, and the MP ratio should be in play
-    assertCompoundSegments(h.getCore(), false);
+    try (SolrCore core = h.getCore()) {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
+      assertEquals(expectCFS, iwc.getUseCompoundFile());
+
+      TieredMergePolicy tieredMP = assertAndCast(TieredMergePolicy.class, iwc.getMergePolicy());
+
+      // set by legacy <mergeFactor> setting
+      assertEquals(7, tieredMP.getMaxMergeAtOnce());
+
+      // mp-specific setters
+      assertEquals(19, tieredMP.getMaxMergeAtOnceExplicit());
+      assertEquals(0.1D, tieredMP.getNoCFSRatio(), 0.0D);
+      // make sure we overrode segmentsPerTier
+      // (split from maxMergeAtOnce out of mergeFactor)
+      assertEquals(9D, tieredMP.getSegmentsPerTier(), 0.001);
+
+      assertCommitSomeNewDocs();
+      // even though we have a single segment (which is 100% of the size of
+      // the index which is higher then our 0.6D threshold) the
+      // compound ratio doesn't matter because the segment was never merged
+      assertCompoundSegments(core, expectCFS);
+
+      assertCommitSomeNewDocs();
+      assertNumSegments(core, 2);
+      assertCompoundSegments(core, expectCFS);
+
+      assertU(optimize("maxSegments", "1"));
+      assertNumSegments(core, 1);
+      // we've now forced a merge, and the MP ratio should be in play
+      assertCompoundSegments(core, false);
+    }
   }
 
   public void testNoMergePolicyFactoryConfig() throws Exception {
-    initCore("solrconfig-nomergepolicyfactory.xml","schema-minimal.xml");
-    IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
-    NoMergePolicy mergePolicy = assertAndCast(NoMergePolicy.class,
-        iwc.getMergePolicy());
+    initCore("solrconfig-nomergepolicyfactory.xml", "schema-minimal.xml");
+    NoMergePolicy mergePolicy;
+    try (SolrCore core = h.getCore()) {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
+      mergePolicy = assertAndCast(NoMergePolicy.class, iwc.getMergePolicy());
 
-    assertCommitSomeNewDocs();
+      assertCommitSomeNewDocs();
 
-    assertCommitSomeNewDocs();
-    assertNumSegments(h.getCore(), 2);
+      assertCommitSomeNewDocs();
+      assertNumSegments(core, 2);
 
-    assertU(optimize());
-    assertNumSegments(h.getCore(), 2);
+      assertU(optimize());
+      assertNumSegments(core, 2);
+    }
     deleteCore();
-    initCore("solrconfig-nomergepolicyfactory.xml","schema-minimal.xml");
-    iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
-    assertEquals(mergePolicy, iwc.getMergePolicy());
-
-    UpdateHandler updater = h.getCore().getUpdateHandler();
-    SolrQueryRequest req = req();
-    CommitUpdateCommand cmtCmd = new CommitUpdateCommand(req, true);
-    req.close();
-    cmtCmd.maxOptimizeSegments = -1;
-    expectThrows(IllegalArgumentException.class, () -> {
-      updater.commit(cmtCmd);
-    });
 
+    initCore("solrconfig-nomergepolicyfactory.xml", "schema-minimal.xml");
+    try (SolrCore core = h.getCore()) {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
+      assertEquals(mergePolicy, iwc.getMergePolicy());
+
+      UpdateHandler updater = core.getUpdateHandler();
+      SolrQueryRequest req = req();
+      CommitUpdateCommand cmtCmd = new CommitUpdateCommand(req, true);
+      req.close();
+      cmtCmd.maxOptimizeSegments = -1;
+      expectThrows(IllegalArgumentException.class, () -> {
+        updater.commit(cmtCmd);
+      });
+    }
   }
 
   public void testLogMergePolicyFactoryConfig() throws Exception {
@@ -178,18 +184,19 @@ public class TestMergePolicyConfig extends SolrTestCaseJ4 {
       Class<? extends LogMergePolicy> mpClass) throws Exception {
 
     initCore(solrConfigFileName, "schema-minimal.xml");
-    IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
+    try (SolrCore core = h.getCore()) {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
 
-    // verify some props set to -1 get lucene internal defaults
-    assertEquals(-1, solrConfig.indexConfig.maxBufferedDocs);
-    assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, 
-                 iwc.getMaxBufferedDocs());
-    assertEquals(100, solrConfig.indexConfig.ramBufferSizeMB, 0.0D);
+      // verify some props set to -1 get lucene internal defaults
+      assertEquals(-1, solrConfig.indexConfig.maxBufferedDocs);
+      assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, iwc.getMaxBufferedDocs());
+      assertEquals(100, solrConfig.indexConfig.ramBufferSizeMB, 0.0D);
 
-    LogMergePolicy logMP = assertAndCast(mpClass, iwc.getMergePolicy());
+      LogMergePolicy logMP = assertAndCast(mpClass, iwc.getMergePolicy());
 
-    assertEquals(11, logMP.getMergeFactor());
-    assertEquals(456, logMP.getMaxMergeDocs());
+      assertEquals(11, logMP.getMergeFactor());
+      assertEquals(456, logMP.getMaxMergeDocs());
+    }
   }
 
   /**
diff --git a/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java b/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java
index 54857bb..246dda0 100644
--- a/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java
+++ b/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java
@@ -43,8 +43,6 @@ public class TestQuerySenderListener extends SolrTestCaseJ4 {
   }
 
   public void testListenerCreationCounts() {
-    h.getCore();
-
     assertEquals("Unexpected number of listeners created",
                  EXPECTED_MOCK_LISTENER_INSTANCES, 
                  MockEventListener.getCreateCount() - preInitMockListenerCount);
@@ -53,52 +51,52 @@ public class TestQuerySenderListener extends SolrTestCaseJ4 {
   @Test
   public void testRequestHandlerRegistry() {
     // property values defined in build.xml
-    SolrCore core = h.getCore();
-
-    assertEquals( 2, core.firstSearcherListeners.size() );
-    assertEquals( 2, core.newSearcherListeners.size() );
+    try (SolrCore core = h.getCore()) {
+      assertEquals(2, core.firstSearcherListeners.size());
+      assertEquals(2, core.newSearcherListeners.size());
+    }
   }
 
   @Test
   public void testSearcherEvents() throws Exception {
-    SolrCore core = h.getCore();
-    SolrEventListener newSearcherListener = null;
-    Iterator<SolrEventListener> it = core.newSearcherListeners.iterator();
-    while (it.hasNext()) {
-      SolrEventListener listener = it.next();
-      if (listener instanceof QuerySenderListener) {
-        newSearcherListener = listener;
+    try (SolrCore core = h.getCore()) {
+      SolrEventListener newSearcherListener = null;
+      Iterator<SolrEventListener> it = core.newSearcherListeners.iterator();
+      while (it.hasNext()) {
+        SolrEventListener listener = it.next();
+        if (listener instanceof QuerySenderListener) {
+          newSearcherListener = listener;
+        }
       }
-    }
 
-    assertTrue("Not an instance of QuerySenderListener", newSearcherListener instanceof QuerySenderListener);
-    QuerySenderListener qsl = (QuerySenderListener) newSearcherListener;
+      assertTrue("Not an instance of QuerySenderListener", newSearcherListener instanceof QuerySenderListener);
+      QuerySenderListener qsl = (QuerySenderListener) newSearcherListener;
 
-    h.getCore().withSearcher(currentSearcher -> {
-      qsl.newSearcher(currentSearcher, null);//test new Searcher
+      core.withSearcher(currentSearcher -> {
+        qsl.newSearcher(currentSearcher, null);//test new Searcher
 
-      MockQuerySenderListenerReqHandler mock = (MockQuerySenderListenerReqHandler) core.getRequestHandler("/mock");
-      assertNotNull("Mock is null", mock);
+        MockQuerySenderListenerReqHandler mock = (MockQuerySenderListenerReqHandler) core.getRequestHandler("/mock");
+        assertNotNull("Mock is null", mock);
 
-      {
-        String evt = mock.req.getParams().get(EventParams.EVENT);
-        assertNotNull("Event is null", evt);
-        assertTrue(evt + " is not equal to " + EventParams.FIRST_SEARCHER, evt.equals(EventParams.FIRST_SEARCHER) == true);
+        {
+          String evt = mock.req.getParams().get(EventParams.EVENT);
+          assertNotNull("Event is null", evt);
+          assertTrue(evt + " is not equal to " + EventParams.FIRST_SEARCHER, evt.equals(EventParams.FIRST_SEARCHER) == true);
 
-        assertU(adoc("id", "1"));
-        assertU(commit());
-      }
-
-//      h.getCore().withSearcher(newSearcher -> {
-//        String evt = mock.req.getParams().get(EventParams.EVENT);
-//        assertNotNull("Event is null", evt);
-//        assertTrue(evt + " is not equal to " + EventParams.NEW_SEARCHER, evt.equals(EventParams.NEW_SEARCHER) == true);
-//        return null;
-//      });
+          assertU(adoc("id", "1"));
+          assertU(commit());
+        }
 
-      return null;
-    });
+        //      h.getCore().withSearcher(newSearcher -> {
+        //        String evt = mock.req.getParams().get(EventParams.EVENT);
+        //        assertNotNull("Event is null", evt);
+        //        assertTrue(evt + " is not equal to " + EventParams.NEW_SEARCHER, evt.equals(EventParams.NEW_SEARCHER) == true);
+        //        return null;
+        //      });
 
+        return null;
+      });
+    }
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/core/TestQuerySenderNoQuery.java b/solr/core/src/test/org/apache/solr/core/TestQuerySenderNoQuery.java
index 3bf6148..17d328a 100644
--- a/solr/core/src/test/org/apache/solr/core/TestQuerySenderNoQuery.java
+++ b/solr/core/src/test/org/apache/solr/core/TestQuerySenderNoQuery.java
@@ -42,8 +42,6 @@ public class TestQuerySenderNoQuery extends SolrTestCaseJ4 {
   }
 
   public void testListenerCreationCounts() {
-    SolrCore core = h.getCore();
-
     assertEquals("Unexpected number of listeners created",
                  EXPECTED_MOCK_LISTENER_INSTANCES,
                  MockEventListener.getCreateCount() - preInitMockListenerCount);
@@ -52,47 +50,46 @@ public class TestQuerySenderNoQuery extends SolrTestCaseJ4 {
   @Test
   public void testRequestHandlerRegistry() {
     // property values defined in build.xml
-    SolrCore core = h.getCore();
-
-    assertEquals( 2, core.firstSearcherListeners.size() );
-    assertEquals( 2, core.newSearcherListeners.size() );
+    try (SolrCore core = h.getCore()) {
+      assertEquals(2, core.firstSearcherListeners.size());
+      assertEquals(2, core.newSearcherListeners.size());
+    }
   }
 
   // Determine that when the query lists are commented out of both new and
   // first searchers in the config, we don't throw an NPE
   @Test
   public void testSearcherEvents() throws Exception {
-    SolrCore core = h.getCore();
-    SolrEventListener nsl = null;
-    boolean foundQuerySenderListener = false;
-    Iterator<SolrEventListener> it = core.newSearcherListeners.iterator();
-    while (it.hasNext()) {
-      SolrEventListener newSearcherListener = it.next();
-      if (newSearcherListener instanceof QuerySenderListener) {
-        foundQuerySenderListener = true;
-        nsl = newSearcherListener;
+    try (SolrCore core = h.getCore()) {
+      SolrEventListener nsl = null;
+      boolean foundQuerySenderListener = false;
+      Iterator<SolrEventListener> it = core.newSearcherListeners.iterator();
+      while (it.hasNext()) {
+        SolrEventListener newSearcherListener = it.next();
+        if (newSearcherListener instanceof QuerySenderListener) {
+          foundQuerySenderListener = true;
+          nsl = newSearcherListener;
+        }
       }
-    }
 
+      assertTrue("Not an instance of QuerySenderListener", foundQuerySenderListener);
+      QuerySenderListener qsl = (QuerySenderListener) nsl;
 
+      core.withSearcher(currentSearcher -> {
+        SolrIndexSearcher dummy = null;
+        qsl.newSearcher(currentSearcher, dummy);//test first Searcher (since param is null)
+        MockQuerySenderListenerReqHandler mock = (MockQuerySenderListenerReqHandler) core.getRequestHandler("/mock");
+        assertNotNull("Mock is null", mock);
+        assertNull("Req (firstsearcher) is not null", mock.req);
 
-    assertTrue("Not an instance of QuerySenderListener", foundQuerySenderListener);
-    QuerySenderListener qsl = (QuerySenderListener) nsl;
+        SolrIndexSearcher newSearcher = new SolrIndexSearcher(core, core.getNewIndexDir(), core.getLatestSchema(), core.getSolrConfig().indexConfig, "testQuerySenderNoQuery", false, core.getDirectoryFactory());
 
-    h.getCore().withSearcher(currentSearcher -> {
-      SolrIndexSearcher dummy = null;
-      qsl.newSearcher(currentSearcher, dummy);//test first Searcher (since param is null)
-      MockQuerySenderListenerReqHandler mock = (MockQuerySenderListenerReqHandler) core.getRequestHandler("/mock");
-      assertNotNull("Mock is null", mock);
-      assertNull("Req (firstsearcher) is not null", mock.req);
-
-      SolrIndexSearcher newSearcher = new SolrIndexSearcher(core, core.getNewIndexDir(), core.getLatestSchema(), core.getSolrConfig().indexConfig, "testQuerySenderNoQuery", false, core.getDirectoryFactory());
-
-      qsl.newSearcher(newSearcher, currentSearcher); // get newSearcher.
-      assertNull("Req (newsearcher) is not null", mock.req);
-      newSearcher.close();
-      return null;
-    });
+        qsl.newSearcher(newSearcher, currentSearcher); // get newSearcher.
+        assertNull("Req (newsearcher) is not null", mock.req);
+        newSearcher.close();
+        return null;
+      });
+    }
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/core/TestSimpleTextCodec.java b/solr/core/src/test/org/apache/solr/core/TestSimpleTextCodec.java
index 9f6776b..95d7dca 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSimpleTextCodec.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSimpleTextCodec.java
@@ -32,32 +32,32 @@ public class TestSimpleTextCodec extends SolrTestCaseJ4 {
   }
 
   public void test() throws Exception {
-    SolrConfig config = h.getCore().getSolrConfig();
-    String codecFactory =  config.get("codecFactory/@class");
-    assertEquals("Unexpected solrconfig codec factory", "solr.SimpleTextCodecFactory", codecFactory);
-
-    assertEquals("Unexpected core codec", "SimpleText", h.getCore().getCodec().getName());
-
-    RefCounted<IndexWriter> writerRef = h.getCore().getSolrCoreState().getIndexWriter(h.getCore());
-    try {
-      IndexWriter writer = writerRef.get();
-      assertEquals("Unexpected codec in IndexWriter config", 
-          "SimpleText", writer.getConfig().getCodec().getName()); 
-    } finally {
-      writerRef.decref();
+    try (SolrCore core = h.getCore()) {
+      SolrConfig config = core.getSolrConfig();
+      String codecFactory = config.get("codecFactory/@class");
+      assertEquals("Unexpected solrconfig codec factory", "solr.SimpleTextCodecFactory", codecFactory);
+
+      assertEquals("Unexpected core codec", "SimpleText", core.getCodec().getName());
+
+      RefCounted<IndexWriter> writerRef = core.getSolrCoreState().getIndexWriter(core);
+      try {
+        IndexWriter writer = writerRef.get();
+        assertEquals("Unexpected codec in IndexWriter config", "SimpleText", writer.getConfig().getCodec().getName());
+      } finally {
+        writerRef.decref();
+      }
+
+      assertU(add(doc("id", "1", "text", "textual content goes here")));
+      assertU(commit());
+
+      core.withSearcher(searcher -> {
+        SegmentInfos infos = SegmentInfos.readLatestCommit(searcher.getIndexReader().directory());
+        SegmentInfo info = infos.info(infos.size() - 1).info;
+        assertEquals("Unexpected segment codec", "SimpleText", info.getCodec().getName());
+        return null;
+      });
+
+      assertQ(req("q", "id:1"), "*[count(//doc)=1]");
     }
-
-    assertU(add(doc("id","1", "text","textual content goes here")));
-    assertU(commit());
-
-    h.getCore().withSearcher(searcher -> {
-      SegmentInfos infos = SegmentInfos.readLatestCommit(searcher.getIndexReader().directory());
-      SegmentInfo info = infos.info(infos.size() - 1).info;
-      assertEquals("Unexpected segment codec", "SimpleText", info.getCodec().getName());
-      return null;
-    });
-
-    assertQ(req("q", "id:1"),
-        "*[count(//doc)=1]");
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrDeletionPolicy1.java b/solr/core/src/test/org/apache/solr/core/TestSolrDeletionPolicy1.java
index 375a58a..7eea5d4 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrDeletionPolicy1.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrDeletionPolicy1.java
@@ -88,49 +88,48 @@ public class TestSolrDeletionPolicy1 extends SolrTestCaseJ4 {
 
   @Test
   public void testKeepOptimizedOnlyCommits() {
-
-    IndexDeletionPolicyWrapper delPolicy = h.getCore().getDeletionPolicy();
-    addDocs();
-    Map<Long, IndexCommit> commits = delPolicy.getCommits();
-    IndexCommit latest = delPolicy.getLatestCommit();
-    for (Long gen : commits.keySet()) {
-      if (commits.get(gen) == latest)
-        continue;
-      assertEquals(1, commits.get(gen).getSegmentCount());
+    try (SolrCore core = h.getCore()) {
+      IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
+      addDocs();
+      Map<Long,IndexCommit> commits = delPolicy.getCommits();
+      IndexCommit latest = delPolicy.getLatestCommit();
+      for (Long gen : commits.keySet()) {
+        if (commits.get(gen) == latest) continue;
+        assertEquals(1, commits.get(gen).getSegmentCount());
+      }
     }
   }
 
   @Test
   public void testNumCommitsConfigured() {
-    IndexDeletionPolicyWrapper delPolicy = h.getCore().getDeletionPolicy();
-    addDocs();
-    Map<Long, IndexCommit> commits = delPolicy.getCommits();
-    assertTrue(commits.size() <= ((SolrDeletionPolicy) (delPolicy.getWrappedDeletionPolicy())).getMaxOptimizedCommitsToKeep());
+    try (SolrCore core = h.getCore()) {
+      IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
+      addDocs();
+      Map<Long,IndexCommit> commits = delPolicy.getCommits();
+      assertTrue(commits.size() <= ((SolrDeletionPolicy) (delPolicy.getWrappedDeletionPolicy())).getMaxOptimizedCommitsToKeep());
+    }
   }
 
   @Test
   public void testCommitAge() throws InterruptedException {
     assumeFalse("This test is not working on Windows (or maybe machines with only 2 CPUs)",
       Constants.WINDOWS);
-  
-    IndexDeletionPolicyWrapper delPolicy = h.getCore().getDeletionPolicy();
-    addDocs();
-    Map<Long, IndexCommit> commits = delPolicy.getCommits();
-    IndexCommit ic = delPolicy.getLatestCommit();
-    String agestr = ((SolrDeletionPolicy) (delPolicy.getWrappedDeletionPolicy())).getMaxCommitAge().replaceAll("[a-zA-Z]", "").replaceAll("-", "");
-    long age = Long.parseLong(agestr);
-    Thread.sleep(age);
-
-    assertU(adoc("id", String.valueOf(6),
-            "name", "name" + String.valueOf(6)));
-    assertU(optimize());
-    assertQ("return all docs",
-            req("id:[0 TO 6]"),
-            "*[count(//doc)=6]"
-    );
-
-    commits = delPolicy.getCommits();
-    assertTrue(!commits.containsKey(ic.getGeneration()));
+    try (SolrCore core = h.getCore()) {
+      IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
+      addDocs();
+      Map<Long,IndexCommit> commits = delPolicy.getCommits();
+      IndexCommit ic = delPolicy.getLatestCommit();
+      String agestr = ((SolrDeletionPolicy) (delPolicy.getWrappedDeletionPolicy())).getMaxCommitAge().replaceAll("[a-zA-Z]", "").replaceAll("-", "");
+      long age = Long.parseLong(agestr);
+      Thread.sleep(age);
+
+      assertU(adoc("id", String.valueOf(6), "name", "name" + String.valueOf(6)));
+      assertU(optimize());
+      assertQ("return all docs", req("id:[0 TO 6]"), "*[count(//doc)=6]");
+
+      commits = delPolicy.getCommits();
+      assertTrue(!commits.containsKey(ic.getGeneration()));
+    }
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrDeletionPolicy2.java b/solr/core/src/test/org/apache/solr/core/TestSolrDeletionPolicy2.java
index 99f3a5b..0153606 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrDeletionPolicy2.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrDeletionPolicy2.java
@@ -31,31 +31,26 @@ public class TestSolrDeletionPolicy2 extends SolrTestCaseJ4 {
 
   @Test
   public void testFakeDeletionPolicyClass() {
+    try (SolrCore core = h.getCore()) {
+      IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
+      assertTrue(delPolicy.getWrappedDeletionPolicy() instanceof FakeDeletionPolicy);
 
-    IndexDeletionPolicyWrapper delPolicy = h.getCore().getDeletionPolicy();
-    assertTrue(delPolicy.getWrappedDeletionPolicy() instanceof FakeDeletionPolicy);
+      FakeDeletionPolicy f = (FakeDeletionPolicy) delPolicy.getWrappedDeletionPolicy();
 
-    FakeDeletionPolicy f = (FakeDeletionPolicy) delPolicy.getWrappedDeletionPolicy();
+      assertTrue("value1".equals(f.getVar1()));
+      assertTrue("value2".equals(f.getVar2()));
 
-    assertTrue("value1".equals(f.getVar1()));
-    assertTrue("value2".equals(f.getVar2()));
+      assertU(adoc("id", String.valueOf(1), "name", "name" + String.valueOf(1)));
 
-    assertU(adoc("id", String.valueOf(1),
-            "name", "name" + String.valueOf(1)));
+      assertTrue(System.getProperty("onInit").equals("test.org.apache.solr.core.FakeDeletionPolicy.onInit"));
+      assertU(commit());
+      assertQ("return all docs", req("id:[0 TO 1]"), "*[count(//doc)=1]");
 
+      assertTrue(System.getProperty("onCommit").equals("test.org.apache.solr.core.FakeDeletionPolicy.onCommit"));
 
-    assertTrue(System.getProperty("onInit").equals("test.org.apache.solr.core.FakeDeletionPolicy.onInit"));
-    assertU(commit());
-    assertQ("return all docs",
-            req("id:[0 TO 1]"),
-            "*[count(//doc)=1]"
-    );
-
-
-    assertTrue(System.getProperty("onCommit").equals("test.org.apache.solr.core.FakeDeletionPolicy.onCommit"));
-
-    System.clearProperty("onInit");
-    System.clearProperty("onCommit");
+      System.clearProperty("onInit");
+      System.clearProperty("onCommit");
+    }
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java b/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java
index 6cad20e..fff04c2 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java
@@ -35,22 +35,25 @@ public class TestSolrIndexConfig extends SolrTestCaseJ4 {
   }
 
   public void testLiveWriter() throws Exception {
-    SolrCore core = h.getCore();
-    RefCounted<IndexWriter> iw = core.getUpdateHandler().getSolrCoreState().getIndexWriter(core);
-    try {
-      checkIndexWriterConfig(iw.get().getConfig());
-    } finally {
-      if (null != iw) iw.decref();
+    try (SolrCore core = h.getCore()) {
+      RefCounted<IndexWriter> iw = core.getUpdateHandler().getSolrCoreState().getIndexWriter(core);
+      try {
+        checkIndexWriterConfig(iw.get().getConfig());
+      } finally {
+        if (null != iw) iw.decref();
+      }
     }
   }
 
   
   public void testIndexConfigParsing() throws Exception {
-    IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
-    try {
-      checkIndexWriterConfig(iwc);
-    } finally {
-      iwc.getInfoStream().close();
+    try (SolrCore core = h.getCore()) {
+      IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(core);
+      try {
+        checkIndexWriterConfig(iwc);
+      } finally {
+        iwc.getInfoStream().close();
+      }
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java
index 34fb186..6270321 100644
--- a/solr/core/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java
@@ -24,6 +24,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.ContentStreamBase;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.junit.Before;
@@ -80,7 +81,8 @@ public class DocumentAnalysisRequestHandlerTest extends AnalysisRequestHandlerTe
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.add("analysis.query", "The Query String");
     params.add("analysis.showmatch", "true");
-    SolrQueryRequest req = new SolrQueryRequestBase(h.getCore(), params) {
+    SolrCore core = h.getCore();
+    SolrQueryRequest req = new SolrQueryRequestBase(core, params) {
       @Override
       public Iterable<ContentStream> getContentStreams() {
         return Collections.singleton(cs);
@@ -106,7 +108,7 @@ public class DocumentAnalysisRequestHandlerTest extends AnalysisRequestHandlerTe
     field = document.getField("text");
     assertNotNull(field);
     assertEquals("The Text", field.getFirstValue());
-
+    core.close();
     req.close();
   }
 
@@ -151,7 +153,8 @@ public class DocumentAnalysisRequestHandlerTest extends AnalysisRequestHandlerTe
     final ContentStream cs = new ByteStream(xmlBytes, "application/xml");
     
     ModifiableSolrParams params = new ModifiableSolrParams();
-    SolrQueryRequest req = new SolrQueryRequestBase(h.getCore(), params) {
+    SolrCore core = h.getCore();
+    SolrQueryRequest req = new SolrQueryRequestBase(core, params) {
       @Override
       public Iterable<ContentStream> getContentStreams() {
         return Collections.singleton(cs);
@@ -165,6 +168,7 @@ public class DocumentAnalysisRequestHandlerTest extends AnalysisRequestHandlerTe
     assertEquals(1, documents.size());
     SolrInputDocument doc = documents.get(0);
     assertEquals("Müller", doc.getField("id").getValue());
+    core.close();
   }
 
   // This test should also test charset detection in UpdateRequestHandler,
@@ -183,7 +187,8 @@ public class DocumentAnalysisRequestHandlerTest extends AnalysisRequestHandlerTe
     final ContentStream cs = new ByteStream(xmlBytes, "application/xml; charset=ISO-8859-1");
     
     ModifiableSolrParams params = new ModifiableSolrParams();
-    SolrQueryRequest req = new SolrQueryRequestBase(h.getCore(), params) {
+    SolrCore core = h.getCore();
+    SolrQueryRequest req = new SolrQueryRequestBase(core, params) {
       @Override
       public Iterable<ContentStream> getContentStreams() {
         return Collections.singleton(cs);
@@ -197,6 +202,7 @@ public class DocumentAnalysisRequestHandlerTest extends AnalysisRequestHandlerTe
     assertEquals(1, documents.size());
     SolrInputDocument doc = documents.get(0);
     assertEquals("Müller", doc.getField("id").getValue());
+    core.close();
   }
 
   /**
@@ -216,8 +222,9 @@ public class DocumentAnalysisRequestHandlerTest extends AnalysisRequestHandlerTe
             .setQuery("JUMPING")
             .setShowMatch(true)
             .addDocument(document);
-
-    NamedList<Object> result = handler.handleAnalysisRequest(request, h.getCore().getLatestSchema());
+    SolrCore core = h.getCore();
+    NamedList<Object> result = handler.handleAnalysisRequest(request, core.getLatestSchema());
+    core.close();
     assertNotNull("result is null and it shouldn't be", result);
     NamedList<NamedList<NamedList<Object>>> documentResult = (NamedList<NamedList<NamedList<Object>>>) result.get("1");
     assertNotNull("An analysis for document with key '1' should be returned", documentResult);
diff --git a/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java b/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
index fc17bdf..a5a6400 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
@@ -80,7 +80,8 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
 
     // even w/o a user sending any data, the SolrCore initialiation logic should have automatically created
     // an "empty" commit point that can be backed up...
-    final IndexCommit empty = h.getCore().getDeletionPolicy().getLatestCommit();
+    SolrCore core = h.getCore();
+    final IndexCommit empty = core.getDeletionPolicy().getLatestCommit();
     assertNotNull(empty);
     
     // white box sanity check that the commit point of the "reader" available from SolrIndexSearcher
@@ -88,7 +89,9 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
     // 
     // this is important to ensure that backup/snapshot behavior is consistent with user expection
     // when using typical commit + openSearcher
-    assertEquals(empty, h.getCore().withSearcher(s -> s.getIndexReader().getIndexCommit()));
+    assertEquals(empty, core.withSearcher(s -> s.getIndexReader().getIndexCommit()));
+
+    core.close();
 
     assertEquals(1L, empty.getGeneration());
     assertNotNull(empty.getSegmentsFileName());
@@ -188,8 +191,10 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
     assertU(commit());
     assertQ(req("q", "id:99"), "//result[@numFound='1']");
     assertQ(req("q", "*:*"), "//result[@numFound='1']");
-    
-    final IndexCommit oneDocCommit = h.getCore().getDeletionPolicy().getLatestCommit();
+
+    SolrCore core = h.getCore();
+    final IndexCommit oneDocCommit = core.getDeletionPolicy().getLatestCommit();
+    core.close();
     assertNotNull(oneDocCommit);
     final String oneDocSegmentFile = oneDocCommit.getSegmentsFileName();
   
@@ -308,7 +313,8 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
     assertQ(req("q", "*:*"), "//result[@numFound='0']");
     
     // sanity check what the searcher/reader of this empty index report about current commit
-    final IndexCommit empty = h.getCore().withSearcher(s -> {
+    SolrCore core = h.getCore();
+    final IndexCommit empty = core.withSearcher(s -> {
         // sanity check we are empty...
         assertEquals(0L, (long) s.getIndexReader().numDocs());
         
@@ -317,6 +323,7 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
         assertEquals(EXPECTED_GEN_OF_EMPTY_INDEX, (long) commit.getGeneration());
         return commit;
       });
+    core.close();
 
     // now let's add & soft commit 1 doc...
     assertU(adoc("id", "42"));
@@ -325,9 +332,9 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
     // verify it's "searchable" ...
     assertQ(req("q", "id:42"), "//result[@numFound='1']");
 
-    try (SolrCore core = h.getCore()) {
+    try (SolrCore core2 = h.getCore()) {
       // sanity check what the searcher/reader of this empty index report about current commit
-      IndexCommit oneDoc = core.withSearcher(s -> {
+      IndexCommit oneDoc = core2.withSearcher(s -> {
         // sanity check this really is the searcher/reader that has the new doc...
         assertEquals(1L, (long) s.getIndexReader().numDocs());
 
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
index 0b09a56..ac9b603 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
@@ -23,6 +23,7 @@ import org.apache.lucene.index.SegmentCommitInfo;
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.util.Version;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.index.NoMergePolicyFactory;
 import org.apache.solr.util.RefCounted;
 import org.junit.AfterClass;
@@ -54,41 +55,43 @@ public class SegmentsInfoRequestHandlerTest extends SolrTestCaseJ4 {
     
     System.setProperty("enable.update.log", "false"); // no _version_ in our schema
     initCore("solrconfig.xml", "schema12.xml"); // segments API shouldn't depend on _version_ or ulog
-    
-    // build up an index with at least 2 segments and some deletes
-    for (int i = 0; i < DOC_COUNT; i++) {
-      assertU(adoc("id","SOLR100" + i, "name","Apache Solr:" + i));
-    }
-    for (int i = 0; i < DEL_COUNT; i++) {
-      assertU(delI("SOLR100" + i));
-    }
-    assertU(commit());
-    for (int i = 0; i < DOC_COUNT; i++) {
-      assertU(adoc("id","SOLR200" + i, "name","Apache Solr:" + i));
+    try (SolrCore core = h.getCore()) {
+      // build up an index with at least 2 segments and some deletes
+      for (int i = 0; i < DOC_COUNT; i++) {
+        assertU(adoc("id", "SOLR100" + i, "name", "Apache Solr:" + i));
+      }
+      for (int i = 0; i < DEL_COUNT; i++) {
+        assertU(delI("SOLR100" + i));
+      }
+      assertU(commit());
+      for (int i = 0; i < DOC_COUNT; i++) {
+        assertU(adoc("id", "SOLR200" + i, "name", "Apache Solr:" + i));
+      }
+      assertU(commit());
+      core.withSearcher((searcher) -> {
+        int numSegments = SegmentInfos.readLatestCommit(searcher.getIndexReader().directory()).size();
+        // if this is not NUM_SEGMENTS, there was some unexpected flush or merge
+        assertEquals("Unexpected number of segment in the index: " + numSegments, NUM_SEGMENTS, numSegments);
+        return null;
+      });
+      // see SOLR-14431
+      RefCounted<IndexWriter> iwRef = core.getSolrCoreState().getIndexWriter(core);
+      initialRefCount = iwRef.getRefcount();
+      iwRef.decref();
     }
-    assertU(commit());
-    h.getCore().withSearcher((searcher) -> {
-      int numSegments = SegmentInfos.readLatestCommit(searcher.getIndexReader().directory()).size();
-      // if this is not NUM_SEGMENTS, there was some unexpected flush or merge
-      assertEquals("Unexpected number of segment in the index: " + numSegments, 
-          NUM_SEGMENTS, numSegments);
-      return null;
-    });
-    // see SOLR-14431
-    RefCounted<IndexWriter> iwRef = h.getCore().getSolrCoreState().getIndexWriter(h.getCore());
-    initialRefCount = iwRef.getRefcount();
-    iwRef.decref();
   }
   
   @AfterClass
   public static void afterClass() throws Exception {
-    RefCounted<IndexWriter> iwRef = h.getCore().getSolrCoreState().getIndexWriter(h.getCore());
-    int finalRefCount = iwRef.getRefcount();
-    iwRef.decref();
-    assertEquals("IW refcount mismatch", initialRefCount, finalRefCount);
-    systemClearPropertySolrTestsMergePolicyFactory();
-    System.clearProperty("solr.tests.maxBufferedDocs");
-    System.clearProperty("solr.tests.ramBufferSizeMB");
+    try (SolrCore core = h.getCore()) {
+      RefCounted<IndexWriter> iwRef = core.getSolrCoreState().getIndexWriter(core);
+      int finalRefCount = iwRef.getRefcount();
+      iwRef.decref();
+      assertEquals("IW refcount mismatch", initialRefCount, finalRefCount);
+      systemClearPropertySolrTestsMergePolicyFactory();
+      System.clearProperty("solr.tests.maxBufferedDocs");
+      System.clearProperty("solr.tests.ramBufferSizeMB");
+    }
   }
 
   @Test
@@ -107,20 +110,20 @@ public class SegmentsInfoRequestHandlerTest extends SolrTestCaseJ4 {
   
   @Test
   public void testSegmentNames() throws IOException {
-    String[] segmentNamePatterns = new String[NUM_SEGMENTS];
-    h.getCore().withSearcher((searcher) -> {
-      int i = 0;
-      for (SegmentCommitInfo sInfo : SegmentInfos.readLatestCommit(searcher.getIndexReader().directory())) {
-        assertTrue("Unexpected number of segment in the index: " + i, i < NUM_SEGMENTS);
-        segmentNamePatterns[i] = "//lst[@name='segments']/lst/str[@name='name'][.='" + sInfo.info.name + "']";
-        i++;
-      }
-      
-      return null;
-    });
-    assertQ("Unexpected segment names returned",
-        req("qt","/admin/segments"),
-        segmentNamePatterns);
+    try (SolrCore core = h.getCore()) {
+      String[] segmentNamePatterns = new String[NUM_SEGMENTS];
+      core.withSearcher((searcher) -> {
+        int i = 0;
+        for (SegmentCommitInfo sInfo : SegmentInfos.readLatestCommit(searcher.getIndexReader().directory())) {
+          assertTrue("Unexpected number of segment in the index: " + i, i < NUM_SEGMENTS);
+          segmentNamePatterns[i] = "//lst[@name='segments']/lst/str[@name='name'][.='" + sInfo.info.name + "']";
+          i++;
+        }
+
+        return null;
+      });
+      assertQ("Unexpected segment names returned", req("qt", "/admin/segments"), segmentNamePatterns);
+    }
   }
   
   @Test
@@ -142,20 +145,20 @@ public class SegmentsInfoRequestHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testFieldInfo() throws Exception {
-    String[] segmentNamePatterns = new String[NUM_SEGMENTS];
-    h.getCore().withSearcher((searcher) -> {
-      int i = 0;
-      for (SegmentCommitInfo sInfo : SegmentInfos.readLatestCommit(searcher.getIndexReader().directory())) {
-        assertTrue("Unexpected number of segment in the index: " + i, i < NUM_SEGMENTS);
-        segmentNamePatterns[i] = "boolean(//lst[@name='segments']/lst[@name='" + sInfo.info.name + "']/lst[@name='fields']/lst[@name='id']/str[@name='flags'])";
-        i++;
-      }
+    try (SolrCore core = h.getCore()) {
+      String[] segmentNamePatterns = new String[NUM_SEGMENTS];
+      core.withSearcher((searcher) -> {
+        int i = 0;
+        for (SegmentCommitInfo sInfo : SegmentInfos.readLatestCommit(searcher.getIndexReader().directory())) {
+          assertTrue("Unexpected number of segment in the index: " + i, i < NUM_SEGMENTS);
+          segmentNamePatterns[i] = "boolean(//lst[@name='segments']/lst[@name='" + sInfo.info.name + "']/lst[@name='fields']/lst[@name='id']/str[@name='flags'])";
+          i++;
+        }
 
-      return null;
-    });
-    assertQ("Unexpected field infos returned",
-        req("qt","/admin/segments", "fieldInfo", "true"),
-        segmentNamePatterns);
+        return null;
+      });
+      assertQ("Unexpected field infos returned", req("qt", "/admin/segments", "fieldInfo", "true"), segmentNamePatterns);
+    }
   }
 
 
diff --git a/solr/core/src/test/org/apache/solr/handler/component/PhrasesIdentificationComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/PhrasesIdentificationComponentTest.java
index b54b7a9..8ec150b 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/PhrasesIdentificationComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/PhrasesIdentificationComponentTest.java
@@ -27,6 +27,7 @@ import java.util.stream.IntStream;
 import java.util.stream.Stream;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.component.PhrasesIdentificationComponent.Phrase;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -75,7 +76,9 @@ public class PhrasesIdentificationComponentTest extends SolrTestCaseJ4 {
   }
 
   public void testWhiteBoxPhraseParsingLongInput() throws Exception {
-    final SchemaField field = h.getCore().getLatestSchema().getField("multigrams_body");
+    SolrCore core = h.getCore();
+    final SchemaField field = core.getLatestSchema().getField("multigrams_body");
+    core.close();
     assertNotNull(field);
     final List<Phrase> phrases = Phrase.extractPhrases
       (" did  a Quick    brown FOX perniciously jump over the lAZy dog", field, 3, 7);
@@ -167,7 +170,9 @@ public class PhrasesIdentificationComponentTest extends SolrTestCaseJ4 {
     for (Map.Entry<String,Integer> entry : fields.entrySet()) {
       try {
         final int maxQ = entry.getValue();
-        final SchemaField field = h.getCore().getLatestSchema().getField(entry.getKey());
+        SolrCore core = h.getCore();
+        final SchemaField field = core.getLatestSchema().getField(entry.getKey());
+        core.close();
         assertNotNull(field);
         
         // empty input shouldn't break anything
@@ -302,7 +307,9 @@ public class PhrasesIdentificationComponentTest extends SolrTestCaseJ4 {
   }
 
   public void testWhiteboxStats() throws Exception {
-    final SchemaField analysisField = h.getCore().getLatestSchema().getField("multigrams_body");
+    SolrCore core = h.getCore();
+    final SchemaField analysisField = core.getLatestSchema().getField("multigrams_body");
+    core.close();
     assertNotNull(analysisField);
     final String input = "BROWN fox lAzY  dog xxxyyyzzz";
 
@@ -382,7 +389,9 @@ public class PhrasesIdentificationComponentTest extends SolrTestCaseJ4 {
   }
   
   public void testWhiteboxScores() throws Exception {
-    final SchemaField analysisField = h.getCore().getLatestSchema().getField("multigrams_body");
+    SolrCore core = h.getCore();
+    final SchemaField analysisField = core.getLatestSchema().getField("multigrams_body");
+    core.close();
     assertNotNull(analysisField);
     final Map<String,Double> fieldWeights = new TreeMap<>();
     fieldWeights.put("multigrams_title", 1.0D);
@@ -460,7 +469,9 @@ public class PhrasesIdentificationComponentTest extends SolrTestCaseJ4 {
     
     { // If our analysisField uses all terms,
       // be we also generate scores from a field that filters stopwords...
-      final SchemaField analysisField = h.getCore().getLatestSchema().getField("multigrams_title");
+      SolrCore core = h.getCore();
+      final SchemaField analysisField = core.getLatestSchema().getField("multigrams_title");
+      core.close();
       assertNotNull(analysisField);
       
       final List<Phrase> phrases = Phrase.extractPhrases(input, analysisField, 3, 7);
@@ -495,7 +506,9 @@ public class PhrasesIdentificationComponentTest extends SolrTestCaseJ4 {
       //
       // (NOTE: the parser will still generate _some_ candidate phrases spaning the stop word position,
       // but not ones that start with the stopword)
-      final SchemaField analysisField = h.getCore().getLatestSchema().getField("multigrams_title_stop");
+      SolrCore core = h.getCore();
+      final SchemaField analysisField = core.getLatestSchema().getField("multigrams_title_stop");
+      core.close();
       assertNotNull(analysisField);
       
       final List<Phrase> phrases = Phrase.extractPhrases(input, analysisField, 3, 7);
@@ -565,7 +578,9 @@ public class PhrasesIdentificationComponentTest extends SolrTestCaseJ4 {
   }
   
   public void testMaxShingleSizeHelper() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     
     assertEquals(3, PhrasesIdentificationComponent.getMaxShingleSize
                  (schema.getFieldTypeByName("multigrams_3_7").getIndexAnalyzer()));
diff --git a/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
index 58beb9c..4ef318e 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
@@ -550,134 +550,76 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
 
   @Test
   public void testSorting() throws Exception {
-    try {
-      init("schema12.xml");
+
+    init("schema12.xml");
+    try (SolrCore core = h.getCore()) {
       assertU(adoc("id", "a", "title", "ipod trash trash", "str_s1", "group1"));
       assertU(adoc("id", "b", "title", "ipod ipod  trash", "str_s1", "group2"));
       assertU(adoc("id", "c", "title", "ipod ipod  ipod ", "str_s1", "group2"));
 
-      assertU(adoc("id", "x", "title", "boosted",                 "str_s1", "group1"));
-      assertU(adoc("id", "y", "title", "boosted boosted",         "str_s1", "group2"));
+      assertU(adoc("id", "x", "title", "boosted", "str_s1", "group1"));
+      assertU(adoc("id", "y", "title", "boosted boosted", "str_s1", "group2"));
       assertU(adoc("id", "z", "title", "boosted boosted boosted", "str_s1", "group2"));
       assertU(commit());
 
       final String query = "title:ipod";
 
-      final SolrParams baseParams = params(
-          "qt", "/elevate",
-          "q", query,
-          "fl", "id,score",
-          "indent", "true");
+      final SolrParams baseParams = params("qt", "/elevate", "q", query, "fl", "id,score", "indent", "true");
 
-      QueryElevationComponent booster = (QueryElevationComponent) h.getCore().getSearchComponent("elevate");
-      IndexReader reader = h.getCore().withSearcher(SolrIndexSearcher::getIndexReader);
+      QueryElevationComponent booster = (QueryElevationComponent) core.getSearchComponent("elevate");
+      IndexReader reader = core.withSearcher(SolrIndexSearcher::getIndexReader);
 
-      assertQ("Make sure standard sort works as expected", req(baseParams)
-          , "//*[@numFound='3']"
-          , "//result/doc[1]/str[@name='id'][.='c']"
-          , "//result/doc[2]/str[@name='id'][.='b']"
-          , "//result/doc[3]/str[@name='id'][.='a']"
-      );
+      assertQ("Make sure standard sort works as expected", req(baseParams), "//*[@numFound='3']", "//result/doc[1]/str[@name='id'][.='c']", "//result/doc[2]/str[@name='id'][.='b']",
+          "//result/doc[3]/str[@name='id'][.='a']");
 
       // Explicitly set what gets boosted
-      booster.setTopQueryResults(reader, query, false, new String[]{"x", "y", "z"}, null);
-
-      assertQ("All six should make it", req(baseParams)
-          , "//*[@numFound='6']"
-          , "//result/doc[1]/str[@name='id'][.='x']"
-          , "//result/doc[2]/str[@name='id'][.='y']"
-          , "//result/doc[3]/str[@name='id'][.='z']"
-          , "//result/doc[4]/str[@name='id'][.='c']"
-          , "//result/doc[5]/str[@name='id'][.='b']"
-          , "//result/doc[6]/str[@name='id'][.='a']"
-      );
+      booster.setTopQueryResults(reader, query, false, new String[] {"x", "y", "z"}, null);
+
+      assertQ("All six should make it", req(baseParams), "//*[@numFound='6']", "//result/doc[1]/str[@name='id'][.='x']", "//result/doc[2]/str[@name='id'][.='y']",
+          "//result/doc[3]/str[@name='id'][.='z']", "//result/doc[4]/str[@name='id'][.='c']", "//result/doc[5]/str[@name='id'][.='b']", "//result/doc[6]/str[@name='id'][.='a']");
 
       // now switch the order:
-      booster.setTopQueryResults(reader, query, false, new String[]{"a", "x"}, null);
-      assertQ(req(baseParams)
-          , "//*[@numFound='4']"
-          , "//result/doc[1]/str[@name='id'][.='a']"
-          , "//result/doc[2]/str[@name='id'][.='x']"
-          , "//result/doc[3]/str[@name='id'][.='c']"
-          , "//result/doc[4]/str[@name='id'][.='b']"
-      );
+      booster.setTopQueryResults(reader, query, false, new String[] {"a", "x"}, null);
+      assertQ(req(baseParams), "//*[@numFound='4']", "//result/doc[1]/str[@name='id'][.='a']", "//result/doc[2]/str[@name='id'][.='x']", "//result/doc[3]/str[@name='id'][.='c']",
+          "//result/doc[4]/str[@name='id'][.='b']");
 
       // Try normal sort by 'id'
       // default 'forceBoost' should be false
       assertFalse(booster.forceElevation);
-      assertQ(req(baseParams, "sort", "id asc")
-          , "//*[@numFound='4']"
-          , "//result/doc[1]/str[@name='id'][.='a']"
-          , "//result/doc[2]/str[@name='id'][.='b']"
-          , "//result/doc[3]/str[@name='id'][.='c']"
-          , "//result/doc[4]/str[@name='id'][.='x']"
-      );
+      assertQ(req(baseParams, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/str[@name='id'][.='a']", "//result/doc[2]/str[@name='id'][.='b']", "//result/doc[3]/str[@name='id'][.='c']",
+          "//result/doc[4]/str[@name='id'][.='x']");
 
-      assertQ("useConfiguredElevatedOrder=false",
-          req(baseParams, "sort", "str_s1 asc,id desc", "useConfiguredElevatedOrder", "false")
-          , "//*[@numFound='4']"
-          , "//result/doc[1]/str[@name='id'][.='x']"//group1
+      assertQ("useConfiguredElevatedOrder=false", req(baseParams, "sort", "str_s1 asc,id desc", "useConfiguredElevatedOrder", "false"), "//*[@numFound='4']", "//result/doc[1]/str[@name='id'][.='x']"
+          //group1
           , "//result/doc[2]/str[@name='id'][.='a']"//group1
-          , "//result/doc[3]/str[@name='id'][.='c']"
-          , "//result/doc[4]/str[@name='id'][.='b']"
-      );
+          , "//result/doc[3]/str[@name='id'][.='c']", "//result/doc[4]/str[@name='id'][.='b']");
 
       booster.forceElevation = true;
-      assertQ(req(baseParams, "sort", "id asc")
-          , "//*[@numFound='4']"
-          , "//result/doc[1]/str[@name='id'][.='a']"
-          , "//result/doc[2]/str[@name='id'][.='x']"
-          , "//result/doc[3]/str[@name='id'][.='b']"
-          , "//result/doc[4]/str[@name='id'][.='c']"
-      );
+      assertQ(req(baseParams, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/str[@name='id'][.='a']", "//result/doc[2]/str[@name='id'][.='x']", "//result/doc[3]/str[@name='id'][.='b']",
+          "//result/doc[4]/str[@name='id'][.='c']");
 
       booster.forceElevation = true;
-      assertQ("useConfiguredElevatedOrder=false and forceElevation",
-          req(baseParams, "sort", "id desc", "useConfiguredElevatedOrder", "false")
-          , "//*[@numFound='4']"
-          , "//result/doc[1]/str[@name='id'][.='x']" // force elevated
+      assertQ("useConfiguredElevatedOrder=false and forceElevation", req(baseParams, "sort", "id desc", "useConfiguredElevatedOrder", "false"), "//*[@numFound='4']",
+          "//result/doc[1]/str[@name='id'][.='x']" // force elevated
           , "//result/doc[2]/str[@name='id'][.='a']" // force elevated
-          , "//result/doc[3]/str[@name='id'][.='c']"
-          , "//result/doc[4]/str[@name='id'][.='b']"
-      );
+          , "//result/doc[3]/str[@name='id'][.='c']", "//result/doc[4]/str[@name='id'][.='b']");
 
       //Test exclusive (not to be confused with exclusion)
-      booster.setTopQueryResults(reader, query, false, new String[]{"x", "a"}, new String[]{});
-      assertQ(req(baseParams, "exclusive", "true")
-          , "//*[@numFound='2']"
-          , "//result/doc[1]/str[@name='id'][.='x']"
-          , "//result/doc[2]/str[@name='id'][.='a']"
-      );
+      booster.setTopQueryResults(reader, query, false, new String[] {"x", "a"}, new String[] {});
+      assertQ(req(baseParams, "exclusive", "true"), "//*[@numFound='2']", "//result/doc[1]/str[@name='id'][.='x']", "//result/doc[2]/str[@name='id'][.='a']");
 
       // Test exclusion
-      booster.setTopQueryResults(reader, query, false, new String[]{"x"}, new String[]{"a"});
-      assertQ(req(baseParams)
-          , "//*[@numFound='3']"
-          , "//result/doc[1]/str[@name='id'][.='x']"
-          , "//result/doc[2]/str[@name='id'][.='c']"
-          , "//result/doc[3]/str[@name='id'][.='b']"
-      );
-
+      booster.setTopQueryResults(reader, query, false, new String[] {"x"}, new String[] {"a"});
+      assertQ(req(baseParams), "//*[@numFound='3']", "//result/doc[1]/str[@name='id'][.='x']", "//result/doc[2]/str[@name='id'][.='c']", "//result/doc[3]/str[@name='id'][.='b']");
 
       // Test setting ids and excludes from http parameters
 
       booster.clearElevationProviderCache();
-      assertQ("All five should make it", req(baseParams, "elevateIds", "x,y,z", "excludeIds", "b")
-          , "//*[@numFound='5']"
-          , "//result/doc[1]/str[@name='id'][.='x']"
-          , "//result/doc[2]/str[@name='id'][.='y']"
-          , "//result/doc[3]/str[@name='id'][.='z']"
-          , "//result/doc[4]/str[@name='id'][.='c']"
-          , "//result/doc[5]/str[@name='id'][.='a']"
-      );
+      assertQ("All five should make it", req(baseParams, "elevateIds", "x,y,z", "excludeIds", "b"), "//*[@numFound='5']", "//result/doc[1]/str[@name='id'][.='x']",
+          "//result/doc[2]/str[@name='id'][.='y']", "//result/doc[3]/str[@name='id'][.='z']", "//result/doc[4]/str[@name='id'][.='c']", "//result/doc[5]/str[@name='id'][.='a']");
 
-      assertQ("All four should make it", req(baseParams, "elevateIds", "x,z,y", "excludeIds", "b,c")
-          , "//*[@numFound='4']"
-          , "//result/doc[1]/str[@name='id'][.='x']"
-          , "//result/doc[2]/str[@name='id'][.='z']"
-          , "//result/doc[3]/str[@name='id'][.='y']"
-          , "//result/doc[4]/str[@name='id'][.='a']"
-      );
+      assertQ("All four should make it", req(baseParams, "elevateIds", "x,z,y", "excludeIds", "b,c"), "//*[@numFound='4']", "//result/doc[1]/str[@name='id'][.='x']",
+          "//result/doc[2]/str[@name='id'][.='z']", "//result/doc[3]/str[@name='id'][.='y']", "//result/doc[4]/str[@name='id'][.='a']");
 
     } finally {
       delete();
@@ -705,13 +647,14 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
 
   @Test
   public void testElevationReloading() throws Exception {
-    try {
-      init("schema12.xml");
+
+    init("schema12.xml");
+    try (SolrCore core = h.getCore()) {
       String testfile = "data-elevation.xml";
-      File configFile = new File(h.getCore().getDataDir(), testfile);
+      File configFile = new File(core.getDataDir(), testfile);
       writeElevationConfigFile(configFile, "aaa", "A");
 
-      QueryElevationComponent comp = (QueryElevationComponent) h.getCore().getSearchComponent("elevate");
+      QueryElevationComponent comp = (QueryElevationComponent) core.getSearchComponent("elevate");
       NamedList<String> args = new NamedList<>();
       args.add(QueryElevationComponent.CONFIG_FILE, testfile);
       comp.init(args);
@@ -750,7 +693,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
       writeElevationConfigFile(configFile, "ccc", "C");
 
       // Without index change, but calling a different method that clears the elevationProviderCache, so we should load a new ElevationProvider.
-      int elevationRuleNumber = comp.loadElevationConfiguration(h.getCore());
+      int elevationRuleNumber = comp.loadElevationConfiguration(core);
       assertEquals(1, elevationRuleNumber);
       try (SolrQueryRequest req = req()) {
         elevationProvider = comp.getElevationProvider(req.getSearcher().getIndexReader(), req.getCore());
@@ -901,24 +844,25 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
   public void testElevatedIds() throws Exception {
     try (QueryElevationComponent comp = new QueryElevationComponent()) {
       init("schema12.xml");
-      SolrCore core = h.getCore();
+      try (SolrCore core = h.getCore()) {
 
-      NamedList<String> args = new NamedList<>();
-      args.add(QueryElevationComponent.FIELD_TYPE, "text");
-      args.add(QueryElevationComponent.CONFIG_FILE, "elevate.xml");
+        NamedList<String> args = new NamedList<>();
+        args.add(QueryElevationComponent.FIELD_TYPE, "text");
+        args.add(QueryElevationComponent.CONFIG_FILE, "elevate.xml");
 
-      comp.init(args);
-      comp.inform(core);
+        comp.init(args);
+        comp.inform(core);
 
-      SolrQueryRequest req = req();
-      IndexReader reader = req.getSearcher().getIndexReader();
-      QueryElevationComponent.ElevationProvider elevationProvider = comp.getElevationProvider(reader, core);
-      req.close();
+        SolrQueryRequest req = req();
+        IndexReader reader = req.getSearcher().getIndexReader();
+        QueryElevationComponent.ElevationProvider elevationProvider = comp.getElevationProvider(reader, core);
+        req.close();
 
-      assertEquals(toIdSet("1"), elevationProvider.getElevationForQuery("xxxx").elevatedIds);
-      assertEquals(toIdSet("10", "11", "12"), elevationProvider.getElevationForQuery("bb DD CC vv").elevatedIds);
-      assertEquals(toIdSet("10", "11", "12", "13"), elevationProvider.getElevationForQuery("BB Cc").elevatedIds);
-      assertEquals(toIdSet("10", "11", "12", "14"), elevationProvider.getElevationForQuery("aa bb dd cc aa").elevatedIds);
+        assertEquals(toIdSet("1"), elevationProvider.getElevationForQuery("xxxx").elevatedIds);
+        assertEquals(toIdSet("10", "11", "12"), elevationProvider.getElevationForQuery("bb DD CC vv").elevatedIds);
+        assertEquals(toIdSet("10", "11", "12", "13"), elevationProvider.getElevationForQuery("BB Cc").elevatedIds);
+        assertEquals(toIdSet("10", "11", "12", "14"), elevationProvider.getElevationForQuery("aa bb dd cc aa").elevatedIds);
+      }
     } finally {
       delete();
     }
diff --git a/solr/core/src/test/org/apache/solr/handler/export/TestExportWriter.java b/solr/core/src/test/org/apache/solr/handler/export/TestExportWriter.java
index 0948987..8fcc3fa 100644
--- a/solr/core/src/test/org/apache/solr/handler/export/TestExportWriter.java
+++ b/solr/core/src/test/org/apache/solr/handler/export/TestExportWriter.java
@@ -667,6 +667,7 @@ public class TestExportWriter extends SolrTestCaseJ4 {
   }
 
   @Test
+  @Nightly
   public void testMultipleSorts() throws Exception {
     assertU(delQ("*:*"));
     assertU(commit());
diff --git a/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java b/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java
index d3334e0..41e1e59 100644
--- a/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java
+++ b/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java
@@ -34,6 +34,7 @@ import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.HighlightParams;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.component.HighlightComponent;
 import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.handler.component.SearchComponent;
@@ -71,7 +72,8 @@ public class HighlighterTest extends SolrTestCaseJ4 {
   @Test
   public void testConfig()
   {
-    DefaultSolrHighlighter highlighter = (DefaultSolrHighlighter) HighlightComponent.getHighlighter(h.getCore());
+    SolrCore core = h.getCore();
+    DefaultSolrHighlighter highlighter = (DefaultSolrHighlighter) HighlightComponent.getHighlighter(core);
 
     // Make sure we loaded the one formatter
     SolrFormatter fmt1 = highlighter.formatters.get( null );
@@ -87,6 +89,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
     assertSame( gap, frag );
     assertTrue(gap instanceof GapFragmenter);
     assertTrue(regex instanceof RegexFragmenter);
+    core.close();
   }
 
   @Test
@@ -861,6 +864,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
   
   @Test
   public void testGetHighlightFields() {
+    SolrCore core = h.getCore();
     HashMap<String, String> args = new HashMap<>();
     args.put("fl", "id score");
     args.put("hl", "true");
@@ -880,7 +884,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
         10, args);
 
     SolrQueryRequest request = lrf.makeRequest("test");
-    SolrHighlighter highlighter = HighlightComponent.getHighlighter(h.getCore());
+    SolrHighlighter highlighter = HighlightComponent.getHighlighter(core);
     List<String> highlightFieldNames = Arrays.asList(highlighter
         .getHighlightFields(null, request, new String[] {}));
     assertTrue("Expected to highlight on field \"title\"", highlightFieldNames
@@ -894,7 +898,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
     args.put("hl.fl", "foo_*");
     lrf = h.getRequestFactory("", 0, 10, args);
     request = lrf.makeRequest("test");
-    highlighter = HighlightComponent.getHighlighter(h.getCore());
+    highlighter = HighlightComponent.getHighlighter(core);
     highlightFieldNames = Arrays.asList(highlighter.getHighlightFields(null,
         request, new String[] {}));
     assertEquals("Expected one field to highlight on", 1, highlightFieldNames
@@ -911,7 +915,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
     highlightedSetExpected.add("foo_s");
     highlightedSetExpected.add("bar_s");
     try (LocalSolrQueryRequest localRequest = lrf.makeRequest("test")) {
-      highlighter = HighlightComponent.getHighlighter(h.getCore());
+      highlighter = HighlightComponent.getHighlighter(core);
       final Set<String> highlightedSetActual = new HashSet<String>(
           Arrays.asList(highlighter.getHighlightFields(null,
               localRequest, new String[] {})));
@@ -922,7 +926,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
     args.put("hl.fl", "title, text"); // comma then space
     lrf = h.getRequestFactory("", 0, 10, args);
     request = lrf.makeRequest("test");
-    highlighter = HighlightComponent.getHighlighter(h.getCore());
+    highlighter = HighlightComponent.getHighlighter(core);
     highlightFieldNames = Arrays.asList(highlighter.getHighlightFields(null,
         request, new String[] {}));
     assertEquals("Expected one field to highlight on", 2, highlightFieldNames
@@ -935,6 +939,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
         highlightFieldNames.contains(""));
 
     request.close();
+    core.close();
   }
 
   @Test
@@ -1231,6 +1236,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
 
   @Test
   public void payloadFilteringSpanQuery() throws IOException {
+    SolrCore core = h.getCore();
     clearIndex();
 
     String FIELD_NAME = "payloadDelimited";
@@ -1244,7 +1250,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
         Collections.singletonList(new BytesRef(new byte[]{0, 0, 0, 7})));//bytes for integer 7
 
     //invoke highlight component... the hard way
-    final SearchComponent hlComp = h.getCore().getSearchComponent("highlight");
+    final SearchComponent hlComp = core.getSearchComponent("highlight");
     SolrQueryRequest req = req("hl", "true", "hl.fl", FIELD_NAME, HighlightParams.USE_PHRASE_HIGHLIGHTER, "true");
     try {
       SolrQueryResponse resp = new SolrQueryResponse();
@@ -1260,5 +1266,6 @@ public class HighlighterTest extends SolrTestCaseJ4 {
     } finally {
       req.close();
     }
+    core.close();
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java
index 949f7be..3e168c5 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java
@@ -17,11 +17,11 @@
 package org.apache.solr.highlight;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.component.HighlightComponent;
 import org.apache.solr.schema.IndexSchema;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 
 /** simple tests for PostingsSolrHighlighter */
 public class TestPostingsSolrHighlighter extends SolrTestCaseJ4 {
@@ -31,13 +31,13 @@ public class TestPostingsSolrHighlighter extends SolrTestCaseJ4 {
     initCore("solrconfig-postingshighlight.xml", "schema-postingshighlight.xml");
     
     // test our config is sane, just to be sure:
-    
+    SolrCore core = h.getCore();
     // postingshighlighter should be used
-    SolrHighlighter highlighter = HighlightComponent.getHighlighter(h.getCore());
+    SolrHighlighter highlighter = HighlightComponent.getHighlighter(core);
     assertTrue("wrong highlighter: " + highlighter.getClass(), highlighter instanceof PostingsSolrHighlighter);
     
     // 'text' and 'text3' should have offsets, 'text2' should not
-    IndexSchema schema = h.getCore().getLatestSchema();
+    IndexSchema schema = core.getLatestSchema();
     assertTrue(schema.getField("text").storeOffsetsWithPositions());
     assertTrue(schema.getField("text3").storeOffsetsWithPositions());
     assertFalse(schema.getField("text2").storeOffsetsWithPositions());
@@ -45,6 +45,7 @@ public class TestPostingsSolrHighlighter extends SolrTestCaseJ4 {
     assertU(adoc("text", "document one", "text2", "document one", "text3", "crappy document", "id", "101"));
     assertU(adoc("text", "second document", "text2", "second document", "text3", "crappier document", "id", "102"));
     assertU(commit());
+    core.close();
   }
   
   @After
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
index f6cec1b..373c65f 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
@@ -23,6 +23,7 @@ import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.reporters.MockMetricReporter;
 import org.apache.solr.schema.FieldType;
@@ -47,8 +48,10 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
     System.setProperty("solr.enableMetrics", "true");
     useFactory(null);
     initCore("solrconfig-basic.xml", "schema.xml");
-    coreMetricManager = h.getCore().getCoreMetricManager();
-    metricManager = h.getCore().getCoreContainer().getMetricManager();
+    SolrCore core = h.getCore();
+    coreMetricManager = core.getCoreMetricManager();
+    metricManager = core.getCoreContainer().getMetricManager();
+    core.close();
   }
 
   @After
@@ -159,8 +162,10 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testNonCloudRegistryName() throws Exception {
-    String registryName = h.getCore().getCoreMetricManager().getRegistryName();
-    String leaderRegistryName = h.getCore().getCoreMetricManager().getLeaderRegistryName();
+    SolrCore core = h.getCore();
+    String registryName = core.getCoreMetricManager().getRegistryName();
+    String leaderRegistryName = core.getCoreMetricManager().getLeaderRegistryName();
+    core.close();
     assertNotNull(registryName);
     assertEquals("solr.core.collection1", registryName);
     assertNull(leaderRegistryName);
diff --git a/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java b/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java
index faa7d97..7ff5ead 100644
--- a/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java
+++ b/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java
@@ -43,6 +43,7 @@ public class TestWriterPerf extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
+    useFactory(null);
     // we need DVs on point fields to compute stats & facets
     if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
     initCore("solrconfig-functionquery.xml", "schema11.xml");
@@ -95,7 +96,7 @@ public class TestWriterPerf extends SolrTestCaseJ4 {
   /** make sure to close req after you are done using the response */
   public SolrQueryResponse getResponse(SolrQueryRequest req) throws Exception {
     SolrQueryResponse rsp = new SolrQueryResponse();
-    h.getCore().execute(h.getCore().getRequestHandler(null),req,rsp);
+    req.getCore().execute(req.getCore().getRequestHandler(null),req,rsp);
     if (rsp.getException() != null) {
       throw rsp.getException();
     }
@@ -105,26 +106,25 @@ public class TestWriterPerf extends SolrTestCaseJ4 {
 
   void doPerf(String writerName, SolrQueryRequest req, int encIter, int decIter) throws Exception {
     SolrQueryResponse rsp = getResponse(req);
-    QueryResponseWriter w = h.getCore().getQueryResponseWriter(writerName);
+    QueryResponseWriter w = req.getCore().getQueryResponseWriter(writerName);
 
-
-    ByteArrayOutputStream out=null;
+    ByteArrayOutputStream out = null;
 
     RTimer timer = new RTimer();
-    for (int i=0; i<encIter; i++) {
-    if (w instanceof BinaryQueryResponseWriter) {
-      BinaryQueryResponseWriter binWriter = (BinaryQueryResponseWriter) w;
-      out = new ByteArrayOutputStream();
-      binWriter.write(out, req, rsp);
-      out.close();
-    } else {
-      out = new ByteArrayOutputStream();
-      // to be fair, from my previous tests, much of the performance will be sucked up
-      // by java's UTF-8 encoding/decoding, not the actual writing
-      Writer writer = new OutputStreamWriter(out, StandardCharsets.UTF_8);
-      w.write(writer, req, rsp);
-      writer.close();
-    }
+    for (int i = 0; i < encIter; i++) {
+      if (w instanceof BinaryQueryResponseWriter) {
+        BinaryQueryResponseWriter binWriter = (BinaryQueryResponseWriter) w;
+        out = new ByteArrayOutputStream();
+        binWriter.write(out, req, rsp);
+        out.close();
+      } else {
+        out = new ByteArrayOutputStream();
+        // to be fair, from my previous tests, much of the performance will be sucked up
+        // by java's UTF-8 encoding/decoding, not the actual writing
+        Writer writer = new OutputStreamWriter(out, StandardCharsets.UTF_8);
+        w.write(writer, req, rsp);
+        writer.close();
+      }
     }
 
     double encodeTime = timer.getTime();
@@ -133,7 +133,7 @@ public class TestWriterPerf extends SolrTestCaseJ4 {
 
     timer = new RTimer();
     writerName = writerName.intern();
-    for (int i=0; i<decIter; i++) {
+    for (int i = 0; i < decIter; i++) {
       ResponseParser rp = null;
       if (writerName == "xml") {
         rp = new XMLResponseParser();
@@ -143,17 +143,14 @@ public class TestWriterPerf extends SolrTestCaseJ4 {
         break;
       }
       ByteArrayInputStream in = new ByteArrayInputStream(arr);
-      rp.processResponse(in, "UTF-8");      
+      rp.processResponse(in, "UTF-8");
     }
 
     double decodeTime = timer.getTime();
 
     if (log.isInfoEnabled()) {
-      log.info("writer {}, size={}, encodeRate={} decodeRate={}"
-          , writerName, out.size(), (encIter * 1000L / encodeTime), (decIter * 1000L / decodeTime));
+      log.info("writer {}, size={}, encodeRate={} decodeRate={}", writerName, out.size(), (encIter * 1000L / encodeTime), (decIter * 1000L / decodeTime));
     }
-
-    req.close();
   }
 
   public void testPerf() throws Exception {
@@ -196,6 +193,7 @@ public class TestWriterPerf extends SolrTestCaseJ4 {
     // doPerf("json", req, encIter, decIter);
     //doPerf("javabin", req, encIter, decIter);
     // doPerf("javabin", req, 1, decIter);
+    req.close();
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/schema/ChangedSchemaMergeTest.java b/solr/core/src/test/org/apache/solr/schema/ChangedSchemaMergeTest.java
index fdff888..7f79358 100644
--- a/solr/core/src/test/org/apache/solr/schema/ChangedSchemaMergeTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/ChangedSchemaMergeTest.java
@@ -129,7 +129,7 @@ public class ChangedSchemaMergeTest extends SolrTestCaseJ4 {
       changed = cc.getCore("changed");
       addDoc(changed, "id", "4", "which", "15", "text", "some stuff with which");
       changed = cc.getCore("changed");
-      SolrQueryRequest req = new LocalSolrQueryRequest(changed, new NamedList<>());
+      SolrQueryRequest req = new LocalSolrQueryRequest(changed, new NamedList<>(), true);
       changed.getUpdateHandler().commit(new CommitUpdateCommand(req, false));
       req.close();
       // write the new schema out and make it current
diff --git a/solr/core/src/test/org/apache/solr/schema/CopyFieldTest.java b/solr/core/src/test/org/apache/solr/schema/CopyFieldTest.java
index d6923d7..696a76d 100644
--- a/solr/core/src/test/org/apache/solr/schema/CopyFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/CopyFieldTest.java
@@ -112,14 +112,13 @@ public class CopyFieldTest extends SolrTestCaseJ4 {
   @Test
   public void testCopyFieldFunctionality() 
     {
-      SolrCore core = h.getCore();
       assertU(adoc("id", "5", "title", "test copy field", "text_en", "this is a simple test of the copy field functionality"));
       assertU(commit());
       
       Map<String,String> args = new HashMap<>();
       args.put( CommonParams.Q, "text_en:simple" );
       args.put( "indent", "true" );
-      SolrQueryRequest req = new LocalSolrQueryRequest( core, new MapSolrParams( args) );
+      SolrQueryRequest req = new LocalSolrQueryRequest( h.getCore(), new MapSolrParams( args), true );
       
       assertQ("Make sure they got in", req
               ,"//*[@numFound='1']"
@@ -129,8 +128,8 @@ public class CopyFieldTest extends SolrTestCaseJ4 {
       args = new HashMap<>();
       args.put( CommonParams.Q, "highlight:simple" );
       args.put( "indent", "true" );
-      core = h.getCore();
-      req = new LocalSolrQueryRequest( core, new MapSolrParams( args) );
+      SolrCore core = h.getCore();
+      req = new LocalSolrQueryRequest( core, new MapSolrParams( args), true );
       assertQ("dynamic source", req
               ,"//*[@numFound='1']"
               ,"//result/doc[1]/str[@name='id'][.='5']"
@@ -141,7 +140,7 @@ public class CopyFieldTest extends SolrTestCaseJ4 {
       args.put( CommonParams.Q, "text_en:functionality" );
       args.put( "indent", "true" );
       core = h.getCore();
-      req = new LocalSolrQueryRequest( core, new MapSolrParams( args) );
+      req = new LocalSolrQueryRequest( core, new MapSolrParams( args), true );
       assertQ("Make sure they got in", req
               ,"//*[@numFound='1']");
       
@@ -149,7 +148,7 @@ public class CopyFieldTest extends SolrTestCaseJ4 {
       args.put( CommonParams.Q, "highlight:functionality" );
       args.put( "indent", "true" );
       core = h.getCore();
-      req = new LocalSolrQueryRequest( core, new MapSolrParams( args) );
+      req = new LocalSolrQueryRequest( core, new MapSolrParams( args), true );
       assertQ("dynamic source", req
               ,"//*[@numFound='0']");
     }
@@ -244,19 +243,16 @@ public class CopyFieldTest extends SolrTestCaseJ4 {
   }
 
   public void testCatchAllCopyField() {
-    SolrCore core = h.getCore();
-    IndexSchema schema = core.getLatestSchema();
-    core.close();
+    try (SolrCore core = h.getCore()) {
+      IndexSchema schema = core.getLatestSchema();
 
-    assertNull("'*' should not be (or match) a dynamic field", 
-               schema.getDynamicPattern("*"));
-    
-    assertU(adoc("id", "5", "sku1", "10-1839ACX-93", "testing123_s", "AAM46"));
-    assertU(commit());
-    for (String q : new String[] {"5", "10-1839ACX-93", "AAM46" }) {
-      assertQ(req("q","catchall_t:" + q)
-              ,"//*[@numFound='1']"
-              ,"//result/doc[1]/str[@name='id'][.='5']");
+      assertNull("'*' should not be (or match) a dynamic field", schema.getDynamicPattern("*"));
+
+      assertU(adoc("id", "5", "sku1", "10-1839ACX-93", "testing123_s", "AAM46"));
+      assertU(commit());
+      for (String q : new String[] {"5", "10-1839ACX-93", "AAM46"}) {
+        assertQ(req("q", "catchall_t:" + q), "//*[@numFound='1']", "//result/doc[1]/str[@name='id'][.='5']");
+      }
     }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/schema/EnumFieldTest.java b/solr/core/src/test/org/apache/solr/schema/EnumFieldTest.java
index 98e55fa..7c1dd71 100644
--- a/solr/core/src/test/org/apache/solr/schema/EnumFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/EnumFieldTest.java
@@ -25,6 +25,7 @@ import java.util.regex.Pattern;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.search.SolrQueryParser;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -54,7 +55,9 @@ public class EnumFieldTest extends SolrTestCaseJ4 {
         System.getProperty("solr.tests.EnumFieldType").equals("solr.EnumFieldType")
             && System.getProperty("solr.tests.numeric.dv").equals("false"));
 
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
 
     SchemaField enumField = schema.getField(FIELD_NAME);
     assertNotNull(enumField);
@@ -463,8 +466,9 @@ public class EnumFieldTest extends SolrTestCaseJ4 {
             && System.getProperty("solr.tests.numeric.dv").equals("false"));
 
     clearIndex();
-
-    SchemaField sf = h.getCore().getLatestSchema().getField(FIELD_NAME);
+    SolrCore core = h.getCore();
+    SchemaField sf = core.getLatestSchema().getField(FIELD_NAME);
+    core.close();
     Set<String> enumStrs = ((AbstractEnumField)sf.getType()).getEnumMapping().enumStringToIntMap.keySet();
     assertTrue(enumStrs.size() > SolrQueryParser.TERMS_QUERY_THRESHOLD);
 
@@ -512,8 +516,9 @@ public class EnumFieldTest extends SolrTestCaseJ4 {
             && System.getProperty("solr.tests.numeric.dv").equals("false"));
 
     clearIndex();
-
-    SchemaField sf = h.getCore().getLatestSchema().getField(MV_FIELD_NAME);
+    SolrCore core = h.getCore();
+    SchemaField sf = core.getLatestSchema().getField(MV_FIELD_NAME);
+    core.close();
     Set<String> enumStrs = ((AbstractEnumField)sf.getType()).getEnumMapping().enumStringToIntMap.keySet();
     assertTrue(enumStrs.size() > SolrQueryParser.TERMS_QUERY_THRESHOLD);
 
diff --git a/solr/core/src/test/org/apache/solr/schema/IndexSchemaRuntimeFieldTest.java b/solr/core/src/test/org/apache/solr/schema/IndexSchemaRuntimeFieldTest.java
index 8e1a7fd..4649211 100644
--- a/solr/core/src/test/org/apache/solr/schema/IndexSchemaRuntimeFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/IndexSchemaRuntimeFieldTest.java
@@ -46,6 +46,7 @@ public class IndexSchemaRuntimeFieldTest extends SolrTestCaseJ4 {
 
     SolrCore core = h.getCore();
     IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "runtimefield";
     SchemaField sf = new SchemaField( fieldName, schema.getFieldTypes().get( "string" ) );
     schema.getFields().put( fieldName, sf );
diff --git a/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java b/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java
index 317b542..2414092 100644
--- a/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java
@@ -40,8 +40,9 @@ public class PolyFieldTest extends SolrTestCaseJ4 {
 
   @Test
   public void testSchemaBasics() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
-
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
 
     SchemaField home = schema.getField("home");
     assertNotNull(home);
@@ -76,6 +77,7 @@ public class PolyFieldTest extends SolrTestCaseJ4 {
   public void testPointFieldType() throws Exception {
     SolrCore core = h.getCore();
     IndexSchema schema = core.getLatestSchema();
+    core.close();
     SchemaField home = schema.getField("home");
     assertNotNull(home);
     assertTrue("home is not a poly field", home.isPolyField());
@@ -166,6 +168,7 @@ public class PolyFieldTest extends SolrTestCaseJ4 {
   public void testSearchDetails() throws Exception {
     SolrCore core = h.getCore();
     IndexSchema schema = core.getLatestSchema();
+    core.close();
     double[] xy = new double[]{35.0, -79.34};
     String point = xy[0] + "," + xy[1];
     //How about some queries?
diff --git a/solr/core/src/test/org/apache/solr/schema/PreAnalyzedFieldTest.java b/solr/core/src/test/org/apache/solr/schema/PreAnalyzedFieldTest.java
index cec1040..84c96a3 100644
--- a/solr/core/src/test/org/apache/solr/schema/PreAnalyzedFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/PreAnalyzedFieldTest.java
@@ -25,6 +25,7 @@ import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.document.Field;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.PreAnalyzedField.PreAnalyzedParser;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -103,21 +104,23 @@ public class PreAnalyzedFieldTest extends SolrTestCaseJ4 {
   
   @Test
   public void testValidSimple() {
-    PreAnalyzedField paf = new PreAnalyzedField();
-    // use Simple format
-    HashMap<String,String> args = new HashMap<>();
-    args.put(PreAnalyzedField.PARSER_IMPL, SimplePreAnalyzedParser.class.getName());
-    paf.init(h.getCore().getLatestSchema(), args);
-    PreAnalyzedParser parser = new SimplePreAnalyzedParser();
-    for (int i = 0; i < valid.length; i++) {
-      String s = valid[i];
-      try {
-        Field f = (Field)paf.fromString(field, s);
-        //System.out.println(" - toString: '" + sb.toString() + "'");
-        assertEquals(validParsed[i], parser.toFormattedString(f));
-      } catch (Exception e) {
-        log.error("", e);
-        fail("Should pass: '" + s + "', exception: " + e);
+    try (SolrCore core = h.getCore()) {
+      PreAnalyzedField paf = new PreAnalyzedField();
+      // use Simple format
+      HashMap<String,String> args = new HashMap<>();
+      args.put(PreAnalyzedField.PARSER_IMPL, SimplePreAnalyzedParser.class.getName());
+      paf.init(core.getLatestSchema(), args);
+      PreAnalyzedParser parser = new SimplePreAnalyzedParser();
+      for (int i = 0; i < valid.length; i++) {
+        String s = valid[i];
+        try {
+          Field f = (Field) paf.fromString(field, s);
+          //System.out.println(" - toString: '" + sb.toString() + "'");
+          assertEquals(validParsed[i], parser.toFormattedString(f));
+        } catch (Exception e) {
+          log.error("", e);
+          fail("Should pass: '" + s + "', exception: " + e);
+        }
       }
     }
   }
@@ -182,45 +185,49 @@ public class PreAnalyzedFieldTest extends SolrTestCaseJ4 {
 
   @Test
   public void testInvalidSimple() {
-    PreAnalyzedField paf = new PreAnalyzedField();
-    paf.init(h.getCore().getLatestSchema(), Collections.<String,String>emptyMap());
-    for (String s : invalidSimple) {
-      try {
-        paf.fromString(field, s);
-        fail("should fail: '" + s + "'");
-      } catch (Exception e) {
-        //
+    try (SolrCore core = h.getCore()) {
+      PreAnalyzedField paf = new PreAnalyzedField();
+      paf.init(core.getLatestSchema(), Collections.<String,String>emptyMap());
+      for (String s : invalidSimple) {
+        try {
+          paf.fromString(field, s);
+          fail("should fail: '" + s + "'");
+        } catch (Exception e) {
+          //
+        }
       }
     }
   }
 
   public void testInvalidJson() throws Exception {
-    PreAnalyzedField paf = new PreAnalyzedField();
-    paf.init(h.getCore().getLatestSchema(), Collections.emptyMap());
-    Analyzer preAnalyzer = paf.getIndexAnalyzer();
-    for (String s: invalidJson) {
-      TokenStream stream = null;
-      try {
-        stream = preAnalyzer.tokenStream("dummy", s);
-        stream.reset(); // exception should be triggered here.
-        fail("should fail: '" + s + "'");
-      } catch (Exception e) {
-        // expected
-      } finally {
-        if (stream != null) {
-          stream.close();
+    try (SolrCore core = h.getCore()) {
+      PreAnalyzedField paf = new PreAnalyzedField();
+      paf.init(core.getLatestSchema(), Collections.emptyMap());
+      Analyzer preAnalyzer = paf.getIndexAnalyzer();
+      for (String s : invalidJson) {
+        TokenStream stream = null;
+        try {
+          stream = preAnalyzer.tokenStream("dummy", s);
+          stream.reset(); // exception should be triggered here.
+          fail("should fail: '" + s + "'");
+        } catch (Exception e) {
+          // expected
+        } finally {
+          if (stream != null) {
+            stream.close();
+          }
         }
       }
+      // make sure the analyzer can now handle properly formatted input
+      TokenStream stream = preAnalyzer.tokenStream("dummy", validJson);
+      CharTermAttribute termAttr = stream.addAttribute(CharTermAttribute.class);
+      stream.reset();
+      while (stream.incrementToken()) {
+        assertFalse("zero-length token", termAttr.length() == 0);
+      }
+      stream.end();
+      stream.close();
     }
-    // make sure the analyzer can now handle properly formatted input
-    TokenStream stream = preAnalyzer.tokenStream("dummy", validJson);
-    CharTermAttribute termAttr = stream.addAttribute(CharTermAttribute.class);
-    stream.reset();
-    while (stream.incrementToken()) {
-      assertFalse("zero-length token", termAttr.length() == 0);
-    }
-    stream.end();
-    stream.close();
   }
   
   // "1 =test ąćęłńóśźż \u0001=one,i=22,s=123,e=128,p=deadbeef,y=word two,i=1,s=5,e=8,y=word three,i=1,s=20,e=22,y=foobar"
@@ -233,26 +240,28 @@ public class PreAnalyzedFieldTest extends SolrTestCaseJ4 {
   
   @Test
   public void testParsers() throws Exception {
-    PreAnalyzedField paf = new PreAnalyzedField();
-    // use Simple format
-    HashMap<String,String> args = new HashMap<>();
-    args.put(PreAnalyzedField.PARSER_IMPL, SimplePreAnalyzedParser.class.getName());
-    paf.init(h.getCore().getLatestSchema(), args);
-    {
-      Field f = (Field)paf.fromString(field, valid[0]);
-    }
+    try (SolrCore core = h.getCore()) {
+      PreAnalyzedField paf = new PreAnalyzedField();
+      // use Simple format
+      HashMap<String,String> args = new HashMap<>();
+      args.put(PreAnalyzedField.PARSER_IMPL, SimplePreAnalyzedParser.class.getName());
+      paf.init(core.getLatestSchema(), args);
+      {
+        Field f = (Field) paf.fromString(field, valid[0]);
+      }
 
-    // use JSON format
-    args.put(PreAnalyzedField.PARSER_IMPL, JsonPreAnalyzedParser.class.getName());
-    paf.init(h.getCore().getLatestSchema(), args);
-    expectThrows(Exception.class, () -> paf.fromString(field, valid[0]));
+      // use JSON format
+      args.put(PreAnalyzedField.PARSER_IMPL, JsonPreAnalyzedParser.class.getName());
+      paf.init(core.getLatestSchema(), args);
+      expectThrows(Exception.class, () -> paf.fromString(field, valid[0]));
 
-    byte[] deadbeef = new byte[]{(byte)0xd, (byte)0xe, (byte)0xa, (byte)0xd, (byte)0xb, (byte)0xe, (byte)0xe, (byte)0xf};
-    PreAnalyzedParser parser = new JsonPreAnalyzedParser();
+      byte[] deadbeef = new byte[] {(byte) 0xd, (byte) 0xe, (byte) 0xa, (byte) 0xd, (byte) 0xb, (byte) 0xe, (byte) 0xe, (byte) 0xf};
+      PreAnalyzedParser parser = new JsonPreAnalyzedParser();
 
-    {
-      Field f = (Field)paf.fromString(field, jsonValid);
-      assertEquals(jsonValid, parser.toFormattedString(f));
+      {
+        Field f = (Field) paf.fromString(field, jsonValid);
+        assertEquals(jsonValid, parser.toFormattedString(f));
+      }
     }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/schema/RequiredFieldsTest.java b/solr/core/src/test/org/apache/solr/schema/RequiredFieldsTest.java
index 6827002..aa2b050 100644
--- a/solr/core/src/test/org/apache/solr/schema/RequiredFieldsTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/RequiredFieldsTest.java
@@ -41,6 +41,7 @@ public class RequiredFieldsTest extends SolrTestCaseJ4 {
   public void testRequiredFieldsConfig() {
     SolrCore core = h.getCore();
     IndexSchema schema = core.getLatestSchema();
+    core.close();
     SchemaField uniqueKey = schema.getUniqueKeyField();
 
     // Make sure the uniqueKey is required
@@ -84,7 +85,8 @@ public class RequiredFieldsTest extends SolrTestCaseJ4 {
     assertU(commit());
     
     // Check to make sure this submission did not succeed
-    assertQ("should not find any", req("id:531") ,"//result[@numFound=0]" ); 
+    assertQ("should not find any", req("id:531") ,"//result[@numFound=0]" );
+    core.close();
   }
   
   @Test
diff --git a/solr/core/src/test/org/apache/solr/schema/ResolveAnalyzerByNameTest.java b/solr/core/src/test/org/apache/solr/schema/ResolveAnalyzerByNameTest.java
index 366dbaa..cb45b96 100644
--- a/solr/core/src/test/org/apache/solr/schema/ResolveAnalyzerByNameTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/ResolveAnalyzerByNameTest.java
@@ -31,7 +31,7 @@ import org.junit.Test;
  *
  */
 
-public class ResolveAnalyzerByNameTest extends SolrTestCaseJ4 {
+public class                ResolveAnalyzerByNameTest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeTests() throws Exception {
@@ -42,6 +42,7 @@ public class ResolveAnalyzerByNameTest extends SolrTestCaseJ4 {
   public void testSchemaLoadingSimpleAnalyzer() {
     SolrCore core = h.getCore();
     IndexSchema schema = core.getLatestSchema();
+    core.close();
     assertTrue( schema.getFieldTypes().containsKey("text_ws") );
     SimpleOrderedMap<Object> analyzerProps =
         (SimpleOrderedMap<Object>)schema.getFieldTypeByName("text_ws")
@@ -56,6 +57,7 @@ public class ResolveAnalyzerByNameTest extends SolrTestCaseJ4 {
   public void testSchemaLoadingComplexAnalyzer() {
     SolrCore core = h.getCore();
     IndexSchema schema = core.getLatestSchema();
+    core.close();
     assertTrue( schema.getFieldTypes().containsKey("text") );
 
     SimpleOrderedMap<Object> indexAnalyzerProps =
@@ -78,6 +80,7 @@ public class ResolveAnalyzerByNameTest extends SolrTestCaseJ4 {
   public void testSchemaLoadingAnalyzerWithCharFilters() {
     SolrCore core = h.getCore();
     IndexSchema schema = core.getLatestSchema();
+    core.close();
     assertTrue( schema.getFieldTypes().containsKey("charfilthtmlmap") );
     SimpleOrderedMap<Object> analyzerProps =
         (SimpleOrderedMap<Object>)schema.getFieldTypeByName("charfilthtmlmap")
diff --git a/solr/core/src/test/org/apache/solr/schema/TestSchemaField.java b/solr/core/src/test/org/apache/solr/schema/TestSchemaField.java
index 11912e3..8963a6b 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestSchemaField.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestSchemaField.java
@@ -41,7 +41,9 @@ public class TestSchemaField extends SolrTestCaseJ4 {
   }
 
   private void assertFieldTypeFormats(String fieldTypeName, String expectedPostingsFormat, String expectedDocValuesFormat) {
-    FieldType ft = h.getCore().getLatestSchema().getFieldTypeByName(fieldTypeName);
+    SolrCore core = h.getCore();
+    FieldType ft = core.getLatestSchema().getFieldTypeByName(fieldTypeName);
+    core.close();
     assertNotNull("Field type " + fieldTypeName + " not found  - schema got changed?", ft);
     assertEquals("Field type " + ft.getTypeName() + " wrong " + FieldProperties.POSTINGS_FORMAT
             + "  - schema got changed?",
diff --git a/solr/core/src/test/org/apache/solr/schema/TestSortableTextField.java b/solr/core/src/test/org/apache/solr/schema/TestSortableTextField.java
index 9cc1ac8..b4f8f76 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestSortableTextField.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestSortableTextField.java
@@ -53,34 +53,30 @@ public class TestSortableTextField extends SolrTestCaseJ4 {
   
   @BeforeClass
   public static void create() throws Exception {
-    initCore("solrconfig-minimal.xml","schema-sorting-text.xml");
-    
-    // sanity check our fields & types...
-
-    // these should all use docValues (either explicitly or implicitly)...
-    for (String n : Arrays.asList("keyword_stxt", 
-                                  "whitespace_stxt", "whitespace_f_stxt", "whitespace_l_stxt")) {
-           try (SolrCore core = h.getCore()) {
-             FieldType ft = core.getLatestSchema().getFieldTypeByName(n);
-             assertEquals("type " + ft.getTypeName() + " should have docvalues - schema got changed?", true, ft.getNamedPropertyValues(true).get("docValues"));
-           }
-    }
-    for (String n : Arrays.asList("keyword_stxt", "keyword_dv_stxt",
-                                  "whitespace_stxt", "whitespace_nois_stxt",
-                                  "whitespace_f_stxt", "whitespace_l_stxt")) {
-                                  
-      SchemaField sf = h.getCore().getLatestSchema().getField(n);
-      assertTrue("field " + sf.getName() + " should have docvalues - schema got changed?",
-                 sf.hasDocValues()) ;
-    }
+    initCore("solrconfig-minimal.xml", "schema-sorting-text.xml");
+    try (SolrCore core = h.getCore()) {
+      // sanity check our fields & types...
+
+      // these should all use docValues (either explicitly or implicitly)...
+      for (String n : Arrays.asList("keyword_stxt", "whitespace_stxt", "whitespace_f_stxt", "whitespace_l_stxt")) {
+
+        FieldType ft = core.getLatestSchema().getFieldTypeByName(n);
+        assertEquals("type " + ft.getTypeName() + " should have docvalues - schema got changed?", true, ft.getNamedPropertyValues(true).get("docValues"));
+
+      }
+      for (String n : Arrays.asList("keyword_stxt", "keyword_dv_stxt", "whitespace_stxt", "whitespace_nois_stxt", "whitespace_f_stxt", "whitespace_l_stxt")) {
+
+        SchemaField sf = core.getLatestSchema().getField(n);
+        assertTrue("field " + sf.getName() + " should have docvalues - schema got changed?", sf.hasDocValues());
+      }
+
+      { // this field should *NOT* have docValues .. should behave like a plain old TextField
 
-    { // this field should *NOT* have docValues .. should behave like a plain old TextField
-      try (SolrCore core = h.getCore()) {
         SchemaField sf = core.getLatestSchema().getField("whitespace_nodv_stxt");
         assertFalse("field " + sf.getName() + " should not have docvalues - schema got changed?", sf.hasDocValues());
+
       }
     }
-    
   }
   
   @Before
@@ -387,7 +383,8 @@ public class TestSortableTextField extends SolrTestCaseJ4 {
    */
   public void testUseDocValuesAsStored() throws Exception {
     ignoreException("when useDocValuesAsStored=true \\(length=");
-    
+    int num_fields_found = 0;
+    List<String> xpaths = new ArrayList<>(42);
     // first things first...
     // unlike most field types, SortableTextField should default to useDocValuesAsStored==false
     // (check a handful that should have the default behavior)
@@ -402,26 +399,20 @@ public class TestSortableTextField extends SolrTestCaseJ4 {
           assertEquals("field " + sf.getName() + " should not default to useDocValuesAsStored", false, sf.useDocValuesAsStored());
         }
       }
-    }
-    
-    // but it should be possible to set useDocValuesAsStored=true explicitly on types...
-    int num_types_found = 0;
-    for (Map.Entry<String,FieldType> entry : h.getCore().getLatestSchema().getFieldTypes().entrySet()) {
-      if (entry.getKey().endsWith("_has_usedvs")) {
-        num_types_found++;
-        FieldType ft = entry.getValue();
-        assertEquals("type " + ft.getTypeName() + " has unexpected useDocValuesAsStored value",
-                     true, ft.useDocValuesAsStored()) ;
+
+      // but it should be possible to set useDocValuesAsStored=true explicitly on types...
+      int num_types_found = 0;
+      for (Map.Entry<String,FieldType> entry : core.getLatestSchema().getFieldTypes().entrySet()) {
+        if (entry.getKey().endsWith("_has_usedvs")) {
+          num_types_found++;
+          FieldType ft = entry.getValue();
+          assertEquals("type " + ft.getTypeName() + " has unexpected useDocValuesAsStored value", true, ft.useDocValuesAsStored());
+        }
       }
-    }
-    assertEquals("sanity check: wrong number of *_has_usedvs types found -- schema changed?",
-                 2, num_types_found);
+      assertEquals("sanity check: wrong number of *_has_usedvs types found -- schema changed?", 2, num_types_found);
+
+      // ...and it should be possible to set/override useDocValuesAsStored=true on fields...
 
-    
-    // ...and it should be possible to set/override useDocValuesAsStored=true on fields...
-    int num_fields_found = 0;
-    List<String> xpaths = new ArrayList<>(42);
-    try (SolrCore core = h.getCore()) {
       for (Map.Entry<String,SchemaField> entry : core.getLatestSchema().getFields().entrySet()) {
         if (entry.getKey().endsWith("_usedvs")) {
           num_fields_found++;
@@ -466,6 +457,7 @@ public class TestSortableTextField extends SolrTestCaseJ4 {
           }
         }
       }
+
     }
     assertEquals("sanity check: wrong number of *_usedvs fields found -- schema changed?",
                  6, num_fields_found);
diff --git a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java
index 89a40a9..c4355bd 100644
--- a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java
@@ -24,6 +24,7 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.LazyDocument;
 import org.apache.lucene.index.IndexableField;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -45,22 +46,16 @@ public class LargeFieldTest extends SolrTestCaseJ4 {
     System.setProperty("enableLazyFieldLoading", "true");
 
     initCore("solrconfig-managed-schema.xml", "ignoredSchemaName");
-
-    // TODO SOLR-10229 will make this easier
-    boolean PERSIST_FALSE = false; // don't write to test resource dir
-    IndexSchema schema = h.getCore().getLatestSchema();
-    schema = schema.addFieldTypes(Collections.singletonList(
-        schema.newFieldType("textType", "solr.TextField", // redundant; TODO improve api
-            map("name", "textType",   "class", "solr.TextField",
-                "analyzer", map("class", "org.apache.lucene.analysis.standard.StandardAnalyzer")))),
-        PERSIST_FALSE);
-    schema = schema.addFields(Arrays.asList(
-        schema.newField(LAZY_FIELD, "textType", map()),
-        schema.newField(BIG_FIELD, "textType", map("large", true))),
-        Collections.emptyMap(),
-        PERSIST_FALSE);
-
-    h.getCore().setLatestSchema(schema);
+    try (SolrCore core = h.getCore()) {
+      // TODO SOLR-10229 will make this easier
+      boolean PERSIST_FALSE = false; // don't write to test resource dir
+      IndexSchema schema = core.getLatestSchema();
+      schema = schema.addFieldTypes(Collections.singletonList(schema.newFieldType("textType", "solr.TextField", // redundant; TODO improve api
+          map("name", "textType", "class", "solr.TextField", "analyzer", map("class", "org.apache.lucene.analysis.standard.StandardAnalyzer")))), PERSIST_FALSE);
+      schema = schema.addFields(Arrays.asList(schema.newField(LAZY_FIELD, "textType", map()), schema.newField(BIG_FIELD, "textType", map("large", true))), Collections.emptyMap(), PERSIST_FALSE);
+
+      core.setLatestSchema(schema);
+    }
   }
 
   @AfterClass
diff --git a/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java b/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java
index aaa5440..95a3f08 100644
--- a/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java
@@ -20,6 +20,7 @@ import java.io.File;
 import java.util.Collections;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.junit.Before;
@@ -52,39 +53,34 @@ public class TestAddFieldRealTimeGet extends TestRTGBase {
   }
 
   public void test() throws Exception {
-    clearIndex();
-    assertU(commit());
+    try (SolrCore core = h.getCore()) {
+      clearIndex();
+      assertU(commit());
 
-    String newFieldName = "newfield";
-    String newFieldType = "string";
-    String newFieldValue = "xyz";
+      String newFieldName = "newfield";
+      String newFieldType = "string";
+      String newFieldValue = "xyz";
 
-    ignoreException("unknown field");
-    assertFailedU("Should fail due to unknown field '" + newFieldName + "'", 
-                  adoc("id", "1", newFieldName, newFieldValue));
-    unIgnoreException("unknown field");
+      ignoreException("unknown field");
+      assertFailedU("Should fail due to unknown field '" + newFieldName + "'", adoc("id", "1", newFieldName, newFieldValue));
+      unIgnoreException("unknown field");
 
-    IndexSchema schema = h.getCore().getLatestSchema();
-    SchemaField newField = schema.newField(newFieldName, newFieldType, Collections.emptyMap());
-    IndexSchema newSchema = schema.addField(newField);
-    h.getCore().setLatestSchema(newSchema);
-    
-    String newFieldKeyValue = "'" + newFieldName + "':'" + newFieldValue + "'"; 
-    assertU(adoc("id", "1", newFieldName, newFieldValue));
-    assertJQ(req("q","id:1"), 
-             "/response/numFound==0");
-    assertJQ(req("qt","/get", "id","1", "fl","id,"+newFieldName),
-             "=={'doc':{'id':'1'," + newFieldKeyValue + "}}");
-    assertJQ(req("qt","/get","ids","1", "fl","id,"+newFieldName),
-             "=={'response':{'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
+      IndexSchema schema = core.getLatestSchema();
+      SchemaField newField = schema.newField(newFieldName, newFieldType, Collections.emptyMap());
+      IndexSchema newSchema = schema.addField(newField);
+      core.setLatestSchema(newSchema);
 
-    assertU(commit());
+      String newFieldKeyValue = "'" + newFieldName + "':'" + newFieldValue + "'";
+      assertU(adoc("id", "1", newFieldName, newFieldValue));
+      assertJQ(req("q", "id:1"), "/response/numFound==0");
+      assertJQ(req("qt", "/get", "id", "1", "fl", "id," + newFieldName), "=={'doc':{'id':'1'," + newFieldKeyValue + "}}");
+      assertJQ(req("qt", "/get", "ids", "1", "fl", "id," + newFieldName), "=={'response':{'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
 
-    assertJQ(req("q","id:1"), 
-             "/response/numFound==1");
-    assertJQ(req("qt","/get", "id","1", "fl","id,"+newFieldName),
-        "=={'doc':{'id':'1'," + newFieldKeyValue + "}}");
-    assertJQ(req("qt","/get","ids","1", "fl","id,"+newFieldName),
-        "=={'response':{'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
+      assertU(commit());
+
+      assertJQ(req("q", "id:1"), "/response/numFound==1");
+      assertJQ(req("qt", "/get", "id", "1", "fl", "id," + newFieldName), "=={'doc':{'id':'1'," + newFieldKeyValue + "}}");
+      assertJQ(req("qt", "/get", "ids", "1", "fl", "id," + newFieldName), "=={'response':{'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
+    }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java b/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java
index cc6b421..42775b4 100644
--- a/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java
+++ b/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java
@@ -37,7 +37,6 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.util.SolrPluginUtils;
 import org.junit.BeforeClass;
@@ -1523,8 +1522,8 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 {
     params.set("qf_fr", "subject_fr title_fr^5");
     params.set("qf_en", "subject_en title_en^5");
     params.set("qf_es", "subject_es title_es^5");
-    
-    MultilanguageQueryParser parser = new MultilanguageQueryParser("foo bar", new ModifiableSolrParams(), params, req(params));
+    SolrQueryRequest req = req(params);
+    MultilanguageQueryParser parser = new MultilanguageQueryParser("foo bar", new ModifiableSolrParams(), params, req);
     Query query = parser.parse();
     assertNotNull(query);
     assertTrue(containsClause(query, "title", "foo", 5, false));
@@ -1533,21 +1532,25 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 {
     assertTrue(containsClause(query, "subject", "bar", 1, false));
     
     params.set("language", "es");
-    parser = new MultilanguageQueryParser("foo bar", new ModifiableSolrParams(), params, req(params));
+    req.close();
+    req = req(params);
+    parser = new MultilanguageQueryParser("foo bar", new ModifiableSolrParams(), params, req);
     query = parser.parse();
     assertNotNull(query);
     assertTrue(containsClause(query, "title_es", "foo", 5, false));
     assertTrue(containsClause(query, "title_es", "bar", 5, false));
     assertTrue(containsClause(query, "subject_es", "foo", 1, false));
     assertTrue(containsClause(query, "subject_es", "bar", 1, false));
-    
-    FuzzyDismaxQParser parser2 = new FuzzyDismaxQParser("foo bar absence", new ModifiableSolrParams(), params, req(params));
+
+    req.close();
+    req = req(params);
+    FuzzyDismaxQParser parser2 = new FuzzyDismaxQParser("foo bar absence", new ModifiableSolrParams(), params, req);
     query = parser2.parse();
     assertNotNull(query);
     assertTrue(containsClause(query, "title", "foo", 5, false));
     assertTrue(containsClause(query, "title", "bar", 5, false));
     assertTrue(containsClause(query, "title", "absence", 5, true));
-    
+    req.close();
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java
index 24ab040..cf3e760 100644
--- a/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java
+++ b/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java
@@ -78,7 +78,7 @@ public class TestHashQParserPlugin extends SolrTestCaseJ4 {
     params.add("partitionKeys", "a_i,a_s,a_i,a_s,a_i");
     params.add("wt", "xml");
     ModifiableSolrParams finalParams = params;
-    expectThrows(SolrException.class, () -> h.query(req(finalParams)));
+    expectThrows(SolrException.class, () -> query(req(finalParams)));
   }
 
   @Test
@@ -89,7 +89,7 @@ public class TestHashQParserPlugin extends SolrTestCaseJ4 {
     params.add("partitionKeys", "a_i");
     params.add("wt", "xml");
     ModifiableSolrParams finalParams = params;
-    expectThrows(SolrException.class, () -> h.query(req(finalParams)));
+    expectThrows(SolrException.class, () -> query(req(finalParams)));
   }
 
   @Test
@@ -107,7 +107,7 @@ public class TestHashQParserPlugin extends SolrTestCaseJ4 {
     params.add("fq", "{!hash worker=0 workers=2 cost="+getCost(random())+"}");
     params.add("partitionKeys", "a_s");
     params.add("wt", "xml");
-    String response = h.query(req(params));
+    String response = query(req(params));
     try (SolrCore core = h.getCore()) {
       h.validateXPath(core.getResourceLoader(), response, "//*[@numFound='4']");
     }
@@ -118,7 +118,7 @@ public class TestHashQParserPlugin extends SolrTestCaseJ4 {
     params.add("fq", "{!hash worker=0 workers=2 cost="+getCost(random())+"}");
     params.add("partitionKeys", "a_i");
     params.add("wt", "xml");
-    response = h.query(req(params));
+    response = query(req(params));
     try (SolrCore core = h.getCore()) {
       h.validateXPath(core.getResourceLoader(), response, "//*[@numFound='4']");
     }
@@ -156,7 +156,7 @@ public class TestHashQParserPlugin extends SolrTestCaseJ4 {
     params.add("rows","50");
     params.add("wt", "xml");
     HashSet set1 = new HashSet();
-    String response = h.query(req(params));
+    String response = query(req(params));
 
     Iterator<String> it = set.iterator();
 
@@ -177,7 +177,7 @@ public class TestHashQParserPlugin extends SolrTestCaseJ4 {
     params.add("rows","50");
     params.add("wt", "xml");
     HashSet set2 = new HashSet();
-    response = h.query(req(params));
+    response = query(req(params));
 
     it = set.iterator();
 
diff --git a/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java b/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
index 71ef5c5..cd041ec 100644
--- a/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
+++ b/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
@@ -138,7 +138,9 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
     int baseRefCount = r3.getRefCount();
     assertEquals(1, baseRefCount);
 
-    Map<String, Metric> metrics = h.getCore().getCoreMetricManager().getRegistry().getMetrics();
+    SolrCore core = h.getCore();
+    Map<String, Metric> metrics = core.getCoreMetricManager().getRegistry().getMetrics();
+    core.close();
     Gauge<Date> g = (Gauge<Date>)metrics.get("SEARCHER.searcher.registeredAt");
     Date sr3SearcherRegAt = g.getValue();
     assertU(commit()); // nothing has changed
@@ -229,13 +231,14 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
   private void createCoreAndValidateListeners(int numTimesCalled, int numTimesCalledFirstSearcher,
       int numTimesCalledAfterGetSearcher, int numTimesCalledFirstSearcherAfterGetSearcher) throws Exception {
     CoreContainer cores = h.getCoreContainer();
-    CoreDescriptor cd = h.getCore().getCoreDescriptor();
+
     SolrCore newCore = null;
     // reset counters
     MockSearcherListener.numberOfTimesCalled = new AtomicInteger();
     MockSearcherListener.numberOfTimesCalledFirstSearcher = new AtomicInteger();
-    
-    try {
+
+    try (SolrCore core = h.getCore()) {
+      CoreDescriptor cd = core.getCoreDescriptor();
       // Create a new core, this should call all the firstSearcherListeners
       newCore = cores.create("core1", cd.getInstanceDir(), ImmutableMap.of("config", "solrconfig-searcher-listeners1.xml"), false);
 
@@ -243,22 +246,22 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
       assertNotNull(newCore.getSearchComponent("mock"));
       assertEquals(MockSearchComponent.class, newCore.getSearchComponent("mock").getClass());
       assertFalse(newCore.getSolrConfig().useColdSearcher);
-      
+
       doQuery(newCore);
-      
+
       assertEquals(numTimesCalled, MockSearcherListener.numberOfTimesCalled.get());
       assertEquals(numTimesCalledFirstSearcher, MockSearcherListener.numberOfTimesCalledFirstSearcher.get());
-      
+
       addDummyDoc(newCore);
-      
+
       // Open a new searcher, this should call the newSearcherListeners
       Future<?>[] future = new Future[1];
       newCore.getSearcher(true, false, future);
       future[0].get();
-      
+
       assertEquals(numTimesCalledAfterGetSearcher, MockSearcherListener.numberOfTimesCalled.get());
       assertEquals(numTimesCalledFirstSearcherAfterGetSearcher, MockSearcherListener.numberOfTimesCalledFirstSearcher.get());
-      
+
     } finally {
       if (newCore != null) {
         cores.unload("core1");
@@ -283,10 +286,11 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
     SlowSearcherListener.latch = new CountDownLatch(1);
     
     CoreContainer cores = h.getCoreContainer();
-    CoreDescriptor cd = h.getCore().getCoreDescriptor();
+
     final SolrCore newCore;
     boolean coreCreated = false;
-    try {
+    try (SolrCore core = h.getCore()) {
+      CoreDescriptor cd = core.getCoreDescriptor();
       // Create a new core, this should call all the firstSearcherListeners
       newCore = cores.create("core1", cd.getInstanceDir(), ImmutableMap.of("config", "solrconfig-searcher-listeners1.xml"), false);
       coreCreated = true;
@@ -345,23 +349,23 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
     final AtomicBoolean querySucceeded = new AtomicBoolean(false);
     SlowSearcherListener.numberOfTimesCalled = new AtomicInteger(0);
     SlowSearcherListener.latch = new CountDownLatch(1);
-    
-    
+
     CoreContainer cores = h.getCoreContainer();
-    CoreDescriptor cd = h.getCore().getCoreDescriptor();
+
     final SolrCore newCore;
     boolean coreCreated = false;
-    try {
+    try (SolrCore core = h.getCore()) {
+      CoreDescriptor cd = core.getCoreDescriptor();
       System.setProperty("tests.solr.useColdSearcher", "true");
       // Create a new core, this should call all the firstSearcherListeners
       newCore = cores.create("core1", cd.getInstanceDir(), ImmutableMap.of("config", "solrconfig-searcher-listeners1.xml"), false);
       coreCreated = true;
-      
+
       //validate that the new core was created with the correct solrconfig
       assertNotNull(newCore.getSearchComponent("mock"));
       assertEquals(MockSearchComponent.class, newCore.getSearchComponent("mock").getClass());
       assertTrue(newCore.getSolrConfig().useColdSearcher);
-      
+
       Thread t = new Thread() {
         public void run() {
           try {
@@ -370,10 +374,12 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
           } catch (Exception e) {
             throw new RuntimeException(e);
           }
-        };
+        }
+
+        ;
       };
       t.start();
-      
+
       // validate that the query runs before the searcher warmer finishes
       for (int i = 0; i <= 1000; i++) {
         if (querySucceeded.get()) {
@@ -384,16 +390,16 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
         }
         Thread.sleep(10);
       }
-      
+
       assertEquals(0, SlowSearcherListener.numberOfTimesCalled.get());
-      
+
     } finally {
       System.getProperties().remove("tests.solr.useColdSearcher");
       if (coreCreated) {
         SlowSearcherListener.latch.countDown();
         cores.unload("core1");
       }
-      
+
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/search/TestMultiWordSynonyms.java b/solr/core/src/test/org/apache/solr/search/TestMultiWordSynonyms.java
index 4d9feed..2f78b11 100644
--- a/solr/core/src/test/org/apache/solr/search/TestMultiWordSynonyms.java
+++ b/solr/core/src/test/org/apache/solr/search/TestMultiWordSynonyms.java
@@ -21,6 +21,7 @@ import java.util.Arrays;
 
 import org.apache.lucene.search.Query;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.request.SolrQueryRequest;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -238,22 +239,28 @@ public class TestMultiWordSynonyms extends SolrTestCaseJ4 {
   }
 
   public void testEdismaxQueryParsing_multiTermWithPf_shouldParseCorrectPhraseQueries() throws Exception {
+    SolrQueryRequest req = req(params("sow", "false", "qf", "text^10", "pf", "text^10", "pf2", "text^5", "pf3", "text^8"));
     Query q = QParser.getParser("foo a b bar","edismax",true,
-        req(params("sow", "false","qf", "text^10","pf", "text^10","pf2", "text^5","pf3", "text^8"))).getQuery();
+       req).getQuery();
+    req.close();
     assertEquals("+(" +
         "((text:foo)^10.0) ((text:a)^10.0) ((text:b)^10.0) (((+text:tropical +text:cyclone) text:bar)^10.0)) " +
         "((text:\"foo a b tropical cyclone\" text:\"foo a b bar\")^10.0) " +
         "(((text:\"foo a\")^5.0) ((text:\"a b\")^5.0) ((text:\"b tropical cyclone\" text:\"b bar\")^5.0)) " +
         "(((text:\"foo a b\")^8.0) ((text:\"a b tropical cyclone\" text:\"a b bar\")^8.0))", q.toString());
 
-    q = QParser.getParser("tropical cyclone foo a b ","edismax",true, req(params("qf", "text^10","pf", "text^10","pf2", "text^5","pf3", "text^8"))).getQuery();
+    req = req(params("qf", "text^10","pf", "text^10","pf2", "text^5","pf3", "text^8"));
+    q = QParser.getParser("tropical cyclone foo a b ","edismax",true, req).getQuery();
+    req.close();
     assertEquals("+(" +
         "((text:bar (+text:tropical +text:cyclone))^10.0) ((text:foo)^10.0) ((text:a)^10.0) ((text:b)^10.0)) " +
         "((text:\"bar foo a b\" text:\"tropical cyclone foo a b\")^10.0) " +
         "(((text:bar text:\"tropical cyclone\")^5.0) ((text:\"cyclone foo\")^5.0) ((text:\"foo a\")^5.0) ((text:\"a b\")^5.0)) " +
         "(((text:\"bar foo\" text:\"tropical cyclone foo\")^8.0) ((text:\"cyclone foo a\")^8.0) ((text:\"foo a b\")^8.0))", q.toString());
 
-    q = QParser.getParser("foo a b tropical cyclone","edismax",true, req(params("qf", "text^10","pf", "text^10","pf2", "text^5","pf3", "text^8"))).getQuery();
+    req = req(params("qf", "text^10","pf", "text^10","pf2", "text^5","pf3", "text^8"));
+    q = QParser.getParser("foo a b tropical cyclone","edismax",true, req).getQuery();
+    req.close();
     assertEquals("+(" +
         "((text:foo)^10.0) ((text:a)^10.0) ((text:b)^10.0) ((text:bar (+text:tropical +text:cyclone))^10.0)) " +
         "((text:\"foo a b bar\" text:\"foo a b tropical cyclone\")^10.0) " +
diff --git a/solr/core/src/test/org/apache/solr/search/TestNoOpRegenerator.java b/solr/core/src/test/org/apache/solr/search/TestNoOpRegenerator.java
index 72af2cd..1db7d76 100644
--- a/solr/core/src/test/org/apache/solr/search/TestNoOpRegenerator.java
+++ b/solr/core/src/test/org/apache/solr/search/TestNoOpRegenerator.java
@@ -17,6 +17,7 @@
 package org.apache.solr.search;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
 import org.junit.BeforeClass;
 
 /** Tests that NoOpRegenerator does what it should */
@@ -32,29 +33,30 @@ public class TestNoOpRegenerator extends SolrTestCaseJ4 {
     assertU(adoc("id", "1"));
     assertU(adoc("id", "2"));
     assertU(commit());
-    
-    // add some items
-    h.getCore().withSearcher(searcher -> {
-      assertEquals(2, searcher.maxDoc());
-      SolrCache<Object,Object> cache = searcher.getCache("myPerSegmentCache");
-      assertEquals(0, cache.size());
-      cache.put("key1", "value1");
-      cache.put("key2", "value2");
-      assertEquals(2, cache.size());
-      return null;
-    });
-    
-    // add a doc and commit: we should see our cached items still there
-    assertU(adoc("id", "3"));
-    assertU(commit());
+    try (SolrCore core = h.getCore()) {
+      // add some items
+      core.withSearcher(searcher -> {
+        assertEquals(2, searcher.maxDoc());
+        SolrCache<Object,Object> cache = searcher.getCache("myPerSegmentCache");
+        assertEquals(0, cache.size());
+        cache.put("key1", "value1");
+        cache.put("key2", "value2");
+        assertEquals(2, cache.size());
+        return null;
+      });
+
+      // add a doc and commit: we should see our cached items still there
+      assertU(adoc("id", "3"));
+      assertU(commit());
 
-    h.getCore().withSearcher(searcher -> {
-      assertEquals(3, searcher.maxDoc());
-      SolrCache<Object,Object> cache = searcher.getCache("myPerSegmentCache");
-      assertEquals(2, cache.size());
-      assertEquals("value1", cache.get("key1"));
-      assertEquals("value2", cache.get("key2"));
-      return null;
-    });
+      core.withSearcher(searcher -> {
+        assertEquals(3, searcher.maxDoc());
+        SolrCache<Object,Object> cache = searcher.getCache("myPerSegmentCache");
+        assertEquals(2, cache.size());
+        assertEquals("value1", cache.get("key1"));
+        assertEquals("value2", cache.get("key2"));
+        return null;
+      });
+    }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/TestPseudoReturnFields.java b/solr/core/src/test/org/apache/solr/search/TestPseudoReturnFields.java
index e56f265..1e898ac 100644
--- a/solr/core/src/test/org/apache/solr/search/TestPseudoReturnFields.java
+++ b/solr/core/src/test/org/apache/solr/search/TestPseudoReturnFields.java
@@ -23,6 +23,7 @@ import java.util.Random;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.cloud.TestCloudPseudoReturnFields;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.SchemaField;
 
 import org.apache.solr.common.params.SolrParams;
@@ -77,12 +78,12 @@ public class TestPseudoReturnFields extends SolrTestCaseJ4 {
     // if the type lookup is done again later, we get the wrong thing). SOLR-4036
 
     // score as psuedo field - precondition checks
-    for (String name : new String[] {"score", "val_ss"}) {
-      SchemaField sf = h.getCore().getLatestSchema().getFieldOrNull(name);
-      assertNotNull("Test depends on a (dynamic) field mtching '"+name+
-                    "', schema was changed out from under us!",sf);
-      assertTrue("Test depends on a multivalued dynamic field matching '"+name+
-                 "', schema was changed out from under us!", sf.multiValued());
+    try (SolrCore core = h.getCore()) {
+      for (String name : new String[] {"score", "val_ss"}) {
+        SchemaField sf = core.getLatestSchema().getFieldOrNull(name);
+        assertNotNull("Test depends on a (dynamic) field mtching '" + name + "', schema was changed out from under us!", sf);
+        assertTrue("Test depends on a multivalued dynamic field matching '" + name + "', schema was changed out from under us!", sf.multiValued());
+      }
     }
 
     // score as psuedo field
diff --git a/solr/core/src/test/org/apache/solr/search/TestReRankQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestReRankQParserPlugin.java
index de59c53..544d71c 100644
--- a/solr/core/src/test/org/apache/solr/search/TestReRankQParserPlugin.java
+++ b/solr/core/src/test/org/apache/solr/search/TestReRankQParserPlugin.java
@@ -21,6 +21,7 @@ import java.util.Map;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.metrics.MetricsMap;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -381,8 +382,9 @@ public class TestReRankQParserPlugin extends SolrTestCaseJ4 {
         "//result/doc[4]/str[@name='id'][.='3']",
         "//result/doc[5]/str[@name='id'][.='2']"
     );
-
-    MetricsMap metrics = (MetricsMap) h.getCore().getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.queryResultCache");
+    SolrCore core = h.getCore();
+    MetricsMap metrics = (MetricsMap) core.getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.queryResultCache");
+    core.close();
     Map<String,Object> stats = metrics.getValue(true, false);
 
     long inserts = (Long) stats.get("inserts");
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
index 9c0e43a..b99b4a1 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
@@ -1353,22 +1353,27 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
   }
     
   public void testSynonymsBoost_edismaxBoost_shouldParseBoostedPhraseQuery() throws Exception {
-    Query q = QParser.getParser("snow leopard lion","edismax",true, req(params("sow", "false","qf", "t_pick_best_boosted_foo^10"))).getQuery();
+    SolrQueryRequest req = req(params("sow", "false", "qf", "t_pick_best_boosted_foo^10"));
+    Query q = QParser.getParser("snow leopard lion","edismax",true, req).getQuery();
     assertEquals("+(" +
         "((((t_pick_best_boosted_foo:\"panthera uncia\")^0.9 | (t_pick_best_boosted_foo:\"big cat\")^0.8 | (t_pick_best_boosted_foo:white_leopard)^0.6 | t_pick_best_boosted_foo:\"snow leopard\"))^10.0)" +
         " ((((t_pick_best_boosted_foo:\"panthera leo\")^0.9 | (t_pick_best_boosted_foo:\"simba leo\")^0.8 | (t_pick_best_boosted_foo:kimba)^0.75))^10.0)" +
         ")", q.toString());
 
-    q = QParser.getParser("snow leopard lion","edismax",true, req(params("sow", "false","qf", "t_as_distinct_boosted_foo^10"))).getQuery();
+    req.close();
+    req = req(params("sow", "false","qf", "t_as_distinct_boosted_foo^10"));
+        q = QParser.getParser("snow leopard lion","edismax",true, req).getQuery();
     assertEquals("+(" +
         "(((t_as_distinct_boosted_foo:\"panthera uncia\")^0.9 (t_as_distinct_boosted_foo:\"big cat\")^0.8 (t_as_distinct_boosted_foo:white_leopard)^0.6 t_as_distinct_boosted_foo:\"snow leopard\")^10.0)" +
         " (((t_as_distinct_boosted_foo:\"panthera leo\")^0.9 (t_as_distinct_boosted_foo:\"simba leo\")^0.8 (t_as_distinct_boosted_foo:kimba)^0.75)^10.0))", q.toString());
 
-    q = QParser.getParser("snow leopard lion","edismax",true, req(params("sow", "false","qf", "t_as_same_term_boosted_foo^10"))).getQuery();
+    req.close();
+    req = req(params("sow", "false","qf", "t_as_same_term_boosted_foo^10"));
+        q = QParser.getParser("snow leopard lion","edismax",true, req).getQuery();
     assertEquals("+(" +
             "(((t_as_same_term_boosted_foo:\"panthera uncia\")^0.9 (t_as_same_term_boosted_foo:\"big cat\")^0.8 (t_as_same_term_boosted_foo:white_leopard)^0.6 t_as_same_term_boosted_foo:\"snow leopard\")^10.0)" +
             " (((t_as_same_term_boosted_foo:\"panthera leo\")^0.9 (t_as_same_term_boosted_foo:\"simba leo\")^0.8 (t_as_same_term_boosted_foo:kimba)^0.75)^10.0))", q.toString());
-
+    req.close();
   }
 
   @AwaitsFix(bugUrl = "nocommit - review difference")
@@ -1461,40 +1466,44 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
 
     for (String existenceQuery : existenceQueries) {
       for (String suffix : fieldSuffix) {
-        try (SolrCore core = h.getCore()) {
-          IndexSchema indexSchema = core.getLatestSchema();
-          String field = "foo_" + suffix;
-          String query = field + ":" + existenceQuery;
-          SolrQueryRequest req = req();
-          QParser qParser = QParser.getParser(query, req);
-          req.close();
-          Query createdQuery = qParser.getQuery();
-          SchemaField schemaField = indexSchema.getField(field);
-
-          // Test float & double realNumber queries differently
-          if ("[* TO *]".equals(existenceQuery) && (schemaField.getType().getNumberType() == NumberType.DOUBLE || schemaField.getType().getNumberType() == NumberType.FLOAT)) {
-            assertFalse("For float and double fields \"" + query + "\" is not an existence query, so the query returned should not be a DocValuesFieldExistsQuery.", createdQuery instanceof DocValuesFieldExistsQuery);
-            assertFalse("For float and double fields \"" + query + "\" is not an existence query, so the query returned should not be a NormsFieldExistsQuery.", createdQuery instanceof NormsFieldExistsQuery);
-            assertFalse("For float and double fields \"" + query + "\" is not an existence query, so NaN should not be matched via a ConstantScoreQuery.", createdQuery instanceof ConstantScoreQuery);
-            assertFalse("For float and double fields\"" + query + "\" is not an existence query, so NaN should not be matched via a BooleanQuery (NaN and [* TO *]).", createdQuery instanceof BooleanQuery);
+
+        String field = "foo_" + suffix;
+        String query = field + ":" + existenceQuery;
+        SolrQueryRequest req = req();
+        IndexSchema indexSchema = req.getCore().getLatestSchema();
+        QParser qParser = QParser.getParser(query, req);
+        req.close();
+        Query createdQuery = qParser.getQuery();
+        SchemaField schemaField = indexSchema.getField(field);
+
+        // Test float & double realNumber queries differently
+        if ("[* TO *]".equals(existenceQuery) && (schemaField.getType().getNumberType() == NumberType.DOUBLE || schemaField.getType().getNumberType() == NumberType.FLOAT)) {
+          assertFalse("For float and double fields \"" + query + "\" is not an existence query, so the query returned should not be a DocValuesFieldExistsQuery.",
+              createdQuery instanceof DocValuesFieldExistsQuery);
+          assertFalse("For float and double fields \"" + query + "\" is not an existence query, so the query returned should not be a NormsFieldExistsQuery.",
+              createdQuery instanceof NormsFieldExistsQuery);
+          assertFalse("For float and double fields \"" + query + "\" is not an existence query, so NaN should not be matched via a ConstantScoreQuery.", createdQuery instanceof ConstantScoreQuery);
+          assertFalse("For float and double fields\"" + query + "\" is not an existence query, so NaN should not be matched via a BooleanQuery (NaN and [* TO *]).",
+              createdQuery instanceof BooleanQuery);
+        } else {
+          if (schemaField.hasDocValues()) {
+            assertTrue("Field has docValues, so existence query \"" + query + "\" should return DocValuesFieldExistsQuery", createdQuery instanceof DocValuesFieldExistsQuery);
+          } else if (!schemaField.omitNorms() && !schemaField.getType().isPointField()) { //TODO: Remove !isPointField() for SOLR-14199
+            assertTrue("Field has norms and no docValues, so existence query \"" + query + "\" should return NormsFieldExistsQuery", createdQuery instanceof NormsFieldExistsQuery);
+          } else if (schemaField.getType().getNumberType() == NumberType.DOUBLE || schemaField.getType().getNumberType() == NumberType.FLOAT) {
+            assertTrue("PointField with NaN values must include \"exists or NaN\" if the field doesn't have norms or docValues: \"" + query + "\".", createdQuery instanceof ConstantScoreQuery);
+            assertTrue("PointField with NaN values must include \"exists or NaN\" if the field doesn't have norms or docValues: \"" + query + "\".",
+                ((ConstantScoreQuery) createdQuery).getQuery() instanceof BooleanQuery);
+            assertEquals("PointField with NaN values must include \"exists or NaN\" if the field doesn't have norms or docValues: \"" + query + "\". This boolean query must be an OR.", 1,
+                ((BooleanQuery) ((ConstantScoreQuery) createdQuery).getQuery()).getMinimumNumberShouldMatch());
+            assertEquals("PointField with NaN values must include \"exists or NaN\" if the field doesn't have norms or docValues: \"" + query + "\". This boolean query must have 2 clauses.", 2,
+                ((BooleanQuery) ((ConstantScoreQuery) createdQuery).getQuery()).clauses().size());
           } else {
-            if (schemaField.hasDocValues()) {
-              assertTrue("Field has docValues, so existence query \"" + query + "\" should return DocValuesFieldExistsQuery", createdQuery instanceof DocValuesFieldExistsQuery);
-            } else if (!schemaField.omitNorms() && !schemaField.getType().isPointField()) { //TODO: Remove !isPointField() for SOLR-14199
-              assertTrue("Field has norms and no docValues, so existence query \"" + query + "\" should return NormsFieldExistsQuery", createdQuery instanceof NormsFieldExistsQuery);
-            } else if (schemaField.getType().getNumberType() == NumberType.DOUBLE || schemaField.getType().getNumberType() == NumberType.FLOAT) {
-              assertTrue("PointField with NaN values must include \"exists or NaN\" if the field doesn't have norms or docValues: \"" + query + "\".", createdQuery instanceof ConstantScoreQuery);
-              assertTrue("PointField with NaN values must include \"exists or NaN\" if the field doesn't have norms or docValues: \"" + query + "\".", ((ConstantScoreQuery) createdQuery).getQuery() instanceof BooleanQuery);
-              assertEquals("PointField with NaN values must include \"exists or NaN\" if the field doesn't have norms or docValues: \"" + query + "\". This boolean query must be an OR.", 1,
-                  ((BooleanQuery) ((ConstantScoreQuery) createdQuery).getQuery()).getMinimumNumberShouldMatch());
-              assertEquals("PointField with NaN values must include \"exists or NaN\" if the field doesn't have norms or docValues: \"" + query + "\". This boolean query must have 2 clauses.", 2,
-                  ((BooleanQuery) ((ConstantScoreQuery) createdQuery).getQuery()).clauses().size());
-            } else {
-              assertFalse("Field doesn't have docValues, so existence query \"" + query + "\" should not return DocValuesFieldExistsQuery", createdQuery instanceof DocValuesFieldExistsQuery);
-              assertFalse("Field doesn't have norms, so existence query \"" + query + "\" should not return NormsFieldExistsQuery", createdQuery instanceof NormsFieldExistsQuery);
-            }
+            assertFalse("Field doesn't have docValues, so existence query \"" + query + "\" should not return DocValuesFieldExistsQuery", createdQuery instanceof DocValuesFieldExistsQuery);
+            assertFalse("Field doesn't have norms, so existence query \"" + query + "\" should not return NormsFieldExistsQuery", createdQuery instanceof NormsFieldExistsQuery);
           }
         }
+
       }
     }
   }
diff --git a/solr/core/src/test/org/apache/solr/search/TestTrieFacet.java b/solr/core/src/test/org/apache/solr/search/TestTrieFacet.java
index ca0fd71..31d7b4e 100644
--- a/solr/core/src/test/org/apache/solr/search/TestTrieFacet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestTrieFacet.java
@@ -18,6 +18,7 @@ package org.apache.solr.search;
 
 import org.apache.lucene.util.TestUtil;
 
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.TrieIntField;
 import org.apache.solr.common.SolrInputDocument;
@@ -30,7 +31,8 @@ public class TestTrieFacet extends SolrTestCaseJ4 {
 
   final static int MIN_VALUE = 20;
   final static int MAX_VALUE = 60;
-  
+
+
   final static String TRIE_INT_P8_S_VALUED = "foo_ti1";
   final static String TRIE_INT_P8_M_VALUED = "foo_ti";
   
@@ -57,69 +59,68 @@ public class TestTrieFacet extends SolrTestCaseJ4 {
     if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
 
     initCore("solrconfig-tlog.xml","schema.xml");
-
-    // don't break the test
-    assertTrue("min value must be less then max value", MIN_VALUE < MAX_VALUE);
-    assertTrue("min value must be greater then zero", 0 < MIN_VALUE);
-    
-    // sanity check no one breaks the schema out from under us...
-    for (String f : M_VALUED) {
-      SchemaField sf = h.getCore().getLatestSchema().getField(f);
-      assertTrue("who changed the schema? test isn't valid: " + f, sf.multiValued());
-    }
-    
-    for (String f : S_VALUED) {
-      SchemaField sf = h.getCore().getLatestSchema().getField(f);
-      assertFalse("who changed the schema? test isn't valid: " + f, sf.multiValued());
-    }
-
-    if (! Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) {
-      for (String f : P0) {
-        SchemaField sf = h.getCore().getLatestSchema().getField(f);
-        assertEquals("who changed the schema? test isn't valid: " + f,
-                     0, assertCastFieldType(sf).getPrecisionStep());
+    try (SolrCore core = h.getCore()) {
+      // don't break the test
+      assertTrue("min value must be less then max value", MIN_VALUE < MAX_VALUE);
+      assertTrue("min value must be greater then zero", 0 < MIN_VALUE);
+
+      // sanity check no one breaks the schema out from under us...
+      for (String f : M_VALUED) {
+        SchemaField sf = core.getLatestSchema().getField(f);
+        assertTrue("who changed the schema? test isn't valid: " + f, sf.multiValued());
       }
-      for (String f : P8) {
-        SchemaField sf = h.getCore().getLatestSchema().getField(f);
-        assertEquals("who changed the schema? test isn't valid: " + f,
-                     8, assertCastFieldType(sf).getPrecisionStep());
+
+      for (String f : S_VALUED) {
+        SchemaField sf = core.getLatestSchema().getField(f);
+        assertFalse("who changed the schema? test isn't valid: " + f, sf.multiValued());
       }
-    }
-    
-    // we don't need a lot of docs -- at least one failure only had ~1000  
-    NUM_DOCS = TestUtil.nextInt(random(), 200, 1500);
-
-    { // ensure at least one doc has every valid value in the multivalued fields
-      SolrInputDocument doc = sdoc("id", "0");
-      for (int val = MIN_VALUE; val <= MAX_VALUE; val++) {
-        for (String f : M_VALUED) {
-          doc.addField(f, val);
+
+      if (!Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) {
+        for (String f : P0) {
+          SchemaField sf = core.getLatestSchema().getField(f);
+          assertEquals("who changed the schema? test isn't valid: " + f, 0, assertCastFieldType(sf).getPrecisionStep());
+        }
+        for (String f : P8) {
+          SchemaField sf = core.getLatestSchema().getField(f);
+          assertEquals("who changed the schema? test isn't valid: " + f, 8, assertCastFieldType(sf).getPrecisionStep());
         }
       }
-      assertU(adoc(doc));
-    }
 
-    // randomized docs (note: starting at i=1)
-    for (int i=1; i < NUM_DOCS; i++) {
-      SolrInputDocument doc = sdoc("id", i+"");
-      if (useField()) {
-        int val = TestUtil.nextInt(random(), MIN_VALUE, MAX_VALUE);
-        for (String f : S_VALUED) {
-          doc.addField(f, val);
+      // we don't need a lot of docs -- at least one failure only had ~1000
+      NUM_DOCS = TestUtil.nextInt(random(), 200, 1500);
+
+      { // ensure at least one doc has every valid value in the multivalued fields
+        SolrInputDocument doc = sdoc("id", "0");
+        for (int val = MIN_VALUE; val <= MAX_VALUE; val++) {
+          for (String f : M_VALUED) {
+            doc.addField(f, val);
+          }
         }
+        assertU(adoc(doc));
       }
-      if (useField()) {
-        int numMulti = atLeast(1);
-        while (0 < numMulti--) {
+
+      // randomized docs (note: starting at i=1)
+      for (int i = 1; i < NUM_DOCS; i++) {
+        SolrInputDocument doc = sdoc("id", i + "");
+        if (useField()) {
           int val = TestUtil.nextInt(random(), MIN_VALUE, MAX_VALUE);
-          for (String f: M_VALUED) {
+          for (String f : S_VALUED) {
             doc.addField(f, val);
           }
         }
+        if (useField()) {
+          int numMulti = atLeast(1);
+          while (0 < numMulti--) {
+            int val = TestUtil.nextInt(random(), MIN_VALUE, MAX_VALUE);
+            for (String f : M_VALUED) {
+              doc.addField(f, val);
+            }
+          }
+        }
+        assertU(adoc(doc));
       }
-      assertU(adoc(doc));
+      assertU(commit());
     }
-    assertU(commit());
   }
 
   /** 
diff --git a/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java b/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
index 73e834e..19447a9 100644
--- a/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
+++ b/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
@@ -33,6 +33,7 @@ import org.apache.lucene.search.join.ScoreMode;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.QParser;
@@ -225,7 +226,7 @@ public class BJQParserTest extends SolrTestCaseJ4 {
     try (SolrQueryRequest req = req("q",query, "sort","score asc", "fl","score")) {
       final String resp = h.query(req);
       return (String) h.
-          evaluateXPath(h.getCore().getResourceLoader(), resp, "(//float[@name='score'])[1]/text()", XPathConstants.STRING);
+          evaluateXPath(req.getCore().getResourceLoader(), resp, "(//float[@name='score'])[1]/text()", XPathConstants.STRING);
     }
   }
 
@@ -288,10 +289,10 @@ public class BJQParserTest extends SolrTestCaseJ4 {
 
   @Test
   public void testCacheHit() throws IOException {
-
-    MetricsMap parentFilterCache = (MetricsMap)(h.getCore().getCoreMetricManager().getRegistry()
+    SolrCore core = h.getCore();
+    MetricsMap parentFilterCache = (MetricsMap)(core.getCoreMetricManager().getRegistry()
         .getMetrics().get("CACHE.searcher.perSegFilter"));
-    MetricsMap filterCache = (MetricsMap)(h.getCore().getCoreMetricManager().getRegistry()
+    MetricsMap filterCache = (MetricsMap)(core.getCoreMetricManager().getRegistry()
         .getMetrics().get("CACHE.searcher.filterCache"));
 
     Map<String,Object> parentsBefore = parentFilterCache.getValue();
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
index 9ad20bf..cac50e8 100644
--- a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
+++ b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
@@ -30,6 +30,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.join.ScoreMode;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
@@ -266,7 +267,9 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
     // however it might be better to extract this method into the separate suite
     // for a while let's nuke a cache content, in case of repetitions
     @SuppressWarnings("rawtypes")
-    SolrCache cache = (SolrCache)h.getCore().getInfoRegistry().get("queryResultCache");
+    SolrCore core = h.getCore();
+    SolrCache cache = (SolrCache)core.getInfoRegistry().get("queryResultCache");
+    core.close();
     cache.clear();
   }
 
diff --git a/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java b/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java
index 850bc2d..af9fd9d 100644
--- a/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java
+++ b/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java
@@ -22,14 +22,15 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
 
 public abstract class BaseSimilarityTestCase extends SolrTestCaseJ4 {
 
   /** returns the similarity in use for the field */
   protected Similarity getSimilarity(String field) {
     Similarity sim = null;
-    try {
-      sim = h.getCore().withSearcher(IndexSearcher::getSimilarity);
+    try (SolrCore core = h.getCore()) {
+      sim = core.withSearcher(IndexSearcher::getSimilarity);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
diff --git a/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java b/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java
index 5252d4b..301f73f 100644
--- a/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java
+++ b/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java
@@ -20,12 +20,15 @@ import java.net.URLEncoder;
 
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.core.SolrCore;
 import org.junit.BeforeClass;
 
 
 public class DirectSolrConnectionTest extends SolrTestCaseJ4 {
 
-  
+  private SolrCore core;
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     initCore("solr/crazy-path-to-config.xml", "solr/crazy-path-to-schema.xml");
@@ -38,7 +41,15 @@ public class DirectSolrConnectionTest extends SolrTestCaseJ4 {
   public void setUp() throws Exception
   {
     super.setUp();
-    direct = new DirectSolrConnection(h.getCore());
+    core = h.getCore();
+    direct = new DirectSolrConnection(core);
+  }
+
+  @Override
+  public void tearDown() throws Exception
+  {
+    super.tearDown();
+    IOUtils.closeQuietly(core);
   }
 
   // Check that a request gets back the echoParams call
diff --git a/solr/core/src/test/org/apache/solr/spelling/DirectSolrSpellCheckerTest.java b/solr/core/src/test/org/apache/solr/spelling/DirectSolrSpellCheckerTest.java
index 6106fb4..786600b 100644
--- a/solr/core/src/test/org/apache/solr/spelling/DirectSolrSpellCheckerTest.java
+++ b/solr/core/src/test/org/apache/solr/spelling/DirectSolrSpellCheckerTest.java
@@ -61,7 +61,7 @@ public class DirectSolrSpellCheckerTest extends SolrTestCaseJ4 {
     SolrCore core = h.getCore();
     checker.init(spellchecker, core);
 
-    h.getCore().withSearcher(searcher -> {
+    core.withSearcher(searcher -> {
 
       // check that 'fob' is corrected to 'foo'
       Collection<Token> tokens = queryConverter.convert("fob");
@@ -83,6 +83,7 @@ public class DirectSolrSpellCheckerTest extends SolrTestCaseJ4 {
       assertTrue("suggestions should be empty", suggestions.isEmpty());
       return null;
     });
+    core.close();
   }
   
   @Test
@@ -115,7 +116,7 @@ public class DirectSolrSpellCheckerTest extends SolrTestCaseJ4 {
     SolrCore core = h.getCore();
     checker.init(spellchecker, core);
 
-    h.getCore().withSearcher(searcher -> {
+    core.withSearcher(searcher -> {
       Collection<Token> tokens = queryConverter.convert("anothar");
       SpellingOptions spellOpts = new SpellingOptions(tokens, searcher.getIndexReader());
       SpellingResult result = checker.getSuggestions(spellOpts);
@@ -133,6 +134,7 @@ public class DirectSolrSpellCheckerTest extends SolrTestCaseJ4 {
 
       return null;
     });
+    core.close();
   }
   
 }
diff --git a/solr/core/src/test/org/apache/solr/spelling/FileBasedSpellCheckerTest.java b/solr/core/src/test/org/apache/solr/spelling/FileBasedSpellCheckerTest.java
index b52e411..5519948 100644
--- a/solr/core/src/test/org/apache/solr/spelling/FileBasedSpellCheckerTest.java
+++ b/solr/core/src/test/org/apache/solr/spelling/FileBasedSpellCheckerTest.java
@@ -73,7 +73,7 @@ public class FileBasedSpellCheckerTest extends SolrTestCaseJ4 {
     assertTrue(dictName + " is not equal to " + "external", dictName.equals("external") == true);
     checker.build(core, null);
 
-    h.getCore().withSearcher(searcher -> {
+    core.withSearcher(searcher -> {
       Collection<Token> tokens = queryConverter.convert("fob");
       SpellingOptions spellOpts = new SpellingOptions(tokens, searcher.getIndexReader());
       SpellingResult result = checker.getSuggestions(spellOpts);
@@ -90,7 +90,7 @@ public class FileBasedSpellCheckerTest extends SolrTestCaseJ4 {
       assertTrue("suggestions is not null and it should be", suggestions == null);
       return null;
     });
-
+    core.close();
   }
 
   @Test
@@ -113,7 +113,7 @@ public class FileBasedSpellCheckerTest extends SolrTestCaseJ4 {
     checker.build(core, null);
 
     Collection<Token> tokens = queryConverter.convert("Solar");
-    h.getCore().withSearcher(searcher -> {
+    core.withSearcher(searcher -> {
       SpellingOptions spellOpts = new SpellingOptions(tokens, searcher.getIndexReader());
       SpellingResult result = checker.getSuggestions(spellOpts);
       assertTrue("result is null and it shouldn't be", result != null);
@@ -132,6 +132,7 @@ public class FileBasedSpellCheckerTest extends SolrTestCaseJ4 {
       assertTrue("suggestions is not null and it should be", suggestions == null);
       return null;
     });
+    core.close();
   }
 
   /**
@@ -155,7 +156,7 @@ public class FileBasedSpellCheckerTest extends SolrTestCaseJ4 {
     assertTrue(dictName + " is not equal to " + "external", dictName.equals("external") == true);
     checker.build(core, null);
 
-    h.getCore().withSearcher(searcher -> {
+    core.withSearcher(searcher -> {
       Collection<Token> tokens = queryConverter.convert("solar");
       SpellingOptions spellOpts = new SpellingOptions(tokens, searcher.getIndexReader());
       SpellingResult result = checker.getSuggestions(spellOpts);
@@ -175,5 +176,6 @@ public class FileBasedSpellCheckerTest extends SolrTestCaseJ4 {
       assertTrue("suggestions size should be 0", suggestions.size()==0);
       return null;
     });
+    core.close();
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java b/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java
index d0a8b54..82d12a7 100644
--- a/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java
+++ b/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java
@@ -813,14 +813,15 @@ public class AddBlockUpdateTest extends SolrTestCaseJ4 {
   }
 
   private void indexSolrInputDocumentsDirectly(SolrInputDocument ... docs) throws IOException {
-    SolrQueryRequest coreReq = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams());
+    SolrQueryRequest coreReq = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams(), true);
     AddUpdateCommand updateCmd = new AddUpdateCommand(coreReq);
     for (SolrInputDocument doc: docs) {
       updateCmd.solrDoc = doc;
-      h.getCore().getUpdateHandler().addDoc(updateCmd);
+      coreReq.getCore().getUpdateHandler().addDoc(updateCmd);
       updateCmd.clear();
     }
     assertU(commit());
+    coreReq.close();
   }
 
   /**
diff --git a/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java b/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java
index 1b0794b..8a00a59 100644
--- a/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java
+++ b/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java
@@ -69,6 +69,7 @@ public class DocumentBuilderTest extends SolrTestCaseJ4 {
 
     SolrException ex = expectThrows(SolrException.class, () -> DocumentBuilder.toDocument( doc, core.getLatestSchema() ));
     assertEquals("should be bad request", 400, ex.code());
+    core.close();
   }
 
   @Test
@@ -80,6 +81,7 @@ public class DocumentBuilderTest extends SolrTestCaseJ4 {
     doc.addField( "name", null );
     Document out = DocumentBuilder.toDocument( doc, core.getLatestSchema() );
     assertNull( out.get( "name" ) );
+    core.close();
   }
 
   @Test
@@ -103,6 +105,7 @@ public class DocumentBuilderTest extends SolrTestCaseJ4 {
     // now make sure it is OK
     doc.setField( "weight", "1.34" );
     DocumentBuilder.toDocument( doc, core.getLatestSchema() );
+    core.close();
   }
 
   @Test
@@ -116,6 +119,7 @@ public class DocumentBuilderTest extends SolrTestCaseJ4 {
     assertNotNull( out.get( "home" ) );//contains the stored value and term vector, if there is one
     assertNotNull( out.getField( "home_0" + FieldType.POLY_FIELD_SEPARATOR + "double") );
     assertNotNull( out.getField( "home_1" + FieldType.POLY_FIELD_SEPARATOR + "double") );
+    core.close();
   }
   
   /**
@@ -297,6 +301,7 @@ public class DocumentBuilderTest extends SolrTestCaseJ4 {
     out = DocumentBuilder.toDocument(doc, core.getLatestSchema());
     assertEquals(testValue, out.get("title"));
     assertEquals(truncatedValue, out.get("max_chars"));
+    core.close();
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java b/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
index e6c08eb..2e41a16 100644
--- a/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
+++ b/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
@@ -25,6 +25,7 @@ import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.component.RealTimeGetComponent;
 import org.apache.solr.request.SolrQueryRequest;
 import org.junit.After;
@@ -65,7 +66,8 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
   /**
    * @see org.apache.solr.update.UpdateLog#applyPartialUpdates(BytesRef,long,long,SolrDocumentBase)
    */
-  public void testApplyPartialUpdatesOnMultipleInPlaceUpdatesInSequence() {    
+  public void testApplyPartialUpdatesOnMultipleInPlaceUpdatesInSequence() {
+    SolrCore core = h.getCore();
     // Add a full update, two in-place updates and verify applying partial updates is working
     ulogAdd(ulog, null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
     ulogAdd(ulog, 100L, sdoc("id", "1", "price", "1000", "val1_i_dvo", "2", "_version_", "101"));
@@ -73,7 +75,7 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
 
     Object partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
     SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), 
-        h.getCore().getLatestSchema());
+        core.getLatestSchema());
     long prevVersion = (Long)((List)partialUpdate).get(3);
     long prevPointer = (Long)((List)partialUpdate).get(2);
 
@@ -97,7 +99,7 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
     ulogAdd(ulog, 200L, sdoc("id", "1", "val1_i_dvo", "5", "_version_", "201"));
 
     partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
-    partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), h.getCore().getLatestSchema());
+    partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), core.getLatestSchema());
     prevVersion = (Long)((List)partialUpdate).get(3);
     prevPointer = (Long)((List)partialUpdate).get(2);
 
@@ -110,10 +112,13 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
     assertEquals(2000, Integer.parseInt(partialDoc.getFieldValue("price").toString()));
     assertEquals(5L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
     assertEquals("title1", partialDoc.getFieldValue("title_s"));
+
+    core.close();
   }
   
   @Test
-  public void testApplyPartialUpdatesAfterMultipleCommits() {    
+  public void testApplyPartialUpdatesAfterMultipleCommits() {
+    SolrCore core = h.getCore();
     ulogAdd(ulog, null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
     ulogAdd(ulog, 100L, sdoc("id", "1", "price", "1000", "val1_i_dvo", "2", "_version_", "101"));
     ulogAdd(ulog, 101L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "102"));
@@ -124,7 +129,7 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
     ulogAdd(ulog, 101L, sdoc("id", "1", "val1_i_dvo", "6", "_version_", "300"));
 
     Object partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
-    SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), h.getCore().getLatestSchema());
+    SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), core.getLatestSchema());
     long prevVersion = (Long)((List)partialUpdate).get(3);
     long prevPointer = (Long)((List)partialUpdate).get(2);
 
@@ -134,6 +139,7 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
     long returnVal = ulog.applyPartialUpdates(DOC_1_INDEXED_ID, prevPointer, prevVersion, null, partialDoc);
 
     assertEquals(-1, returnVal);
+    core.close();
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactoryTest.java
index d9336ae..076c7b4 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactoryTest.java
@@ -28,6 +28,7 @@ import java.util.Locale;
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
 import org.junit.After;
 import org.junit.Before;
@@ -60,7 +61,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testEmptyValue() {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "newFieldABC";
     assertNull(schema.getFieldOrNull(fieldName));
     //UpdateProcessorTestBase#doc doesn't deal with nulls
@@ -75,25 +78,33 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testSingleField() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "newfield1";
     assertNull(schema.getFieldOrNull(fieldName));
     Date date = Date.from(Instant.now());
     SolrInputDocument d = processAdd("add-fields-no-run-processor", doc(f("id", "1"), f(fieldName, date)));
     assertNotNull(d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName));
     assertEquals("pdates", schema.getFieldType(fieldName).getTypeName());
   }
 
   public void testSingleFieldRoundTrip() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "newfield2";
     assertNull(schema.getFieldOrNull(fieldName));
     Float floatValue = -13258.992f;
     SolrInputDocument d = processAdd("add-fields", doc(f("id", "2"), f(fieldName, floatValue)));
     assertNotNull(d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName));
     assertTrue(schema.getFieldType(fieldName).getTypeName().equals("pdoubles") || schema.getFieldType(fieldName).getTypeName().equals("pfloats"));
     assertU(commit());
@@ -102,7 +113,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testSingleFieldMixedFieldTypesRoundTrip() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "newfield3";
     assertNull(schema.getFieldOrNull(fieldName));
     Float fieldValue1 = -13258.0f;
@@ -110,7 +123,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
     SolrInputDocument d = processAdd
         ("add-fields", doc(f("id", "3"), f(fieldName, fieldValue1, fieldValue2)));
     assertNotNull(d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName));
     assertEquals("pdoubles", schema.getFieldType(fieldName).getTypeName());
     assertU(commit());
@@ -120,7 +135,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testSingleFieldDefaultFieldTypeRoundTrip() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "newfield4";
     assertNull(schema.getFieldOrNull(fieldName));
     Float fieldValue1 = -13258.0f;
@@ -129,7 +146,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
     SolrInputDocument d = processAdd
         ("add-fields", doc(f("id", "4"), f(fieldName, fieldValue1, fieldValue2, fieldValue3)));
     assertNotNull(d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName));
     assertEquals("text", schema.getFieldType(fieldName).getTypeName());
     assertEquals(0, schema.getCopyFieldProperties(true, Collections.singleton(fieldName), null).size());
@@ -142,7 +161,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testSingleFieldDefaultTypeMappingRoundTrip() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "newfield4";
     assertNull(schema.getFieldOrNull(fieldName));
     Float fieldValue1 = -13258.0f;
@@ -151,7 +172,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
     SolrInputDocument d = processAdd
         ("add-fields-default-mapping", doc(f("id", "4"), f(fieldName, fieldValue1, fieldValue2, fieldValue3)));
     assertNotNull(d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName));
     assertEquals("text", schema.getFieldType(fieldName).getTypeName());
     assertEquals(1, schema.getCopyFieldProperties(true, Collections.singleton(fieldName), null).size());
@@ -164,7 +187,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testMultipleFieldsRoundTrip() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName1 = "newfield5";
     final String fieldName2 = "newfield6";
     assertNull(schema.getFieldOrNull(fieldName1));
@@ -178,7 +203,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
         ("add-fields", doc(f("id", "5"), f(fieldName1, field1Value1, field1Value2, field1Value3),
                                          f(fieldName2, field2Value1, field2Value2)));
     assertNotNull(d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName1));
     assertNotNull(schema.getFieldOrNull(fieldName2));
     // nocommit - can be either order, not consistent
@@ -194,7 +221,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testParseAndAddMultipleFieldsRoundTrip() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName1 = "newfield7";
     final String fieldName2 = "newfield8";
     final String fieldName3 = "newfield9";
@@ -230,7 +259,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
                                                    f(fieldName3, field3String1, field3String2),
                                                    f(fieldName4, field4String1)));
     assertNotNull(d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName1));
     assertNotNull(schema.getFieldOrNull(fieldName2));
     assertNotNull(schema.getFieldOrNull(fieldName3));
@@ -252,14 +283,18 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testStringWithCopyField() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "stringField";
     final String strFieldName = fieldName+"_str";
     assertNull(schema.getFieldOrNull(fieldName));
     String content = "This is a text that should be copied to a string field but not be cutoff";
     SolrInputDocument d = processAdd("add-fields", doc(f("id", "1"), f(fieldName, content)));
     assertNotNull(d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName));
     assertNotNull(schema.getFieldOrNull(strFieldName));
     assertEquals("text", schema.getFieldType(fieldName).getTypeName());
@@ -267,7 +302,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testStringWithCopyFieldAndMaxChars() throws Exception {
-    IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    core.close();
     final String fieldName = "stringField";
     final String strFieldName = fieldName+"_str";
     assertNull(schema.getFieldOrNull(fieldName));
@@ -275,7 +312,9 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
     SolrInputDocument d = processAdd("add-fields-maxchars", doc(f("id", "1"), f(fieldName, content)));
     assertNotNull(d);
     //System.out.println("Document is "+d);
-    schema = h.getCore().getLatestSchema();
+    core = h.getCore();
+    schema = core.getLatestSchema();
+    core.close();
     assertNotNull(schema.getFieldOrNull(fieldName));
     assertNotNull(schema.getFieldOrNull(strFieldName));
     assertEquals("text", schema.getFieldType(fieldName).getTypeName());
diff --git a/solr/core/src/test/org/apache/solr/update/processor/DefaultValueUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/DefaultValueUpdateProcessorTest.java
index 8ed0d1b..ca48723 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/DefaultValueUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/DefaultValueUpdateProcessorTest.java
@@ -134,7 +134,7 @@ public class DefaultValueUpdateProcessorTest extends SolrTestCaseJ4 {
     SolrQueryResponse rsp = new SolrQueryResponse();
 
     SolrQueryRequest req = new LocalSolrQueryRequest
-      (core, new ModifiableSolrParams());
+      (core, new ModifiableSolrParams(), true);
     try {
       SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req,rsp));
       AddUpdateCommand cmd = new AddUpdateCommand(req);
diff --git a/solr/core/src/test/org/apache/solr/update/processor/DistributedUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/DistributedUpdateProcessorTest.java
index b1eb087..7fa942d 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/DistributedUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/DistributedUpdateProcessorTest.java
@@ -73,7 +73,7 @@ public class DistributedUpdateProcessorTest extends SolrTestCaseJ4 {
 
   @Test
   public void testShouldBufferUpdateZk() throws IOException {
-    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams());
+    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams(), true);
     try (DistributedUpdateProcessor processor = new DistributedUpdateProcessor(
         req, null, null, null)) {
       AddUpdateCommand cmd = new AddUpdateCommand(req);
@@ -86,11 +86,12 @@ public class DistributedUpdateProcessorTest extends SolrTestCaseJ4 {
       cmd.prevVersion = 10;
       assertTrue(processor.shouldBufferUpdate(cmd, false, UpdateLog.State.APPLYING_BUFFERED));
     }
+    req.close();
   }
   
   @Test
   public void testVersionAdd() throws IOException {
-    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams());
+    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams(), true);
     int threads = 5;
     Function<DistributedUpdateProcessor,Boolean> versionAddFunc = (DistributedUpdateProcessor process) -> {
       try {
@@ -109,11 +110,12 @@ public class DistributedUpdateProcessorTest extends SolrTestCaseJ4 {
     succeeded = runCommands(threads, -1, req, versionAddFunc);
     // all should succeed
     assertThat(succeeded, is(threads));
+    req.close();
   }
 
   @Test
   public void testVersionDelete() throws IOException {
-    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams());
+    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams(), true);
 
     int threads = TEST_NIGHTLY ? 5 : 2;
     Function<DistributedUpdateProcessor,Boolean> versionDeleteFunc = (DistributedUpdateProcessor process) -> {
@@ -125,6 +127,7 @@ public class DistributedUpdateProcessorTest extends SolrTestCaseJ4 {
         throw new RuntimeException(e);
       }
     };
+    req.close();
 
     int succeeded = runCommands(threads, 50, req, versionDeleteFunc);
     // only one should succeed
diff --git a/solr/core/src/test/org/apache/solr/update/processor/IgnoreCommitOptimizeUpdateProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/IgnoreCommitOptimizeUpdateProcessorFactoryTest.java
index a858d48..24f8bb6 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/IgnoreCommitOptimizeUpdateProcessorFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/IgnoreCommitOptimizeUpdateProcessorFactoryTest.java
@@ -62,7 +62,7 @@ public class IgnoreCommitOptimizeUpdateProcessorFactoryTest extends SolrTestCase
     assertNotNull("No Chain named: " + chain, pc);
 
     SolrQueryResponse rsp = new SolrQueryResponse();
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams(), true);
 
     if (commitEndPoint != null) {
       ((ModifiableSolrParams)req.getParams()).set(
diff --git a/solr/core/src/test/org/apache/solr/update/processor/UUIDUpdateProcessorFallbackTest.java b/solr/core/src/test/org/apache/solr/update/processor/UUIDUpdateProcessorFallbackTest.java
index b03a6d6..09722f0 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/UUIDUpdateProcessorFallbackTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/UUIDUpdateProcessorFallbackTest.java
@@ -126,17 +126,19 @@ public class UUIDUpdateProcessorFallbackTest extends SolrTestCaseJ4 {
   }
 
   public void testProcessorPrefixReqParam() throws Exception {
-    List<UpdateRequestProcessorFactory> processors = UpdateRequestProcessorChain.getReqProcessors("uuid", h.getCore());
+    SolrCore core =  h.getCore();
+    List<UpdateRequestProcessorFactory> processors = UpdateRequestProcessorChain.getReqProcessors("uuid", core);
     UpdateRequestProcessorFactory processorFactory = processors.get(0);
     assertTrue(processorFactory instanceof UUIDUpdateProcessorFactory);
 
     SolrQueryResponse rsp = new SolrQueryResponse();
-    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), new ModifiableSolrParams());
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams(), true);
     AddUpdateCommand cmd = new AddUpdateCommand(req);
     cmd.solrDoc = new SolrInputDocument();
     cmd.solrDoc.addField("random_s", "random_val");
 
     UpdateRequestProcessor proc = processorFactory.getInstance(req, rsp, null);
+    req.close();
     proc.processAdd(cmd);
     proc.close();
     assertNotNull(cmd.solrDoc);
@@ -198,7 +200,7 @@ public class UUIDUpdateProcessorFallbackTest extends SolrTestCaseJ4 {
     SolrQueryResponse rsp = new SolrQueryResponse();
 
     SolrQueryRequest req = new LocalSolrQueryRequest
-        (core, params);
+        (core, params, true);
     try {
       SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req,rsp));
       AddUpdateCommand cmd = new AddUpdateCommand(req);
diff --git a/solr/core/src/test/org/apache/solr/update/processor/UniqFieldsUpdateProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/UniqFieldsUpdateProcessorFactoryTest.java
index 6156ec9..671a095 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/UniqFieldsUpdateProcessorFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/UniqFieldsUpdateProcessorFactoryTest.java
@@ -106,7 +106,8 @@ public class UniqFieldsUpdateProcessorFactoryTest extends SolrTestCaseJ4 {
     Map<String, String[]> params = new HashMap<>();
     MultiMapSolrParams mmparams = new MultiMapSolrParams(params);
     params.put(UpdateParams.UPDATE_CHAIN, new String[] { "uniq-fields" });
-    SolrQueryRequestBase req = new SolrQueryRequestBase(h.getCore(),
+    SolrCore core = h.getCore();
+    SolrQueryRequestBase req = new SolrQueryRequestBase(core,
         (SolrParams) mmparams) {
     };
 
@@ -117,5 +118,6 @@ public class UniqFieldsUpdateProcessorFactoryTest extends SolrTestCaseJ4 {
     req.setContentStreams(streams);
     handler.handleRequestBody(req, new SolrQueryResponse());
     req.close();
+    core.close();
   }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
index db1c38b..fbd91fb 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
@@ -70,6 +70,7 @@ import org.eclipse.jetty.http.HttpHeader;
 import org.eclipse.jetty.http.HttpMethod;
 import org.eclipse.jetty.http2.client.HTTP2Client;
 import org.eclipse.jetty.http2.client.http.HttpClientTransportOverHTTP2;
+import org.eclipse.jetty.util.BlockingArrayQueue;
 import org.eclipse.jetty.util.Fields;
 import org.eclipse.jetty.util.Pool;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
@@ -99,6 +100,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Phaser;
@@ -225,9 +227,13 @@ public class Http2SolrClient extends SolrClient {
     int minThreads = Integer.getInteger("solr.minHttp2ClientThreads", PROC_COUNT);
 
     minThreads = Math.min( builder.maxThreadPoolSize, minThreads);
+
+    int capacity = Math.max(minThreads, 8) * 32;
+    BlockingQueue<Runnable> queue = new BlockingArrayQueue<>(capacity, capacity);
+
     httpClientExecutor = new SolrQueuedThreadPool("http2Client", builder.maxThreadPoolSize, minThreads,
         this.headers != null && this.headers.containsKey(QoSParams.REQUEST_SOURCE) && this.headers.get(QoSParams.REQUEST_SOURCE).equals(QoSParams.INTERNAL) ? 1000 : 1000,
-        null, -1, null);
+        queue, -1, null);
     httpClientExecutor.setLowThreadsThreshold(-1);
 
     boolean sslOnJava8OrLower = ssl && !Constants.JRE_IS_MINIMUM_JAVA9;
@@ -561,16 +567,7 @@ public class Http2SolrClient extends SolrClient {
     if (req.afterSend != null) {
       req.afterSend.run();
     }
-    return new Cancellable() {
-      @Override
-      public void cancel() {
-        boolean success = req.request.abort(CANCELLED_EXCEPTION);
-      }
-      @Override
-      public InputStream getStream() {
-        return mysl.getInputStream();
-      }
-    };
+    return new MyCancellable(req, mysl);
   }
 
   @Override
@@ -1021,6 +1018,26 @@ public class Http2SolrClient extends SolrClient {
     return serverBaseUrl;
   }
 
+  private static class MyCancellable implements Cancellable {
+    private final TheRequest req;
+    private final MyInputStreamResponseListener mysl;
+
+    public MyCancellable(TheRequest req, MyInputStreamResponseListener mysl) {
+      this.req = req;
+      this.mysl = mysl;
+    }
+
+    @Override
+    public void cancel() {
+      boolean success = req.request.abort(CANCELLED_EXCEPTION);
+    }
+
+    @Override
+    public InputStream getStream() {
+      return mysl.getInputStream();
+    }
+  }
+
   public class AsyncTracker {
 
     private final Semaphore available;
diff --git a/solr/solrj/src/java/org/apache/solr/common/PerThreadExecService.java b/solr/solrj/src/java/org/apache/solr/common/PerThreadExecService.java
index e1cbec1..0815de0 100644
--- a/solr/solrj/src/java/org/apache/solr/common/PerThreadExecService.java
+++ b/solr/solrj/src/java/org/apache/solr/common/PerThreadExecService.java
@@ -138,6 +138,7 @@ public class PerThreadExecService extends AbstractExecutorService {
         try {
           available.acquire();
         } catch (InterruptedException e) {
+          ParWork.propagateInterrupt(e);
           running.decrementAndGet();
           throw new RejectedExecutionException("Interrupted");
         }
@@ -157,19 +158,19 @@ public class PerThreadExecService extends AbstractExecutorService {
       return;
     }
 
+    if (!noCallerRunsAllowed && checkLoad()) {
+      runIt(runnable, true, false);
+      return;
+    }
+
     try {
       available.acquire();
     } catch (InterruptedException e) {
+      ParWork.propagateInterrupt(e);
       running.decrementAndGet();
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
 
-
-    if (!noCallerRunsAllowed && checkLoad()) {
-      runIt(runnable, true, false);
-      return;
-    }
-
     Runnable finalRunnable = runnable;
     try {
       service.submit(() -> runIt(finalRunnable, true, false));
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/SolrQueuedThreadPool.java b/solr/solrj/src/java/org/apache/solr/common/util/SolrQueuedThreadPool.java
index e8dc92b..d39b946 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/SolrQueuedThreadPool.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/SolrQueuedThreadPool.java
@@ -126,7 +126,7 @@ public class SolrQueuedThreadPool extends ContainerLifeCycle implements ThreadFa
     setReservedThreads(0);
     setLowThreadsThreshold(-1);
     if (queue == null) {
-      int capacity = Math.max(_minThreads, 8) * 1024;
+      int capacity = Math.max(_minThreads, 8) * 256;
       queue = new BlockingArrayQueue<>(capacity, capacity);
     }
     _jobs = queue;
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java b/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
index 5616cd1..4e119e7 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
@@ -295,7 +295,7 @@ public class ValidatingJsonMap implements Map<String, Object>, NavigableObject {
   public static ValidatingJsonMap getDeepCopy(Map map, int maxDepth, boolean mutable) {
     if (map == null) return null;
     if (maxDepth < 1) return ValidatingJsonMap.wrap(map);
-    ValidatingJsonMap copy = new ValidatingJsonMap(map.size() * 3);
+    ValidatingJsonMap copy = new ValidatingJsonMap( map.size() );
     for (Object o : map.entrySet()) {
       Map.Entry<String, Object> e = (Entry<String, Object>) o;
       Object v = e.getValue();
@@ -323,15 +323,14 @@ public class ValidatingJsonMap implements Map<String, Object>, NavigableObject {
     return new ObjectBuilder(jp) {
       @Override
       public Object newObject() throws IOException {
-        return new ValidatingJsonMap(32);
+        return new ValidatingJsonMap(8);
       }
     };
   }
 
   public static ValidatingJsonMap parse(String resourceName,
       String includeLocation) {
-    InputStream resource = ValidatingJsonMap.class.getClassLoader()
-        .getResourceAsStream(resourceName);
+    InputStream resource = Thread.currentThread().getContextClassLoader().getResourceAsStream(resourceName);
     if (null == resource) {
       throw new RuntimeException("invalid API spec: " + resourceName);
     }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/Schema2Test.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/Schema2Test.java
new file mode 100644
index 0000000..05ecee1
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/Schema2Test.java
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.solrj.request;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.request.schema.AnalyzerDefinition;
+import org.apache.solr.client.solrj.request.schema.FieldTypeDefinition;
+import org.apache.solr.client.solrj.request.schema.SchemaRequest;
+import org.apache.solr.client.solrj.response.SolrResponseBase;
+import org.apache.solr.client.solrj.response.schema.FieldTypeRepresentation;
+import org.apache.solr.client.solrj.response.schema.SchemaResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.RestTestBase;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import java.io.File;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Test the functionality (accuracy and failure) of the methods exposed by the classes
+ * {@link SchemaRequest} and {@link SchemaResponse}.
+ */
+public class Schema2Test extends RestTestBase {
+
+  private static void assertValidSchemaResponse(SolrResponseBase schemaResponse) {
+    assertEquals("Response contained errors: " + schemaResponse.toString(), 0, schemaResponse.getStatus());
+    assertNull("Response contained errors: " + schemaResponse.toString(), schemaResponse.getResponse().get("errors"));
+  }
+  
+  private static void assertFailedSchemaResponse(ThrowingRunnable runnable, String expectedErrorMessage) {
+    BaseHttpSolrClient.RemoteSolrException e = expectThrows(BaseHttpSolrClient.RemoteSolrException.class, runnable);
+    assertTrue(e.getMessage(), e.getMessage().contains(expectedErrorMessage));
+  }
+
+  private static void createStoredStringField(String fieldName, SolrClient solrClient) throws Exception {
+    Map<String, Object> fieldAttributes = new LinkedHashMap<>();
+    fieldAttributes.put("name", fieldName);
+    fieldAttributes.put("type", "string");
+    fieldAttributes.put("stored", true);
+    SchemaRequest.AddField addFieldRequest = new SchemaRequest.AddField(fieldAttributes);
+    addFieldRequest.process(solrClient);
+  }
+
+  @BeforeClass
+  public static void beforeSolrExampleTestsBase() throws Exception {
+    File tmpSolrHome = createTempDir().toFile();
+    FileUtils.copyDirectory(new File(getFile("solrj/solr/collection1").getParent()), tmpSolrHome.getAbsoluteFile());
+
+    final SortedMap<ServletHolder, String> extraServlets = new TreeMap<>();
+
+    System.setProperty("managed.schema.mutable", "true");
+    System.setProperty("enable.update.log", "false");
+
+    jetty = createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema.xml",
+            "/solr", true, extraServlets);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  @After
+  public void tearDown() throws Exception  {
+    super.tearDown();
+  }
+
+  @Test
+  public void testDeleteFieldAccuracy() throws Exception {
+    String fieldName = "fieldToBeDeleted";
+    Map<String, Object> fieldAttributesRequest = new LinkedHashMap<>();
+    fieldAttributesRequest.put("name", fieldName);
+    fieldAttributesRequest.put("type", "string");
+    SchemaRequest.AddField addFieldUpdateSchemaRequest =
+        new SchemaRequest.AddField(fieldAttributesRequest);
+    SchemaResponse.UpdateResponse addFieldResponse = addFieldUpdateSchemaRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(addFieldResponse);
+
+    SchemaRequest.Field fieldSchemaRequest = new SchemaRequest.Field(fieldName);
+    SchemaResponse.FieldResponse initialFieldResponse = fieldSchemaRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(initialFieldResponse);
+    Map<String, Object> fieldAttributesResponse = initialFieldResponse.getField();
+    assertThat(fieldName, is(equalTo(fieldAttributesResponse.get("name"))));
+
+    SchemaRequest.DeleteField deleteFieldRequest =
+        new SchemaRequest.DeleteField(fieldName);
+    SchemaResponse.UpdateResponse deleteFieldResponse = deleteFieldRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(deleteFieldResponse);
+
+    expectThrows(SolrException.class, () -> fieldSchemaRequest.process(getSolrClient(jetty)));
+  }
+
+  @Test
+  public void deletingAFieldThatDoesntExistInTheSchemaShouldFail() {
+    String fieldName = "fieldToBeDeleted"; 
+    SchemaRequest.DeleteField deleteFieldRequest = new SchemaRequest.DeleteField(fieldName);
+    assertFailedSchemaResponse(() -> deleteFieldRequest.process(getSolrClient(jetty)),
+        "The field '" + fieldName + "' is not present in this schema, and so cannot be deleted.");
+  }
+
+  @Test
+  public void testReplaceFieldAccuracy() throws Exception {
+    // Given
+    Map<String, Object> fieldAttributes = new LinkedHashMap<>();
+    String fieldName = "accuracyFieldReplace";
+    fieldAttributes.put("name", fieldName);
+    fieldAttributes.put("type", "string");
+    fieldAttributes.put("stored", false);
+    fieldAttributes.put("indexed", true);
+    fieldAttributes.put("required", true);
+    SchemaRequest.AddField addFieldUpdateSchemaRequest =
+        new SchemaRequest.AddField(fieldAttributes);
+    SchemaResponse.UpdateResponse addFieldResponse = addFieldUpdateSchemaRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(addFieldResponse);
+
+    // When : update the field definition
+    fieldAttributes.put("stored", true);
+    fieldAttributes.put("indexed", false);
+    SchemaRequest.ReplaceField replaceFieldRequest = new SchemaRequest.ReplaceField(fieldAttributes);
+    SchemaResponse.UpdateResponse replaceFieldResponse = replaceFieldRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(replaceFieldResponse);
+
+    // Then
+    SchemaRequest.Field fieldSchemaRequest = new SchemaRequest.Field(fieldName);
+    SchemaResponse.FieldResponse newFieldResponse = fieldSchemaRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(newFieldResponse);
+    Map<String, Object> newFieldAttributes = newFieldResponse.getField();
+    assertThat(fieldName, is(equalTo(newFieldAttributes.get("name"))));
+    assertThat("string", is(equalTo(newFieldAttributes.get("type"))));
+    assertThat(true, is(equalTo(newFieldAttributes.get("stored"))));
+    assertThat(false, is(equalTo(newFieldAttributes.get("indexed"))));
+    assertThat(true, is(equalTo(newFieldAttributes.get("required"))));
+  }
+
+  @Test
+  public void testReplaceDynamicFieldAccuracy() throws Exception {
+    // Given
+    String fieldName = "*_replace";
+    Map<String, Object> fieldAttributes = new LinkedHashMap<>();
+    fieldAttributes.put("name", fieldName);
+    fieldAttributes.put("type", "string");
+    fieldAttributes.put("stored", false);
+    fieldAttributes.put("indexed", true);
+    SchemaRequest.AddDynamicField addDFieldUpdateSchemaRequest =
+        new SchemaRequest.AddDynamicField(fieldAttributes);
+    SchemaResponse.UpdateResponse addFieldResponse = addDFieldUpdateSchemaRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(addFieldResponse);
+
+    // When : update the field definition
+    Map<String, Object> replaceFieldAttributes = new LinkedHashMap<>(fieldAttributes);
+    replaceFieldAttributes.put("stored", true);
+    replaceFieldAttributes.put("indexed", false);
+    SchemaRequest.ReplaceDynamicField replaceFieldRequest =
+        new SchemaRequest.ReplaceDynamicField(replaceFieldAttributes);
+    SchemaResponse.UpdateResponse replaceFieldResponse = replaceFieldRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(replaceFieldResponse);
+
+    // Then
+    SchemaRequest.DynamicField dynamicFieldSchemaRequest =
+        new SchemaRequest.DynamicField(fieldName);
+    SchemaResponse.DynamicFieldResponse newFieldResponse = dynamicFieldSchemaRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(newFieldResponse);
+    Map<String, Object> newFieldAttributes = newFieldResponse.getDynamicField();
+    assertThat(fieldName, is(equalTo(newFieldAttributes.get("name"))));
+    assertThat("string", is(equalTo(newFieldAttributes.get("type"))));
+    assertThat(true, is(equalTo(newFieldAttributes.get("stored"))));
+    assertThat(false, is(equalTo(newFieldAttributes.get("indexed"))));
+  }
+
+  @Test
+  public void testDeleteFieldTypeAccuracy() throws Exception {
+    Map<String, Object> fieldTypeAttributes = new LinkedHashMap<>();
+    String fieldTypeName = "delInt";
+    fieldTypeAttributes.put("name", fieldTypeName);
+    fieldTypeAttributes.put("class",  RANDOMIZED_NUMERIC_FIELDTYPES.get(Integer.class));
+    fieldTypeAttributes.put("omitNorms", true);
+    fieldTypeAttributes.put("positionIncrementGap", 0);
+    FieldTypeDefinition fieldTypeDefinition = new FieldTypeDefinition();
+    fieldTypeDefinition.setAttributes(fieldTypeAttributes);
+    SchemaRequest.AddFieldType addFieldTypeRequest =
+        new SchemaRequest.AddFieldType(fieldTypeDefinition);
+    SolrClient c = getSolrClient(jetty);
+    SchemaResponse.UpdateResponse addFieldTypeResponse = addFieldTypeRequest.process(c);
+    assertValidSchemaResponse(addFieldTypeResponse);
+
+    SchemaRequest.FieldType fieldTypeRequest = new SchemaRequest.FieldType(fieldTypeName);
+    SchemaResponse.FieldTypeResponse initialFieldTypeResponse = fieldTypeRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(initialFieldTypeResponse);
+    FieldTypeRepresentation responseFieldTypeRepresentation = initialFieldTypeResponse.getFieldType();
+    assertThat(fieldTypeName, is(equalTo(responseFieldTypeRepresentation.getAttributes().get("name"))));
+
+    SchemaRequest.DeleteFieldType deleteFieldTypeRequest =
+        new SchemaRequest.DeleteFieldType(fieldTypeName);
+    SchemaResponse.UpdateResponse deleteFieldTypeResponse = deleteFieldTypeRequest.process(getSolrClient(jetty));
+    assertValidSchemaResponse(deleteFieldTypeResponse);
+
+    try {
+      fieldTypeRequest.process(getSolrClient(jetty));
+      fail(String.format(Locale.ROOT, "after removal, the field type %s shouldn't be anymore available over Schema API",
+          fieldTypeName));
+    } catch (SolrException e) {
+      //success
+    }
+  }
+
+  @Test
+  public void testDeleteCopyFieldAccuracy() throws Exception {
+    String srcFieldName = "copyfield";
+    String destFieldName1 = "destField1", destFieldName2 = "destField2";
+    createStoredStringField(srcFieldName, getSolrClient(jetty));
+    createStoredStringField(destFieldName1, getSolrClient(jetty));
+    createStoredStringField(destFieldName2, getSolrClient(jetty));
+
+    SchemaRequest.AddCopyField addCopyFieldRequest =
+        new SchemaRequest.AddCopyField(srcFieldName,
+            Arrays.asList(destFieldName1, destFieldName2));
+    SchemaResponse.UpdateResponse addCopyFieldResponse = addCopyFieldRequest.process(getSolrClient(jetty));
+    System.out.println(addCopyFieldResponse);
+    assertValidSchemaResponse(addCopyFieldResponse);
+
+    SchemaRequest.DeleteCopyField deleteCopyFieldRequest1 =
+        new SchemaRequest.DeleteCopyField(srcFieldName, Arrays.asList(destFieldName1));
+    assertValidSchemaResponse(deleteCopyFieldRequest1.process(getSolrClient(jetty)));
+
+    SchemaRequest.DeleteCopyField deleteCopyFieldRequest2 =
+        new SchemaRequest.DeleteCopyField(srcFieldName, Arrays.asList(destFieldName2));
+    assertValidSchemaResponse(deleteCopyFieldRequest2.process(getSolrClient(jetty)));
+  }
+
+  @Test
+  public void deleteCopyFieldShouldFailWhenOneOfTheFieldsDoesntExistInTheSchema() throws Exception {
+    String srcFieldName = "copyfield";
+    String destFieldName1 = "destField1", destFieldName2 = "destField2";
+    SchemaRequest.DeleteCopyField deleteCopyFieldsRequest =
+        new SchemaRequest.DeleteCopyField(srcFieldName,
+            Arrays.asList(destFieldName1, destFieldName2));
+    assertFailedSchemaResponse(() -> deleteCopyFieldsRequest.process(getSolrClient(jetty)),
+        "Copy field directive not found: '" + srcFieldName + "' -> '" + destFieldName1 + "'");
+  }
+}
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
index e6b3bd3..503fa8a 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
@@ -22,7 +22,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -297,73 +296,6 @@ public class SchemaTest extends RestTestBase {
   }
 
   @Test
-  public void testDeleteFieldAccuracy() throws Exception {
-    String fieldName = "fieldToBeDeleted";
-    Map<String, Object> fieldAttributesRequest = new LinkedHashMap<>();
-    fieldAttributesRequest.put("name", fieldName);
-    fieldAttributesRequest.put("type", "string");
-    SchemaRequest.AddField addFieldUpdateSchemaRequest =
-        new SchemaRequest.AddField(fieldAttributesRequest);
-    SchemaResponse.UpdateResponse addFieldResponse = addFieldUpdateSchemaRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(addFieldResponse);
-
-    SchemaRequest.Field fieldSchemaRequest = new SchemaRequest.Field(fieldName);
-    SchemaResponse.FieldResponse initialFieldResponse = fieldSchemaRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(initialFieldResponse);
-    Map<String, Object> fieldAttributesResponse = initialFieldResponse.getField();
-    assertThat(fieldName, is(equalTo(fieldAttributesResponse.get("name"))));
-
-    SchemaRequest.DeleteField deleteFieldRequest =
-        new SchemaRequest.DeleteField(fieldName);
-    SchemaResponse.UpdateResponse deleteFieldResponse = deleteFieldRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(deleteFieldResponse);
-
-    expectThrows(SolrException.class, () -> fieldSchemaRequest.process(getSolrClient(jetty)));
-  }
-
-  @Test
-  public void deletingAFieldThatDoesntExistInTheSchemaShouldFail() {
-    String fieldName = "fieldToBeDeleted"; 
-    SchemaRequest.DeleteField deleteFieldRequest = new SchemaRequest.DeleteField(fieldName);
-    assertFailedSchemaResponse(() -> deleteFieldRequest.process(getSolrClient(jetty)),
-        "The field '" + fieldName + "' is not present in this schema, and so cannot be deleted.");
-  }
-
-  @Test
-  public void testReplaceFieldAccuracy() throws Exception {
-    // Given
-    Map<String, Object> fieldAttributes = new LinkedHashMap<>();
-    String fieldName = "accuracyFieldReplace";
-    fieldAttributes.put("name", fieldName);
-    fieldAttributes.put("type", "string");
-    fieldAttributes.put("stored", false);
-    fieldAttributes.put("indexed", true);
-    fieldAttributes.put("required", true);
-    SchemaRequest.AddField addFieldUpdateSchemaRequest =
-        new SchemaRequest.AddField(fieldAttributes);
-    SchemaResponse.UpdateResponse addFieldResponse = addFieldUpdateSchemaRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(addFieldResponse);
-
-    // When : update the field definition
-    fieldAttributes.put("stored", true);
-    fieldAttributes.put("indexed", false);
-    SchemaRequest.ReplaceField replaceFieldRequest = new SchemaRequest.ReplaceField(fieldAttributes);
-    SchemaResponse.UpdateResponse replaceFieldResponse = replaceFieldRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(replaceFieldResponse);
-
-    // Then
-    SchemaRequest.Field fieldSchemaRequest = new SchemaRequest.Field(fieldName);
-    SchemaResponse.FieldResponse newFieldResponse = fieldSchemaRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(newFieldResponse);
-    Map<String, Object> newFieldAttributes = newFieldResponse.getField();
-    assertThat(fieldName, is(equalTo(newFieldAttributes.get("name"))));
-    assertThat("string", is(equalTo(newFieldAttributes.get("type"))));
-    assertThat(true, is(equalTo(newFieldAttributes.get("stored"))));
-    assertThat(false, is(equalTo(newFieldAttributes.get("indexed"))));
-    assertThat(true, is(equalTo(newFieldAttributes.get("required"))));
-  }
-
-  @Test
   public void testAddDynamicFieldAccuracy() throws Exception {
     SchemaRequest.DynamicFields dynamicFieldsSchemaRequest =
         new SchemaRequest.DynamicFields();
@@ -450,41 +382,6 @@ public class SchemaTest extends RestTestBase {
   }
 
   @Test
-  public void testReplaceDynamicFieldAccuracy() throws Exception {
-    // Given
-    String fieldName = "*_replace";
-    Map<String, Object> fieldAttributes = new LinkedHashMap<>();
-    fieldAttributes.put("name", fieldName);
-    fieldAttributes.put("type", "string");
-    fieldAttributes.put("stored", false);
-    fieldAttributes.put("indexed", true);
-    SchemaRequest.AddDynamicField addDFieldUpdateSchemaRequest =
-        new SchemaRequest.AddDynamicField(fieldAttributes);
-    SchemaResponse.UpdateResponse addFieldResponse = addDFieldUpdateSchemaRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(addFieldResponse);
-
-    // When : update the field definition
-    Map<String, Object> replaceFieldAttributes = new LinkedHashMap<>(fieldAttributes);
-    replaceFieldAttributes.put("stored", true);
-    replaceFieldAttributes.put("indexed", false);
-    SchemaRequest.ReplaceDynamicField replaceFieldRequest =
-        new SchemaRequest.ReplaceDynamicField(replaceFieldAttributes);
-    SchemaResponse.UpdateResponse replaceFieldResponse = replaceFieldRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(replaceFieldResponse);
-
-    // Then
-    SchemaRequest.DynamicField dynamicFieldSchemaRequest =
-        new SchemaRequest.DynamicField(fieldName);
-    SchemaResponse.DynamicFieldResponse newFieldResponse = dynamicFieldSchemaRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(newFieldResponse);
-    Map<String, Object> newFieldAttributes = newFieldResponse.getDynamicField();
-    assertThat(fieldName, is(equalTo(newFieldAttributes.get("name"))));
-    assertThat("string", is(equalTo(newFieldAttributes.get("type"))));
-    assertThat(true, is(equalTo(newFieldAttributes.get("stored"))));
-    assertThat(false, is(equalTo(newFieldAttributes.get("indexed"))));
-  }
-
-  @Test
   public void testAddFieldTypeAccuracy() throws Exception {
     SchemaRequest.FieldTypes fieldTypesRequest = new SchemaRequest.FieldTypes();
     SchemaResponse.FieldTypesResponse initialFieldTypesResponse = fieldTypesRequest.process(getSolrClient(jetty));
@@ -642,42 +539,6 @@ public class SchemaTest extends RestTestBase {
   }
 
   @Test
-  public void testDeleteFieldTypeAccuracy() throws Exception {
-    Map<String, Object> fieldTypeAttributes = new LinkedHashMap<>();
-    String fieldTypeName = "delInt";
-    fieldTypeAttributes.put("name", fieldTypeName);
-    fieldTypeAttributes.put("class",  RANDOMIZED_NUMERIC_FIELDTYPES.get(Integer.class));
-    fieldTypeAttributes.put("omitNorms", true);
-    fieldTypeAttributes.put("positionIncrementGap", 0);
-    FieldTypeDefinition fieldTypeDefinition = new FieldTypeDefinition();
-    fieldTypeDefinition.setAttributes(fieldTypeAttributes);
-    SchemaRequest.AddFieldType addFieldTypeRequest =
-        new SchemaRequest.AddFieldType(fieldTypeDefinition);
-    SolrClient c = getSolrClient(jetty);
-    SchemaResponse.UpdateResponse addFieldTypeResponse = addFieldTypeRequest.process(c);
-    assertValidSchemaResponse(addFieldTypeResponse);
-
-    SchemaRequest.FieldType fieldTypeRequest = new SchemaRequest.FieldType(fieldTypeName);
-    SchemaResponse.FieldTypeResponse initialFieldTypeResponse = fieldTypeRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(initialFieldTypeResponse);
-    FieldTypeRepresentation responseFieldTypeRepresentation = initialFieldTypeResponse.getFieldType();
-    assertThat(fieldTypeName, is(equalTo(responseFieldTypeRepresentation.getAttributes().get("name"))));
-
-    SchemaRequest.DeleteFieldType deleteFieldTypeRequest =
-        new SchemaRequest.DeleteFieldType(fieldTypeName);
-    SchemaResponse.UpdateResponse deleteFieldTypeResponse = deleteFieldTypeRequest.process(getSolrClient(jetty));
-    assertValidSchemaResponse(deleteFieldTypeResponse);
-
-    try {
-      fieldTypeRequest.process(getSolrClient(jetty));
-      fail(String.format(Locale.ROOT, "after removal, the field type %s shouldn't be anymore available over Schema API",
-          fieldTypeName));
-    } catch (SolrException e) {
-      //success
-    }
-  }
-
-  @Test
   public void deletingAFieldTypeThatDoesntExistInTheSchemaShouldFail() throws Exception {
     String fieldType = "fieldTypeToBeDeleted"; 
     SchemaRequest.DeleteFieldType deleteFieldTypeRequest = new SchemaRequest.DeleteFieldType(fieldType);
@@ -797,41 +658,6 @@ public class SchemaTest extends RestTestBase {
   }
 
   @Test
-  public void testDeleteCopyFieldAccuracy() throws Exception {
-    String srcFieldName = "copyfield";
-    String destFieldName1 = "destField1", destFieldName2 = "destField2";
-    createStoredStringField(srcFieldName, getSolrClient(jetty));
-    createStoredStringField(destFieldName1, getSolrClient(jetty));
-    createStoredStringField(destFieldName2, getSolrClient(jetty));
-
-    SchemaRequest.AddCopyField addCopyFieldRequest =
-        new SchemaRequest.AddCopyField(srcFieldName,
-            Arrays.asList(destFieldName1, destFieldName2));
-    SchemaResponse.UpdateResponse addCopyFieldResponse = addCopyFieldRequest.process(getSolrClient(jetty));
-    System.out.println(addCopyFieldResponse);
-    assertValidSchemaResponse(addCopyFieldResponse);
-
-    SchemaRequest.DeleteCopyField deleteCopyFieldRequest1 =
-        new SchemaRequest.DeleteCopyField(srcFieldName, Arrays.asList(destFieldName1));
-    assertValidSchemaResponse(deleteCopyFieldRequest1.process(getSolrClient(jetty)));
-
-    SchemaRequest.DeleteCopyField deleteCopyFieldRequest2 =
-        new SchemaRequest.DeleteCopyField(srcFieldName, Arrays.asList(destFieldName2));
-    assertValidSchemaResponse(deleteCopyFieldRequest2.process(getSolrClient(jetty)));
-  }
-
-  @Test
-  public void deleteCopyFieldShouldFailWhenOneOfTheFieldsDoesntExistInTheSchema() throws Exception {
-    String srcFieldName = "copyfield";
-    String destFieldName1 = "destField1", destFieldName2 = "destField2";
-    SchemaRequest.DeleteCopyField deleteCopyFieldsRequest =
-        new SchemaRequest.DeleteCopyField(srcFieldName,
-            Arrays.asList(destFieldName1, destFieldName2));
-    assertFailedSchemaResponse(() -> deleteCopyFieldsRequest.process(getSolrClient(jetty)),
-        "Copy field directive not found: '" + srcFieldName + "' -> '" + destFieldName1 + "'");
-  }
-
-  @Test
   public void testMultipleUpdateRequestAccuracy() throws Exception {
     String fieldTypeName = "accuracyTextFieldMulti";
     SchemaRequest.AddFieldType addFieldTypeRequest = createFieldTypeRequest(fieldTypeName);
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
index 57e8a99..1ec6983 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
@@ -114,10 +114,12 @@ public class SolrTestCaseHS extends SolrTestCaseJ4 {
     List<Doc> docList = new ArrayList<>(fullModel.values());
     Collections.sort(docList, sort);
     List sortedDocs = new ArrayList(rows);
-    for (Doc doc : docList) {
-      if (sortedDocs.size() >= rows) break;
-      Map<String,Object> odoc = toObject(doc, h.getCore().getLatestSchema(), fieldNames);
-      sortedDocs.add(toObject(doc, h.getCore().getLatestSchema(), fieldNames));
+    try (SolrCore core = h.getCore()) {
+      for (Doc doc : docList) {
+        if (sortedDocs.size() >= rows) break;
+        Map<String,Object> odoc = toObject(doc, core.getLatestSchema(), fieldNames);
+        sortedDocs.add(toObject(doc, core.getLatestSchema(), fieldNames));
+      }
     }
     return sortedDocs;
   }
@@ -287,7 +289,9 @@ public class SolrTestCaseHS extends SolrTestCaseJ4 {
       }
 
       public  void assertQ(SolrClient client, SolrParams args, String... tests) throws Exception {
-        SolrTestCaseHS.assertQ(h.getCore().getResourceLoader(), client, args, tests);
+        try (SolrCore core = h.getCore()) {
+          SolrTestCaseHS.assertQ(core.getResourceLoader(), client, args, tests);
+        }
       }
     }