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/10 15:58:08 UTC

[lucene-solr] branch reference_impl_dev updated: @1338 Start converting to persistent watchers, enable security manager, cleanup.

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


The following commit(s) were added to refs/heads/reference_impl_dev by this push:
     new 14050fe  @1338 Start converting to persistent watchers, enable security manager, cleanup.
14050fe is described below

commit 14050fe250f7bff2a8f477c0eefbaac6120aef78
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Wed Feb 10 09:53:46 2021 -0600

    @1338 Start converting to persistent watchers, enable security manager, cleanup.
---
 .../apache/lucene/store/NRTCachingDirectory.java   |   4 +-
 solr/bin/solr                                      |  12 +-
 .../apache/solr/cloud/OverseerTaskProcessor.java   |   2 -
 .../org/apache/solr/cloud/OverseerTaskQueue.java   |   5 +-
 .../java/org/apache/solr/cloud/ZkController.java   | 153 ++++----
 .../apache/solr/cloud/overseer/ZkStateWriter.java  |  29 +-
 .../java/org/apache/solr/core/CoreContainer.java   |   9 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |  19 +-
 .../apache/solr/core/StandardDirectoryFactory.java |   3 +-
 .../java/org/apache/solr/handler/IndexFetcher.java |   3 +-
 .../apache/solr/handler/RequestHandlerBase.java    |   2 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  25 +-
 .../StatelessScriptUpdateProcessorFactory.java     |   1 +
 .../apache/solr/AnalysisAfterCoreReloadTest.java   |   2 -
 .../test/org/apache/solr/TestCrossCoreJoin.java    |   3 -
 .../embedded/TestEmbeddedSolrServerSchemaAPI.java  |   2 -
 .../org/apache/solr/cloud/DeleteShardTest.java     |   3 +-
 .../org/apache/solr/cloud/MoveReplicaTest.java     |   2 -
 .../CollectionsAPIDistClusterPerZkTest.java        |   2 +-
 .../handler/FieldAnalysisRequestHandlerTest.java   |  47 ++-
 .../solr/handler/MoreLikeThisHandlerTest.java      | 239 ++++++-----
 .../solr/handler/TestSystemCollAutoCreate.java     |   1 -
 .../apache/solr/handler/V2ApiIntegrationTest.java  |   1 -
 .../component/MoreLikeThisComponentTest.java       |  45 +--
 .../solr/handler/component/StatsComponentTest.java | 181 +++++----
 .../test/org/apache/solr/schema/PolyFieldTest.java |   2 +
 .../org/apache/solr/search/ReturnFieldsTest.java   | 228 ++++++-----
 .../solr/spelling/SpellCheckCollatorTest.java      |  26 +-
 .../solr/spelling/suggest/SuggesterFSTTest.java    |   2 -
 .../solr/spelling/suggest/SuggesterTSTTest.java    |   2 -
 .../solr/spelling/suggest/SuggesterTest.java       |   6 +-
 .../solr/spelling/suggest/SuggesterWFSTTest.java   |   2 -
 .../solr/update/DirectUpdateHandlerTest.java       | 223 ++++++-----
 .../apache/solr/update/SolrIndexMetricsTest.java   |  41 +-
 .../StatelessScriptUpdateProcessorFactoryTest.java | 104 +++--
 .../test/org/apache/solr/util/AuthToolTest.java    |   3 -
 solr/server/build.gradle                           |   1 +
 solr/server/resources/jetty-logging.properties     |   1 -
 solr/server/resources/log4j2.xml                   | 219 ++++-------
 .../client/solrj/impl/BaseCloudSolrClient.java     |   3 -
 .../src/java/org/apache/solr/common/ParWork.java   |   8 +-
 .../java/org/apache/solr/common/cloud/Aliases.java |   2 +-
 .../org/apache/solr/common/cloud/SolrZkClient.java |  11 +-
 .../apache/solr/common/cloud/ZkStateReader.java    | 437 ++++++++++++---------
 .../org/apache/solr/cloud/SolrCloudTestCase.java   |   2 +-
 .../src/resources/logconf/log4j2-std-debug.xml     |   4 +-
 46 files changed, 1040 insertions(+), 1082 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
index fee7f51..09e8c70 100644
--- a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
@@ -212,7 +212,7 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
     // it for defensive reasons... or in case the app is
     // doing something custom (creating outputs directly w/o
     // using IndexWriter):
-  //  if (Boolean.getBoolean("solr.nrtDirSync")) { // nocommit)
+    if (Boolean.getBoolean("solr.nrtDirSync")) { // nocommit - we rollback instead of close on IW, and this behavior is unwanted
       IOUtils.close(() -> {
         if (!closed.getAndSet(true)) {
           for (String fileName : cacheDirectory.listAll()) {
@@ -220,7 +220,7 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
           }
         }
       }, cacheDirectory, in);
-  //  }
+    }
   }
 
   /** Subclass can override this to customize logic; return
diff --git a/solr/bin/solr b/solr/bin/solr
index 614546a..1af68fb 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -850,7 +850,7 @@ function stop_solr() {
   JETTY_RUN="$SOLR_PID_DIR"
   JETTY_PID="solr-$SOLR_PORT.pid"
 
-  echo -e "Sending stop command to Solr running on port $SOLR_PORT ...  "$JAVA" -cp $SOLR_TIP/server/lib/ext/solr-core*.jar $SOLR_SSL_OPTS $AUTHC_OPTS org.apache.solr.servlet.StopJetty "-DSTOP.PORT=$THIS_STOP_PORT" "-DSTOP.KEY=$STOP_KEY""
+  echo -e "Sending stop command to Solr running on port $SOLR_PORT ...  $JAVA -cp $SOLR_TIP/server/lib/ext/solr-core*.jar $SOLR_SSL_OPTS $AUTHC_OPTS org.apache.solr.servlet.StopJetty "-DSTOP.PORT=$THIS_STOP_PORT" "-DSTOP.KEY=$STOP_KEY""
   "$JAVA" -cp $SOLR_TIP/server/lib/ext/solr-core*.jar $SOLR_SSL_OPTS $AUTHC_OPTS "-DSTOP.PORT=$THIS_STOP_PORT" "-DSTOP.KEY=$STOP_KEY" org.apache.solr.servlet.StopJetty || true
 
   if [ ! -f "$SOLR_PID_DIR/$JETTY_PID" ] ; then
@@ -2037,8 +2037,7 @@ else
 fi
 
 # Enable java security manager (allowing filesystem access and other things)
-# MRM TODO: enable security manager
-if [ "${SOLR_SECURITY_MANAGER_ENABLED:-false}" == "true" ]; then
+if [ "${SOLR_SECURITY_MANAGER_ENABLED:-true}" == "true" ]; then
   SECURITY_MANAGER_OPTS=('-Djava.security.manager' \
       "-Djava.security.policy=${SOLR_SERVER_DIR}/etc/security.policy" \
       "-Djava.security.properties=${SOLR_SERVER_DIR}/etc/security.properties" \
@@ -2205,7 +2204,7 @@ function start_solr() {
   SOLR_START_OPTS+=("-Dsolr.log.dir=${SOLR_LOGS_DIR}" "-Djetty.port=${SOLR_PORT}" "-DSTOP.PORT=${stop_port}" "-DSTOP.KEY=${STOP_KEY}")
   SOLR_START_OPTS+=("-Duser.timezone=${SOLR_TIMEZONE}" "-Djetty.home=${SOLR_SERVER_DIR}" "-Dsolr.solr.home=${SOLR_HOME}" "-Dsolr.data.home=${SOLR_DATA_HOME}" "-Dsolr.install.dir=${SOLR_TIP}")
   SOLR_START_OPTS+=("-Dorg.apache.xml.dtm.DTMManager=org.apache.xml.dtm.ref.DTMManagerDefault" "-Djava.net.preferIPv4Stack=true" "-Dsolr.default.confdir=${DEFAULT_CONFDIR}")
-  SOLR_START_OPTS+=("-XX:-UseBiasedLocking")
+  SOLR_START_OPTS+=("-XX:-UseBiasedLocking" "-Dorg.eclipse.jetty.util.log.class=org.eclipse.jetty.util.log.Slf4jLog")
 
   # '-OmitStackTraceInFastThrow' ensures stack traces in errors,
   # users who don't care about useful error msgs can override in SOLR_OPTS with +OmitStackTraceInFastThrow
@@ -2239,11 +2238,12 @@ function start_solr() {
 
   if [ "$run_in_foreground" == "true" ]; then
     #set -x
+    SOLR_START_OPTS+=("-Djetty.state=${SOLR_PID_DIR}/jetty.state" "-DLOG4J_APPENDER=STDOUT")
     exec "$JAVA" "${SOLR_START_OPTS[@]}" -jar start.jar "${SOLR_JETTY_CONFIG[@]}"
   else
     # run Solr in the background MRM-TODO: Move relevant parts in from jetty.sh
 
-    SOLR_START_OPTS+=("-Djetty.state=${SOLR_PID_DIR}/jetty.state" "-Dsolr.log.muteconsole=true")
+    SOLR_START_OPTS+=("-Djetty.state=${SOLR_PID_DIR}/jetty.state" "-DLOG4J_APPENDER=MainLogFile")
 
     export JAVA=$JAVA
     export JETTY_HOME="${SOLR_SERVER_DIR}"
@@ -2251,7 +2251,7 @@ function start_solr() {
     export JETTY_RUN="$SOLR_PID_DIR"
     export JETTY_PID="$SOLR_PID_DIR/solr-$SOLR_PORT.pid"
     export JETTY_ARGS="${SOLR_JETTY_CONFIG[*]}"
-    export JAVA_OPTIONS="${SOLR_START_OPTS[@]}"
+    export JAVA_OPTIONS="${SOLR_START_OPTS[*]}"
     export JETTY_STATE="${SOLR_PID_DIR}/jetty.state"
     bash "${SOLR_SERVER_DIR}"/../bin/jetty.sh -d start
   fi
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
index f5b00d2..f804e70e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
@@ -116,8 +116,6 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
 
   };
 
-  private final Object waitLock = new Object();
-
   protected final OverseerMessageHandlerSelector selector;
 
   private final String thisNode;
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
index ebe1d30..9ccea46 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
@@ -294,11 +294,10 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
       if (bytes == null || bytes.length == 0) {
         log.error("Found no data at response node, Overseer likely changed {}", watchID);
       }
-      // create the event before deleting the node, otherwise we can get the deleted
-      // event from the watcher.
+
       QueueEvent event =  new QueueEvent(watchID, bytes, watcher.getWatchedEvent());
 
-      // zookeeper.delete(watchID,  -1);
+      zookeeper.delete(watchID,-1);
 
       return event;
     } finally {
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 7e73fe6..b871421 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -66,6 +66,7 @@ import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.servlet.SolrLifcycleListener;
 import org.apache.solr.update.UpdateLog;
+import org.apache.zookeeper.AddWatchMode;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
@@ -100,7 +101,6 @@ import java.nio.file.Paths;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -2095,20 +2095,24 @@ public class ZkController implements Closeable, Runnable {
   }
 
   private void unregisterConfListener(String confDir, Runnable listener) {
-    synchronized (confDirectoryListeners) {
-      final Set<Runnable> listeners = confDirectoryListeners.get(confDir);
-      if (listeners == null) {
-        log.warn("{} has no more registered listeners, but a live one attempted to unregister!", confDir);
-        return;
-      }
-      if (listeners.remove(listener)) {
-        log.debug("removed listener for config directory [{}]", confDir);
-      }
-      if (listeners.isEmpty()) {
-        // no more listeners for this confDir, remove it from the map
-        log.debug("No more listeners for config directory [{}]", confDir);
-        confDirectoryListeners.remove(confDir);
-      }
+    final ConfListeners confListeners = confDirectoryListeners.get(confDir);
+    if (confListeners == null) {
+      log.warn("{} has no more registered listeners, but a live one attempted to unregister!", confDir);
+      return;
+    }
+    if (confListeners.confDirListeners.remove(listener)) {
+      if (log.isDebugEnabled()) log.debug("removed listener for config directory [{}]", confDir);
+    }
+    if (confListeners.confDirListeners.isEmpty()) {
+      // no more listeners for this confDir, remove it from the map
+      if (log.isDebugEnabled()) log.debug("No more listeners for config directory [{}]", confDir);
+      zkClient.getSolrZooKeeper().removeWatches(COLLECTIONS_ZKNODE, confListeners.watcher, Watcher.WatcherType.Any, true, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception removing watch for " + path, ex);
+        }
+      }, "confWatcher");
+      confDirectoryListeners.remove(confDir);
     }
   }
 
@@ -2122,52 +2126,65 @@ public class ZkController implements Closeable, Runnable {
     if (listener == null) {
       throw new NullPointerException("listener cannot be null");
     }
-    synchronized (confDirectoryListeners) {
-      final Set<Runnable> confDirListeners = getConfDirListeners(confDir);
-      confDirListeners.add(listener);
-      core.addCloseHook(new CloseHook() {
-        @Override
-        public void preClose(SolrCore core) {
-          unregisterConfListener(confDir, listener);
-        }
+    final ConfListeners confDirListeners = getConfDirListeners(confDir);
+    confDirListeners.confDirListeners.add(listener);
+    core.addCloseHook(new CloseHook() {
+      @Override
+      public void preClose(SolrCore core) {
+        unregisterConfListener(confDir, listener);
+      }
 
-        @Override
-        public void postClose(SolrCore core) {
-        }
-      });
+      @Override
+      public void postClose(SolrCore core) {
+      }
+    });
+  }
+
+  private static class ConfListeners {
+
+    private Set<Runnable> confDirListeners;
+    private final Watcher watcher;
+
+    ConfListeners( Set<Runnable> confDirListeners, Watcher watcher) {
+      this.confDirListeners = confDirListeners;
+      this.watcher = watcher;
     }
   }
 
-  // this method is called in a protected confDirListeners block
-  private Set<Runnable> getConfDirListeners(final String confDir) {
-    assert Thread.holdsLock(confDirectoryListeners) : "confDirListeners lock not held by thread";
-    Set<Runnable> confDirListeners = confDirectoryListeners.get(confDir);
+  private ConfListeners getConfDirListeners(final String confDir) {
+    ConfListeners confDirListeners = confDirectoryListeners.get(confDir);
     if (confDirListeners == null) {
-      log.debug("watch zkdir {}" , confDir);
-      confDirListeners = new HashSet<>();
+      if (log.isDebugEnabled()) log.debug("watch zkdir {}" , confDir);
+      ConfDirWatcher watcher = new ConfDirWatcher(confDir, cc, confDirectoryListeners);
+      confDirListeners = new ConfListeners(ConcurrentHashMap.newKeySet(), watcher);
       confDirectoryListeners.put(confDir, confDirListeners);
-      setConfWatcher(confDir, new WatcherImpl(confDir), null);
+      setConfWatcher(confDir, watcher, null, cc, confDirectoryListeners, cc.getZkController().getZkClient());
     }
     return confDirListeners;
   }
 
-  private final Map<String, Set<Runnable>> confDirectoryListeners = new ConcurrentHashMap<>();
+  private final Map<String, ConfListeners> confDirectoryListeners = new ConcurrentHashMap<>();
 
-  private class
-  WatcherImpl implements Watcher {
+  private static class ConfDirWatcher implements Watcher {
     private final String zkDir;
+    private final CoreContainer cc;
+    private final SolrZkClient zkClient;
+    private final Map<String, ConfListeners> confDirectoryListeners;
 
-    private WatcherImpl(String dir) {
+    private ConfDirWatcher(String dir, CoreContainer cc, Map<String, ConfListeners> confDirectoryListeners) {
       this.zkDir = dir;
+      this.cc = cc;
+      this.zkClient = cc.getZkController().getZkClient();
+      this.confDirectoryListeners = confDirectoryListeners;
     }
 
     @Override
     public void process(WatchedEvent event) {
       // session events are not change events, and do not remove the watcher
-      if (Event.EventType.None.equals(event.getType()) || isClosed() || cc.isShutDown()) {
+      if (Event.EventType.None.equals(event.getType())) {
         return;
       }
-      if (isClosed() || getCoreContainer().isShutDown() || isDcCalled()) {
+      if (cc.getZkController().isClosed() || cc.isShutDown() || cc.getZkController().isDcCalled()) {
         return;
       }
       Stat stat = null;
@@ -2180,56 +2197,44 @@ public class ZkController implements Closeable, Runnable {
         return;
       }
 
-      boolean resetWatcher = false;
-      try {
-        resetWatcher = fireEventListeners(zkDir);
-      } finally {
-
-        if (!isClosed() && !cc.isShutDown()) {
-          if (resetWatcher) {
-            setConfWatcher(zkDir, this, stat);
-          } else {
-            log.debug("A node got unwatched for {}", zkDir);
-          }
-        }
-      }
+      fireEventListeners(zkDir, confDirectoryListeners, cc);
     }
   }
 
-  private boolean fireEventListeners(String zkDir) {
+  private static boolean fireEventListeners(String zkDir, Map<String, ConfListeners> confDirectoryListeners, CoreContainer cc) {
     if (cc.isShutDown()) {
       return false;
     }
 
     // if this is not among directories to be watched then don't set the watcher anymore
     if (!confDirectoryListeners.containsKey(zkDir)) {
-      log.debug("Watcher on {} is removed ", zkDir);
+      if (log.isDebugEnabled()) log.debug("Watcher on {} is removed ", zkDir);
       return false;
     }
 
-    final Set<Runnable> listeners = confDirectoryListeners.get(zkDir);
+    final Set<Runnable> listeners = confDirectoryListeners.get(zkDir).confDirListeners;
     if (listeners != null) {
-
-      // run these in a separate thread because this can be long running
-      if (cc.isShutDown() || isDcCalled()) {
+      if (cc.isShutDown() || cc.getZkController().isDcCalled()) {
         return false;
       }
-      listeners.forEach(Runnable::run);
-
+      listeners.forEach(runnable -> cc.coreContainerExecutor.submit(runnable));
     }
     return true;
   }
 
-  private void setConfWatcher(String zkDir, Watcher watcher, Stat stat) {
-    if (isClosed() || isDcCalled() || getCoreContainer().isShutDown()) {
-      return;
-    }
+  private static void setConfWatcher(String zkDir, Watcher watcher, Stat stat, CoreContainer cc, Map<String, ConfListeners> confDirectoryListeners, SolrZkClient zkClient) {
     try {
-      Stat newStat = zkClient.exists(zkDir, watcher);
+      zkClient.getSolrZooKeeper().addWatch(zkDir, watcher, AddWatchMode.PERSISTENT, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception creating watch for " + path, ex);
+        }
+      }, "confWatcher");
+      Stat newStat = zkClient.exists(zkDir, null);
       if (stat != null && newStat.getVersion() > stat.getVersion()) {
         //a race condition where a we missed an event fired
         //so fire the event listeners
-        fireEventListeners(zkDir);
+        fireEventListeners(zkDir, confDirectoryListeners, cc);
       }
     } catch (KeeperException e) {
       log.error("failed to set watcher for conf dir {} ", zkDir);
@@ -2242,7 +2247,7 @@ public class ZkController implements Closeable, Runnable {
   }
 
   public OnReconnect getConfigDirListener() {
-    return new ZkControllerOnReconnect(confDirectoryListeners);
+    return new ZkControllerOnReconnect(confDirectoryListeners, cc);
   }
 
   /**
@@ -2313,19 +2318,21 @@ public class ZkController implements Closeable, Runnable {
 //    }
   }
 
-  private class ZkControllerOnReconnect implements OnReconnect {
+  private static class ZkControllerOnReconnect implements OnReconnect {
 
-    private final Map<String,Set<Runnable>> confDirectoryListeners;
+    private final Map<String, ConfListeners> confDirectoryListeners;
+    private final CoreContainer cc;
 
-    ZkControllerOnReconnect(Map<String, Set<Runnable>> confDirectoryListeners) {
+    ZkControllerOnReconnect(Map<String, ConfListeners> confDirectoryListeners, CoreContainer cc) {
       this.confDirectoryListeners = confDirectoryListeners;
+      this.cc = cc;
     }
     
     @Override
     public void command() {
         confDirectoryListeners.forEach((s, runnables) -> {
-          setConfWatcher(s, new WatcherImpl(s), null);
-          fireEventListeners(s);
+          setConfWatcher(s, new ConfDirWatcher(s, cc, confDirectoryListeners), null, cc, confDirectoryListeners, cc.getZkController().getZkClient());
+          fireEventListeners(s, confDirectoryListeners, cc);
         });
     }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
index e23a8c1..9bf3836 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
@@ -286,7 +286,7 @@ public class ZkStateWriter {
           //            return new SliceMutator(cloudManager).addRoutingRule(clusterState, message);
           //          case REMOVEROUTINGRULE:
           //            return new SliceMutator(cloudManager).removeRoutingRule(clusterState, message);
-          case UPDATESHARDSTATE:
+          case UPDATESHARDSTATE:  // MRM TODO: look at how we handle this and make it so it can use StatePublisher
             String collection = message.getStr("collection");
             message.getProperties().remove("collection");
             message.getProperties().remove(StatePublisher.OPERATION);
@@ -469,6 +469,7 @@ public class ZkStateWriter {
                 ZkNodeProps updates = stateUpdates.get(collection.getName());
                 if (updates != null) {
                   updates.getProperties().clear();
+                  writeStateUpdates(lastVersion, collection, updates);
                 }
               }
 
@@ -499,17 +500,7 @@ public class ZkStateWriter {
             if (dirtyState.contains(collection.getName())) {
               ZkNodeProps updates = stateUpdates.get(collection.getName());
               if (updates != null) {
-                String stateUpdatesPath = ZkStateReader.getCollectionStateUpdatesPath(collection.getName());
-                if (log.isDebugEnabled()) log.debug("write state updates for collection {} {}", collection.getName(), updates);
-                dirtyState.remove(collection.getName());
-                try {
-                  reader.getZkClient().setData(stateUpdatesPath, Utils.toJSON(updates), -1, true);
-                } catch (KeeperException.NoNodeException e) {
-                    if (log.isDebugEnabled()) log.debug("No node found for state.json", e);
-                    lastVersion.set(-1);
-                    trackVersions.remove(collection.getName());
-                    // likely deleted
-                  }
+                writeStateUpdates(lastVersion, collection, updates);
               }
             }
 
@@ -546,6 +537,20 @@ public class ZkStateWriter {
     //    }
   }
 
+  private void writeStateUpdates(AtomicInteger lastVersion, DocCollection collection, ZkNodeProps updates) throws KeeperException, InterruptedException {
+    String stateUpdatesPath = ZkStateReader.getCollectionStateUpdatesPath(collection.getName());
+    if (log.isDebugEnabled()) log.debug("write state updates for collection {} {}", collection.getName(), updates);
+    dirtyState.remove(collection.getName());
+    try {
+      reader.getZkClient().setData(stateUpdatesPath, Utils.toJSON(updates), -1, true);
+    } catch (KeeperException.NoNodeException e) {
+      if (log.isDebugEnabled()) log.debug("No node found for state.json", e);
+      lastVersion.set(-1);
+      trackVersions.remove(collection.getName());
+      // likely deleted
+    }
+  }
+
   private void waitForStateWePublishedToComeBack() {
     cs.forEachCollection(collection -> {
       if (dirtyStructure.contains(collection.getName()) || dirtyState.contains(collection.getName())) {
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 cd413dc..584ea16 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -117,6 +117,7 @@ import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGI
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
@@ -1625,7 +1626,7 @@ public class CoreContainer implements Closeable {
                 while (new File(dataDir).exists()) {
                   try {
                     Files.walk(new File(dataDir).toPath()).sorted(Comparator.reverseOrder()).forEach(new FileConsumer());
-                  } catch (NoSuchFileException e) {
+                  } catch (NoSuchFileException | UncheckedIOException e) {
 
                   }
                 }
@@ -2043,7 +2044,7 @@ public class CoreContainer implements Closeable {
           while (Files.exists(cd.getInstanceDir())) {
             try {
               Files.walk(cd.getInstanceDir()).sorted(Comparator.reverseOrder()).forEach(new FileConsumer());
-            } catch (NoSuchFileException e) {
+            } catch (NoSuchFileException | UncheckedIOException e) {
 
             }
           }
@@ -2372,8 +2373,8 @@ public class CoreContainer implements Closeable {
     public void accept(Path file) {
       try {
         Files.delete(file);
-      }catch (IOException e) {
-        log.warn("Could not delete file {}", file, e);
+      } catch (IOException e) {
+        log.info("Could not delete file {} {} {}", file, e.getClass().getName(), e.getMessage());
       }
     }
   }
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 109ecd7..3c3117a 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -142,6 +142,7 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
 import java.io.Writer;
 import java.lang.invoke.MethodHandles;
 import java.lang.reflect.Constructor;
@@ -2618,15 +2619,8 @@ public final class SolrCore implements SolrInfoBean, Closeable {
         if (_searcher != null && !forceNew) {
           if (returnSearcher) {
             _searcher.incref();
-
-            if (log.isDebugEnabled()) {
-              log.debug("getSearcher(boolean, boolean, Future[], boolean) - end return={}", _searcher);
-            }
             return _searcher;
           } else {
-            if (log.isDebugEnabled()) {
-              log.debug("getSearcher(boolean, boolean, Future[], boolean) - end return=null");
-            }
             return null;
           }
         }
@@ -2644,15 +2638,8 @@ public final class SolrCore implements SolrInfoBean, Closeable {
         if (_searcher != null && !forceNew) {
           if (returnSearcher) {
             _searcher.incref();
-
-            if (log.isDebugEnabled()) {
-              log.debug("getSearcher(boolean, boolean, Future[], boolean) - end return={}", _searcher);
-            }
             return _searcher;
           } else {
-            if (log.isDebugEnabled()) {
-              log.debug("getSearcher(boolean, boolean, Future[], boolean) - end return=null");
-            }
             return null;
           }
         }
@@ -3422,7 +3409,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
           while (Files.exists(dataDir)) {
             try {
               Files.walk(dataDir).sorted(Comparator.reverseOrder()).forEach(new CoreContainer.FileConsumer());
-            } catch (NoSuchFileException e) {
+            } catch (NoSuchFileException | UncheckedIOException e) {
 
             }
           }
@@ -3435,7 +3422,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
         while (Files.exists(cd.getInstanceDir())) {
           try {
             Files.walk(cd.getInstanceDir()).sorted(Comparator.reverseOrder()).forEach(new CoreContainer.FileConsumer());
-          } catch (NoSuchFileException e) {
+          } catch (NoSuchFileException | UncheckedIOException e) {
 
           }
         }
diff --git a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
index 1a6c341..d9ab29a 100644
--- a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
@@ -17,6 +17,7 @@
 package org.apache.solr.core;
 import java.io.File;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.AtomicMoveNotSupportedException;
 import java.nio.file.FileSystems;
@@ -112,7 +113,7 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
     while (Files.exists(dirFile)) {
       try {
         Files.walk(dirFile).sorted(Comparator.reverseOrder()).forEach(new CoreContainer.FileConsumer());
-      } catch (NoSuchFileException e) {
+      } catch (NoSuchFileException | UncheckedIOException e) {
 
       }
     }
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 8a05f8d..04674ed 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -94,6 +94,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
 import java.io.Writer;
 import java.lang.invoke.MethodHandles;
 import java.nio.ByteBuffer;
@@ -1562,7 +1563,7 @@ public class IndexFetcher {
 
       try {
         Files.walk(dir.toPath()).sorted(Comparator.reverseOrder()).forEach(new CoreContainer.FileConsumer());
-      } catch (NoSuchFileException e) {
+      } catch (NoSuchFileException | UncheckedIOException e) {
 
       } catch (IOException e) {
         log.warn("Unable to delete directory : {}", dir, e);
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index a9491b6..a0de453 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -223,7 +223,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
         ParWork.propagateInterrupt(e);
         throw new AlreadyClosedException(e);
     } catch (Exception e) {
-      log.error("Error get", e);
+      log.error("Exception handling request", e);
       if (req.getCore() != null) {
         boolean isTragic = req.getCore().getCoreContainer().checkTragicException(req.getCore());
         if (isTragic) {
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 0a097f8..417b9b9 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -190,7 +190,7 @@ public class HttpSolrCall {
     // put the core container in request attribute
     req.setAttribute("org.apache.solr.CoreContainer", cores);
     path = ServletUtils.getPathAfterContext(req);
-    if (log.isDebugEnabled()) log.debug("Path is parsed as {}", path);
+    if (log.isTraceEnabled()) log.trace("Path is parsed as {}", path);
   }
 
   public String getPath() {
@@ -224,11 +224,11 @@ public class HttpSolrCall {
       }
     }
 
-    if (log.isDebugEnabled()) log.debug("Full path {} Path is parsed as {} managment path is {}", req.getRequestURI() + req.getQueryString(), path);
+    if (log.isTraceEnabled()) log.trace("Full path {} Path is parsed as {} managment path is {}", req.getRequestURI() + req.getQueryString(), path);
       // Check for container handlers
 
     handler = cores.getRequestHandler(path);
-    if (log.isDebugEnabled()) log.debug("Check for handler {} returned {} handlers={}", path, handler, cores.getRequestHandlers().keySet());
+    if (log.isTraceEnabled()) log.trace("Check for handler {} returned {} handlers={}", path, handler, cores.getRequestHandlers().keySet());
     if (handler != null) {
       solrReq = SolrRequestParsers.DEFAULT.parse(null, path, req);
       solrReq.getContext().put(CoreContainer.class.getName(), cores);
@@ -267,11 +267,6 @@ public class HttpSolrCall {
 
       if (log.isDebugEnabled()) log.debug("tried to get core by name {} got {}, existing cores {} found={}", origCorename, core, cores.getAllCoreNames(), core != null);
 
-//      if (core == null) {
-//        // nocommit
-//        log.info("tried to get core by name {} got {}, existing cores {} found={}", origCorename, core, cores.getAllCoreNames(), core != null);
-//      }
-
       if (core != null) {
         path = path.substring(idx);
         if (log.isDebugEnabled()) log.debug("Path is parsed as {}", path);
@@ -448,9 +443,9 @@ public class HttpSolrCall {
       }
     }
     if (uniqueList != null) {
-      return new ArrayList<>(uniqueList);
+      return Collections.unmodifiableList(new ArrayList<>(uniqueList));
     } else {
-      return result;
+      return Collections.unmodifiableList(result);
     }
   }
 
@@ -458,10 +453,10 @@ public class HttpSolrCall {
    * Extract handler from the URL path if not set.
    */
   protected void extractHandlerFromURLPath(SolrRequestParsers parser) throws Exception {
-    if (log.isDebugEnabled()) log.debug("Extract handler from url path {} {}", handler, path);
+    if (log.isTraceEnabled()) log.trace("Extract handler from url path {} {}", handler, path);
     if (handler == null && path.length() > 1) { // don't match "" or "/" as valid path
       handler = core.getRequestHandler(path);
-      if (log.isDebugEnabled()) log.debug("handler={} name={}", handler, path);
+      if (log.isTraceEnabled()) log.trace("handler={} name={}", handler, path);
       // no handler yet but <requestDispatcher> allows us to handle /select with a 'qt' param
       if (handler == null && parser.isHandleSelect()) {
         if ("/select".equals(path) || "/select/".equals(path)) {
@@ -688,7 +683,7 @@ public class HttpSolrCall {
   void destroy() {
     try {
       if (solrReq != null) {
-        log.debug("Closing out SolrRequest: {}", solrReq);
+        if (log.isTraceEnabled()) log.trace("Closing out SolrRequest: {}", solrReq);
         IOUtils.closeQuietly(solrReq);
       }
     } finally {
@@ -1078,9 +1073,9 @@ public class HttpSolrCall {
     }
 
     List<Replica> replicas = collection.getReplicas(cores.getZkController().getNodeName());
-    if (log.isDebugEnabled()) log.debug("replicas for node {} {}", replicas, cores.getZkController().getNodeName());
+    if (log.isTraceEnabled()) log.trace("replicas for node {} {}", replicas, cores.getZkController().getNodeName());
     SolrCore returnCore = randomlyGetSolrCore(cores.getZkController().getZkStateReader().getLiveNodes(), replicas);
-    if (log.isDebugEnabled()) log.debug("returning core by collection {}", returnCore == null ? null : returnCore.getName());
+    if (log.isTraceEnabled()) log.trace("returning core by collection {}", returnCore == null ? null : returnCore.getName());
     return  returnCore;
   }
 
diff --git a/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
index d2f5a07..b84fcec 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
@@ -517,6 +517,7 @@ public class StatelessScriptUpdateProcessorFactory extends UpdateRequestProcesso
     }
 
     public Reader openReader(SolrResourceLoader resourceLoader) throws IOException {
+      // TODO: close the inputstream as well
       InputStream input = resourceLoader.openResource(fileName);
       return org.apache.lucene.util.IOUtils.getDecodingReader
         (input, StandardCharsets.UTF_8);
diff --git a/solr/core/src/test/org/apache/solr/AnalysisAfterCoreReloadTest.java b/solr/core/src/test/org/apache/solr/AnalysisAfterCoreReloadTest.java
index 7d133be..4941bd1 100644
--- a/solr/core/src/test/org/apache/solr/AnalysisAfterCoreReloadTest.java
+++ b/solr/core/src/test/org/apache/solr/AnalysisAfterCoreReloadTest.java
@@ -27,12 +27,10 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.core.SolrCore;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 
 import java.io.File;
 import java.io.IOException;
 
-@Ignore // MRM-TEST TODO: finish closing things right
 public class AnalysisAfterCoreReloadTest extends SolrTestCaseJ4 {
   
   private static String tmpSolrHome;
diff --git a/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java b/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java
index dfa2fed..57ad11a 100644
--- a/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java
+++ b/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java
@@ -35,10 +35,8 @@ import org.apache.solr.search.join.TestScoreJoinQPNoScore;
 import org.apache.solr.servlet.DirectSolrConnection;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
-@Ignore // MRM-TEST TODO: finish closing things right
 public class TestCrossCoreJoin extends SolrTestCaseJ4 {
 
   private SolrCore fromCore;
@@ -138,7 +136,6 @@ public class TestCrossCoreJoin extends SolrTestCaseJ4 {
   @After
   public void nukeAll() throws TimeoutException {
     deleteCore();
-    fromCore.closeAndWait();
     fromCore = null;
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/client/solrj/embedded/TestEmbeddedSolrServerSchemaAPI.java b/solr/core/src/test/org/apache/solr/client/solrj/embedded/TestEmbeddedSolrServerSchemaAPI.java
index da2c11d..88b7177 100644
--- a/solr/core/src/test/org/apache/solr/client/solrj/embedded/TestEmbeddedSolrServerSchemaAPI.java
+++ b/solr/core/src/test/org/apache/solr/client/solrj/embedded/TestEmbeddedSolrServerSchemaAPI.java
@@ -30,10 +30,8 @@ import org.apache.solr.common.SolrException;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
-@Ignore // MRM TEST TODO: core closing
 public class TestEmbeddedSolrServerSchemaAPI extends SolrTestCaseJ4 {
 
   private String fieldName = "VerificationTest";
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
index e430648..bb45f69 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
@@ -56,6 +56,7 @@ public class DeleteShardTest extends SolrCloudTestCase {
   }
 
   @Test
+  // MRM TODO: we need to pump slice changes through the StatePublish mechanism
   public void test() throws Exception {
 
     final String collection = "deleteShard";
@@ -88,8 +89,6 @@ public class DeleteShardTest extends SolrCloudTestCase {
 
   protected void setSliceState(String collection, String slice, State state) throws Exception {
 
-    CloudHttp2SolrClient client = cluster.getSolrClient();
-
     // TODO can this be encapsulated better somewhere?
     DistributedQueue inQueue =  cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getOverseer().getStateUpdateQueue();
     Map<String, Object> propMap = new HashMap<>();
diff --git a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
index f9d4b7d..582a57b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
@@ -37,7 +37,6 @@ import org.apache.solr.util.IdUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,7 +50,6 @@ import java.util.Map;
 import java.util.Set;
 
 @LuceneTestCase.SuppressCodecs({"MockRandom", "Direct", "SimpleText"})
-@Ignore // MRM-TEST TODO:
 public class MoveReplicaTest extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistClusterPerZkTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistClusterPerZkTest.java
index 6d9556b..5d2c829 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistClusterPerZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistClusterPerZkTest.java
@@ -303,7 +303,7 @@ public class CollectionsAPIDistClusterPerZkTest extends SolrCloudTestCase {
     final String collectionName = "reloaded_collection";
     CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2).setMaxShardsPerNode(10).process(cluster.getSolrClient());
 
-    CollectionAdminRequest.reloadCollection(collectionName).processAsync(cluster.getSolrClient());
+    CollectionAdminRequest.reloadCollection(collectionName).process(cluster.getSolrClient());
   }
 
   private void checkInstanceDirs(JettySolrRunner jetty) throws IOException {
diff --git a/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java
index 7ad2964..72ea550 100644
--- a/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java
@@ -40,13 +40,13 @@ import org.apache.solr.common.params.AnalysisParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.TextField;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 /**
@@ -55,7 +55,6 @@ import org.junit.Test;
  *
  * @since solr 1.4
  */
-@Ignore // MRM-TEST TODO: finish closing things right
 public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestBase {
   
   private FieldAnalysisRequestHandler handler;
@@ -77,8 +76,10 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
     FieldAnalysisRequest request = new FieldAnalysisRequest();
     request.addFieldType("pint");
     request.setFieldValue("5");
-    
-    NamedList<NamedList> nl = handler.handleAnalysisRequest(request, h.getCore().getLatestSchema());
+
+    SolrCore core = h.getCore();
+    NamedList<NamedList> nl = handler.handleAnalysisRequest(request, core.getLatestSchema());
+    core.close();
     NamedList pintNL = (NamedList)nl.get("field_types").get("pint");
     NamedList indexNL = (NamedList)pintNL.get("index");
     ArrayList analyzerNL = (ArrayList)indexNL.get("org.apache.solr.schema.FieldType$DefaultAnalyzer$1");
@@ -97,7 +98,7 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
     params.add(AnalysisParams.FIELD_VALUE, "the quick red fox jumped over the lazy brown dogs");
     params.add(CommonParams.Q, "fox brown");
 
-    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params);
+    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params, true);
     FieldAnalysisRequest request = handler.resolveAnalysisRequest(req);
     List<String> fieldNames = request.getFieldNames();
     assertEquals("Expecting 2 field names", 2, fieldNames.size());
@@ -114,20 +115,20 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
 
     // testing overide of query value using analysis.query param
     params.add(AnalysisParams.QUERY, "quick lazy");
-    req=new LocalSolrQueryRequest(h.getCore(), params);
+    req=new LocalSolrQueryRequest(h.getCore(), params, true);
     request = handler.resolveAnalysisRequest(req);
     assertEquals("quick lazy", request.getQuery());
     req.close();
 
     // testing analysis.showmatch param
     params.add(AnalysisParams.SHOW_MATCH, "false");
-    req=new LocalSolrQueryRequest(h.getCore(), params);
+    req=new LocalSolrQueryRequest(h.getCore(), params, true);
     request = handler.resolveAnalysisRequest(req);
     assertFalse(request.isShowMatch());
     req.close();
 
     params.set(AnalysisParams.SHOW_MATCH, "true");
-    req=new LocalSolrQueryRequest(h.getCore(), params);
+    req=new LocalSolrQueryRequest(h.getCore(), params, true);
     request = handler.resolveAnalysisRequest(req);
     assertTrue(request.isShowMatch());
     req.close();
@@ -135,7 +136,7 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
     // testing absence of query value
     params.remove(CommonParams.Q);
     params.remove(AnalysisParams.QUERY);
-    req=new LocalSolrQueryRequest(h.getCore(), params);
+    req=new LocalSolrQueryRequest(h.getCore(), params, true);
     request = handler.resolveAnalysisRequest(req);
     assertNull(request.getQuery());
     req.close();
@@ -145,25 +146,21 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
     params.add(CommonParams.Q, "quick lazy");
     request = handler.resolveAnalysisRequest(req);
     assertEquals("quick lazy", request.getQuery());
-    req.close();
 
     // test absence of index-time value and presence of query
     params.remove(CommonParams.Q);
     params.add(AnalysisParams.QUERY, "quick lazy");
     request = handler.resolveAnalysisRequest(req);
     assertEquals("quick lazy", request.getQuery());
-    req.close();
 
     // must fail if all of q, analysis.query or analysis.value are absent
     params.remove(CommonParams.Q);
     params.remove(AnalysisParams.QUERY);
     params.remove(AnalysisParams.FIELD_VALUE);
-    try (SolrQueryRequest solrQueryRequest = new LocalSolrQueryRequest(h.getCore(), params)) {
+    try (SolrQueryRequest solrQueryRequest = new LocalSolrQueryRequest(h.getCore(), params, true)) {
       SolrException ex = expectThrows(SolrException.class, () -> handler.resolveAnalysisRequest(solrQueryRequest));
       assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, ex.code());
     }
-
-    req.close();
   }
 
   /**
@@ -182,7 +179,9 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
     request.setQuery("fox brown");
     request.setShowMatch(true);
 
-    NamedList<NamedList> result = handler.handleAnalysisRequest(request, h.getCore().getLatestSchema());
+    SolrCore core = h.getCore();
+    NamedList<NamedList> result = handler.handleAnalysisRequest(request, core.getLatestSchema());
+    core.close();
     assertTrue("result is null and it shouldn't be", result != null);
 
     NamedList<NamedList> fieldTypes = result.get("field_types");
@@ -353,7 +352,9 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
     request.setFieldValue("<html><body>whátëvêr</body></html>");
     request.setShowMatch(false);
 
-    NamedList<NamedList> result = handler.handleAnalysisRequest(request, h.getCore().getLatestSchema());
+    SolrCore core = h.getCore();
+    NamedList<NamedList> result = handler.handleAnalysisRequest(request, core.getLatestSchema());
+    core.close();
     assertTrue("result is null and it shouldn't be", result != null);
 
     NamedList<NamedList> fieldTypes = result.get("field_types");
@@ -381,7 +382,9 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
     request.setFieldValue("hi, 3456-12 a Test");
     request.setShowMatch(false);
 
-    NamedList<NamedList> result = handler.handleAnalysisRequest(request, h.getCore().getLatestSchema());
+    SolrCore core = h.getCore();
+    NamedList<NamedList> result = handler.handleAnalysisRequest(request, core.getLatestSchema());
+    core.close();
     assertTrue("result is null and it shouldn't be", result != null);
 
     NamedList<NamedList> fieldTypes = result.get("field_types");
@@ -425,7 +428,9 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
     request.addFieldType("location_rpt");
     request.setFieldValue("MULTIPOINT ((10 40), (40 30), (20 20), (30 10))");
 
-    NamedList<NamedList> result = handler.handleAnalysisRequest(request, h.getCore().getLatestSchema());
+    SolrCore core = h.getCore();
+    NamedList<NamedList> result = handler.handleAnalysisRequest(request, core.getLatestSchema());
+    core.close();
     NamedList<List<NamedList>> tokens = (NamedList<List<NamedList>>)
         ((NamedList)result.get("field_types").get("location_rpt")).get("index");
     List<NamedList> tokenList = tokens.get("org.apache.lucene.spatial.prefix.PrefixTreeStrategy$ShapeTokenStream");
@@ -475,8 +480,10 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB
   public void testNoDefaultField() throws Exception {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.add(CommonParams.Q, "fox brown");
-    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params);
-    handler.resolveAnalysisRequest(req);
+    try (SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params, true)) {
+      handler.resolveAnalysisRequest(req);
+    }
+
   }
 
   /** A custom impl of a standard attribute impl; test this instance is used. */
diff --git a/solr/core/src/test/org/apache/solr/handler/MoreLikeThisHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/MoreLikeThisHandlerTest.java
index 7c9d30a..fcb590a 100644
--- a/solr/core/src/test/org/apache/solr/handler/MoreLikeThisHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/MoreLikeThisHandlerTest.java
@@ -34,7 +34,6 @@ import org.junit.Test;
 /**
  * TODO -- this needs to actually test the results/query etc
  */
-@Ignore // MRM-TEST TODO: finish closing things right
 public class MoreLikeThisHandlerTest extends SolrTestCaseJ4 {
   @BeforeClass
   public static void moreLikeThisBeforeClass() throws Exception {
@@ -44,146 +43,134 @@ public class MoreLikeThisHandlerTest extends SolrTestCaseJ4 {
   @Test
   public void testInterface() throws Exception
   {
-    SolrCore core = h.getCore();
-
-    ModifiableSolrParams params = new ModifiableSolrParams();
-
-    assertU(adoc("id","42","name","Tom Cruise","subword","Top Gun","subword","Risky Business","subword","The Color of Money","subword","Minority Report","subword", "Days of Thunder","subword", "Eyes Wide Shut","subword", "Far and Away", "foo_ti","10"));
-    assertU(adoc("id","43","name","Tom Hanks","subword","The Green Mile","subword","Forest Gump","subword","Philadelphia Story","subword","Big","subword","Cast Away", "foo_ti","10"));
-    assertU(adoc("id","44","name","Harrison Ford","subword","Star Wars","subword","Indiana Jones","subword","Patriot Games","subword","Regarding Henry"));
-    assertU(adoc("id","45","name","George Harrison","subword","Yellow Submarine","subword","Help","subword","Magical Mystery Tour","subword","Sgt. Peppers Lonley Hearts Club Band"));
-    assertU(adoc("id","46","name","Nicole Kidman","subword","Batman","subword","Days of Thunder","subword","Eyes Wide Shut","subword","Far and Away"));
-    assertU(commit());
-
-    params.set(MoreLikeThisParams.MLT, "true");
-    params.set(MoreLikeThisParams.SIMILARITY_FIELDS, "name,subword");
-    params.set(MoreLikeThisParams.INTERESTING_TERMS, "details");
-    params.set(MoreLikeThisParams.MIN_TERM_FREQ,"1");
-    params.set(MoreLikeThisParams.MIN_DOC_FREQ,"1");
-    params.set("indent","true");
-
-    // requires 'q' or a single content stream
-    SolrException ex = expectThrows(SolrException.class, () -> {
-      try (MoreLikeThisHandler mlt = new MoreLikeThisHandler();
-           SolrQueryRequestBase req = new SolrQueryRequestBase(core, params) {}) {
-        mlt.handleRequestBody(req, new SolrQueryResponse());
+    try (SolrCore core = h.getCore()) {
+
+      ModifiableSolrParams params = new ModifiableSolrParams();
+
+      assertU(adoc("id", "42", "name", "Tom Cruise", "subword", "Top Gun", "subword", "Risky Business", "subword", "The Color of Money", "subword", "Minority Report", "subword", "Days of Thunder",
+          "subword", "Eyes Wide Shut", "subword", "Far and Away", "foo_ti", "10"));
+      assertU(adoc("id", "43", "name", "Tom Hanks", "subword", "The Green Mile", "subword", "Forest Gump", "subword", "Philadelphia Story", "subword", "Big", "subword", "Cast Away", "foo_ti", "10"));
+      assertU(adoc("id", "44", "name", "Harrison Ford", "subword", "Star Wars", "subword", "Indiana Jones", "subword", "Patriot Games", "subword", "Regarding Henry"));
+      assertU(adoc("id", "45", "name", "George Harrison", "subword", "Yellow Submarine", "subword", "Help", "subword", "Magical Mystery Tour", "subword", "Sgt. Peppers Lonley Hearts Club Band"));
+      assertU(adoc("id", "46", "name", "Nicole Kidman", "subword", "Batman", "subword", "Days of Thunder", "subword", "Eyes Wide Shut", "subword", "Far and Away"));
+      assertU(commit());
+
+      params.set(MoreLikeThisParams.MLT, "true");
+      params.set(MoreLikeThisParams.SIMILARITY_FIELDS, "name,subword");
+      params.set(MoreLikeThisParams.INTERESTING_TERMS, "details");
+      params.set(MoreLikeThisParams.MIN_TERM_FREQ, "1");
+      params.set(MoreLikeThisParams.MIN_DOC_FREQ, "1");
+      params.set("indent", "true");
+
+      // requires 'q' or a single content stream
+      SolrException ex = expectThrows(SolrException.class, () -> {
+        try (MoreLikeThisHandler mlt = new MoreLikeThisHandler(); SolrQueryRequestBase req = new SolrQueryRequestBase(core, params) {
+        }) {
+          mlt.handleRequestBody(req, new SolrQueryResponse());
+        }
+      });
+      assertEquals(ex.getMessage(), MoreLikeThisHandler.ERR_MSG_QUERY_OR_TEXT_REQUIRED);
+      assertEquals(ex.code(), SolrException.ErrorCode.BAD_REQUEST.code);
+
+      // requires a single content stream (more than one is not supported).
+      ex = expectThrows(SolrException.class, () -> {
+        try (MoreLikeThisHandler mlt = new MoreLikeThisHandler(); SolrQueryRequestBase req = new SolrQueryRequestBase(core, params) {
+        }) {
+          ArrayList<ContentStream> streams = new ArrayList<>(2);
+          streams.add(new ContentStreamBase.StringStream("hello"));
+          streams.add(new ContentStreamBase.StringStream("there"));
+          req.setContentStreams(streams);
+          mlt.handleRequestBody(req, new SolrQueryResponse());
+        }
+      });
+      assertEquals(ex.getMessage(), MoreLikeThisHandler.ERR_MSG_SINGLE_STREAM_ONLY);
+      assertEquals(ex.code(), SolrException.ErrorCode.BAD_REQUEST.code);
+
+      params.set(CommonParams.Q, "id:42");
+
+      try (SolrQueryRequest mltreq = new LocalSolrQueryRequest(core, params)) {
+        assertQ("morelikethis - tom cruise", mltreq, "//result/doc[1]/str[@name='id'][.='46']", "//result/doc[2]/str[@name='id'][.='43']");
       }
-    });
-    assertEquals(ex.getMessage(), MoreLikeThisHandler.ERR_MSG_QUERY_OR_TEXT_REQUIRED);
-    assertEquals(ex.code(), SolrException.ErrorCode.BAD_REQUEST.code);
-
-    // requires a single content stream (more than one is not supported).
-    ex = expectThrows(SolrException.class, () -> {
-      try (MoreLikeThisHandler mlt = new MoreLikeThisHandler();
-           SolrQueryRequestBase req = new SolrQueryRequestBase(core, params) {}) {
-        ArrayList<ContentStream> streams = new ArrayList<>(2);
-        streams.add(new ContentStreamBase.StringStream("hello"));
-        streams.add(new ContentStreamBase.StringStream("there"));
-        req.setContentStreams(streams);
-        mlt.handleRequestBody(req, new SolrQueryResponse());
-      }
-    });
-    assertEquals(ex.getMessage(), MoreLikeThisHandler.ERR_MSG_SINGLE_STREAM_ONLY);
-    assertEquals(ex.code(), SolrException.ErrorCode.BAD_REQUEST.code);
 
-    params.set(CommonParams.Q, "id:42");
+      params.set(MoreLikeThisParams.BOOST, "true");
 
-    try (SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params)) {
-      assertQ("morelikethis - tom cruise", mltreq,
-          "//result/doc[1]/str[@name='id'][.='46']",
-          "//result/doc[2]/str[@name='id'][.='43']");
-    }
-
-    params.set(MoreLikeThisParams.BOOST, "true");
+      try (SolrQueryRequest mltreq = new LocalSolrQueryRequest(core, params)) {
+        assertQ("morelikethis - tom cruise", mltreq, "//result/doc[1]/str[@name='id'][.='46']", "//result/doc[2]/str[@name='id'][.='43']");
+      }
 
-    try (SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params)) {
-      assertQ("morelikethis - tom cruise", mltreq,
-          "//result/doc[1]/str[@name='id'][.='46']",
-          "//result/doc[2]/str[@name='id'][.='43']");
-    }
-    
-    params.set(CommonParams.Q, "id:44");
-    try (SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params)) {
-      assertQ("morelike this - harrison ford", mltreq,
-          "//result/doc[1]/str[@name='id'][.='45']");
-    }
+      params.set(CommonParams.Q, "id:44");
+      try (SolrQueryRequest mltreq = new LocalSolrQueryRequest(core, params)) {
+        assertQ("morelike this - harrison ford", mltreq, "//result/doc[1]/str[@name='id'][.='45']");
+      }
 
-    // test MoreLikeThis debug
-    params.set(CommonParams.DEBUG_QUERY, "true");
-    try (SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params)) {
-      assertQ("morelike this - harrison ford", mltreq,
-          "//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='rawMLTQuery']",
-          "//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='boostedMLTQuery']",
-          "//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='realMLTQuery']",
-          "//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/lst[@name='explain']/str[@name='45']"
-      );
-    }
+      // test MoreLikeThis debug
+      params.set(CommonParams.DEBUG_QUERY, "true");
+      try (SolrQueryRequest mltreq = new LocalSolrQueryRequest(core, params)) {
+        assertQ("morelike this - harrison ford", mltreq, "//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='rawMLTQuery']",
+            "//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='boostedMLTQuery']", "//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='realMLTQuery']",
+            "//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/lst[@name='explain']/str[@name='45']");
+      }
 
-    // test that qparser plugins work
-    params.remove(CommonParams.DEBUG_QUERY);
-    params.set(CommonParams.Q, "{!field f=id}44");
-    try (SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params)) {
-      assertQ(mltreq, "//result/doc[1]/str[@name='id'][.='45']");
-    }
+      // test that qparser plugins work
+      params.remove(CommonParams.DEBUG_QUERY);
+      params.set(CommonParams.Q, "{!field f=id}44");
+      try (SolrQueryRequest mltreq = new LocalSolrQueryRequest(core, params)) {
+        assertQ(mltreq, "//result/doc[1]/str[@name='id'][.='45']");
+      }
 
-    params.set(CommonParams.Q, "id:42");
-    params.set(MoreLikeThisParams.QF,"name^5.0 subword^0.1");
-    try (SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params)) {
-      assertQ("morelikethis with weights", mltreq,
-          "//result/doc[1]/str[@name='id'][.='43']",
-          "//result/doc[2]/str[@name='id'][.='46']");
-    }
+      params.set(CommonParams.Q, "id:42");
+      params.set(MoreLikeThisParams.QF, "name^5.0 subword^0.1");
+      try (SolrQueryRequest mltreq = new LocalSolrQueryRequest(core, params)) {
+        assertQ("morelikethis with weights", mltreq, "//result/doc[1]/str[@name='id'][.='43']", "//result/doc[2]/str[@name='id'][.='46']");
+      }
 
-    // test that qparser plugins work w/ the MoreLikeThisHandler
-    params.set(CommonParams.QT, "/mlt");
-    params.set(CommonParams.Q, "{!field f=id}44");
-    try (SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params)) {
-      assertQ(mltreq, "//result/doc[1]/str[@name='id'][.='45']");
-    }
+      // test that qparser plugins work w/ the MoreLikeThisHandler
+      params.set(CommonParams.QT, "/mlt");
+      params.set(CommonParams.Q, "{!field f=id}44");
+      try (SolrQueryRequest mltreq = new LocalSolrQueryRequest(core, params)) {
+        assertQ(mltreq, "//result/doc[1]/str[@name='id'][.='45']");
+      }
 
-    // test that debugging works (test for MoreLikeThis*Handler*)
-    params.set(CommonParams.QT, "/mlt");
-    params.set(CommonParams.DEBUG_QUERY, "true");
-    try (SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params)) {
-      assertQ(mltreq,
-          "//result/doc[1]/str[@name='id'][.='45']",
-          "//lst[@name='debug']/lst[@name='explain']"
-      );
+      // test that debugging works (test for MoreLikeThis*Handler*)
+      params.set(CommonParams.QT, "/mlt");
+      params.set(CommonParams.DEBUG_QUERY, "true");
+      try (SolrQueryRequest mltreq = new LocalSolrQueryRequest(core, params)) {
+        assertQ(mltreq, "//result/doc[1]/str[@name='id'][.='45']", "//lst[@name='debug']/lst[@name='explain']");
+      }
     }
   }
 
   @Test
   public void testMultifieldSimilarity() throws Exception
   {
-    SolrCore core = h.getCore();
-    ModifiableSolrParams params = new ModifiableSolrParams();
-
-    assertU(adoc("id", "1", "name", "aaa bbb ccc", "subword", "        zzz"));
-    assertU(adoc("id", "2", "name", "    bbb ccc", "subword", "    bbb zzz"));
-    assertU(adoc("id", "3", "name", "        ccc", "subword", "aaa bbb zzz"));
-    assertU(adoc("id", "4", "name", "        ccc", "subword", "    bbb    "));
-    assertU(commit());
-
-    params.set(CommonParams.QT, "/mlt");
-    params.set(MoreLikeThisParams.MLT, "true");
-    params.set(MoreLikeThisParams.SIMILARITY_FIELDS, "name,subword");
-    params.set(MoreLikeThisParams.INTERESTING_TERMS, "details");
-    params.set(MoreLikeThisParams.MIN_TERM_FREQ, "1");
-    params.set(MoreLikeThisParams.MIN_DOC_FREQ, "2");
-    params.set(MoreLikeThisParams.BOOST, true);
-    params.set("indent", "true");
-
-    try (SolrQueryRequestBase req = new SolrQueryRequestBase(core, params) {}) {
-      ArrayList<ContentStream> streams = new ArrayList<>(2);
-      streams.add(new ContentStreamBase.StringStream("bbb", "zzz"));
-      req.setContentStreams(streams);
-
-      // Make sure we have terms from both fields in the interestingTerms array and all documents have been
-      // retrieved as matching.
-      assertQ(req,
-          "//lst[@name = 'interestingTerms']/float[@name = 'subword:bbb']",
-          "//lst[@name = 'interestingTerms']/float[@name = 'name:bbb']",
-          "//result[@name = 'response' and @numFound = '4']");
+    try (SolrCore core = h.getCore()) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+
+      assertU(adoc("id", "1", "name", "aaa bbb ccc", "subword", "        zzz"));
+      assertU(adoc("id", "2", "name", "    bbb ccc", "subword", "    bbb zzz"));
+      assertU(adoc("id", "3", "name", "        ccc", "subword", "aaa bbb zzz"));
+      assertU(adoc("id", "4", "name", "        ccc", "subword", "    bbb    "));
+      assertU(commit());
+
+      params.set(CommonParams.QT, "/mlt");
+      params.set(MoreLikeThisParams.MLT, "true");
+      params.set(MoreLikeThisParams.SIMILARITY_FIELDS, "name,subword");
+      params.set(MoreLikeThisParams.INTERESTING_TERMS, "details");
+      params.set(MoreLikeThisParams.MIN_TERM_FREQ, "1");
+      params.set(MoreLikeThisParams.MIN_DOC_FREQ, "2");
+      params.set(MoreLikeThisParams.BOOST, true);
+      params.set("indent", "true");
+
+      try (SolrQueryRequestBase req = new SolrQueryRequestBase(core, params) {
+      }) {
+        ArrayList<ContentStream> streams = new ArrayList<>(2);
+        streams.add(new ContentStreamBase.StringStream("bbb", "zzz"));
+        req.setContentStreams(streams);
+
+        // Make sure we have terms from both fields in the interestingTerms array and all documents have been
+        // retrieved as matching.
+        assertQ(req, "//lst[@name = 'interestingTerms']/float[@name = 'subword:bbb']", "//lst[@name = 'interestingTerms']/float[@name = 'name:bbb']", "//result[@name = 'response' and @numFound = '4']");
+      }
     }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java b/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java
index 055c90a..f517e385 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestSystemCollAutoCreate.java
@@ -22,7 +22,6 @@ import org.apache.solr.cloud.SolrCloudBridgeTestCase;
 import org.apache.solr.common.cloud.DocCollection;
 import org.junit.Ignore;
 
-@Ignore // MRM-TEST TODO: finish closing things right
 public class TestSystemCollAutoCreate extends SolrCloudBridgeTestCase {
 
   public TestSystemCollAutoCreate() {
diff --git a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
index e724294..38f095d 100644
--- a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
@@ -54,7 +54,6 @@ public class V2ApiIntegrationTest extends SolrCloudTestCase {
         .configure();
     CollectionAdminRequest.createCollection(COLL_NAME, "conf1", 1, 2)
         .process(cluster.getSolrClient());
-    cluster.waitForActiveCollection(COLL_NAME, 1, 2);
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/handler/component/MoreLikeThisComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/MoreLikeThisComponentTest.java
index 41f2823..b55267e 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/MoreLikeThisComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/MoreLikeThisComponentTest.java
@@ -25,7 +25,6 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 /**
@@ -35,7 +34,6 @@ import org.junit.Test;
  * @see MoreLikeThisComponent
  */
 @Slow
-@Ignore // MRM-TEST TODO: finish closing things right
 public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
 
   @BeforeClass
@@ -66,16 +64,15 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     initCommonMoreLikeThisParams(params);
     
     params.set(CommonParams.Q, "id:42");
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params, true);
     assertQ("morelikethis - tom cruise",mltreq
         ,"//result/doc[1]/str[@name='id'][.='46']"
         ,"//result/doc[2]/str[@name='id'][.='43']");
 
     params.set(CommonParams.Q, "id:44");
-    mltreq.close(); mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ("morelike this - harrison ford",mltreq
         ,"//result/doc[1]/str[@name='id'][.='45']");
-    mltreq.close();
   }
 
   @Test
@@ -88,13 +85,12 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     params.set(MoreLikeThisParams.INTERESTING_TERMS, "details");
     
     params.set(CommonParams.Q, "id:42");
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params, true);
     assertQ("morelikethis - tom cruise",mltreq
         ,"//result/doc[1]/str[@name='id'][.='46']"
         ,"//result/doc[2]/str[@name='id'][.='43']",
         "//lst[@name='interestingTerms']/lst[1][count(*)>0]",
         "//lst[@name='interestingTerms']/lst[1]/float[.=1.0]");
-    mltreq.close();
   }
 
   @Test
@@ -107,13 +103,12 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     params.set(MoreLikeThisParams.INTERESTING_TERMS, "list");
 
     params.set(CommonParams.Q, "id:42");
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params, true);
     assertQ("morelikethis - tom cruise",mltreq
         ,"//result/doc[1]/str[@name='id'][.='46']"
         ,"//result/doc[2]/str[@name='id'][.='43']",
         "//lst[@name='interestingTerms']/arr[@name='42'][count(*)>0]",
         "//lst[@name='interestingTerms']/arr[@name='42']/str[.='name:Cruise']");
-    mltreq.close();
   }
 
   @Test
@@ -126,19 +121,17 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     params.set(MoreLikeThisParams.BOOST, "true");
 
     params.set(CommonParams.Q, "id:42");
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params, true);
     assertQ("morelikethis - tom cruise",mltreq
         ,"//result/doc[1]/str[@name='id'][.='46']"
         ,"//result/doc[2]/str[@name='id'][.='43']");
 
     params.set(CommonParams.Q, "id:42");
     params.set(MoreLikeThisParams.QF,"name^5.0 subword^0.1");
-    mltreq.close(); mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ("morelikethis with weights",mltreq
         ,"//result/doc[1]/str[@name='id'][.='43']"
         ,"//result/doc[2]/str[@name='id'][.='46']");
-
-    mltreq.close();
   }
 
   @Test
@@ -152,7 +145,7 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     params.set(MoreLikeThisParams.INTERESTING_TERMS, "details");
     
     params.set(CommonParams.Q, "id:42");
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params, true);
     assertQ("morelikethis - tom cruise",mltreq
         ,"//result/doc[1]/str[@name='id'][.='46']"
         ,"//result/doc[2]/str[@name='id'][.='43']",
@@ -160,14 +153,12 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
         "//lst[@name='interestingTerms']/lst[1]/float[.>1.0]");
     
     params.set(MoreLikeThisParams.QF,"name^5.0 subword^0.1");
-    mltreq.close(); mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ("morelikethis with weights",mltreq
         ,"//result/doc[1]/str[@name='id'][.='43']"
         ,"//result/doc[2]/str[@name='id'][.='46']",
         "//lst[@name='interestingTerms']/lst[1][count(*)>0]",
         "//lst[@name='interestingTerms']/lst[1]/float[.>5.0]");
-
-    mltreq.close();
   }
 
   @Test
@@ -181,7 +172,7 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     params.set(MoreLikeThisParams.INTERESTING_TERMS, "list");
 
     params.set(CommonParams.Q, "id:42");
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest( core, params, true);
     assertQ("morelikethis - tom cruise",mltreq
         ,"//result/doc[1]/str[@name='id'][.='46']"
         ,"//result/doc[2]/str[@name='id'][.='43']",
@@ -189,14 +180,12 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
         "//lst[@name='interestingTerms']/arr[@name='42']/str[.='name:Cruise']");
 
     params.set(MoreLikeThisParams.QF,"name^5.0 subword^0.1");
-    mltreq.close(); mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ("morelikethis with weights",mltreq
         ,"//result/doc[1]/str[@name='id'][.='43']"
         ,"//result/doc[2]/str[@name='id'][.='46']",
         "//lst[@name='interestingTerms']/arr[@name='42'][count(*)>0]",
         "//lst[@name='interestingTerms']/arr[@name='42']/str[.='name:Cruise']");
-
-    mltreq.close();
   }
 
   @Test
@@ -209,7 +198,7 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
 
     params.set(CommonParams.Q, "id:44");
     params.set(CommonParams.DEBUG_QUERY, "true");
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ("morelike this - harrison ford",mltreq
         ,"//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='rawMLTQuery']"
         ,"//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='boostedMLTQuery']"
@@ -219,10 +208,9 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     
     params.remove(CommonParams.DEBUG_QUERY);
     params.set(CommonParams.Q, "{!field f=id}44");
-    mltreq.close(); mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ(mltreq
         ,"//result/doc[1]/str[@name='id'][.='45']");
-    mltreq.close();
   }
 
   @Test
@@ -236,7 +224,7 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
 
     params.set(CommonParams.Q, "id:44");
     params.set(CommonParams.DEBUG_QUERY, "true");
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ("morelike this - harrison ford",mltreq
         ,"//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='rawMLTQuery']"
         ,"//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='boostedMLTQuery']"
@@ -247,12 +235,11 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
 
     params.remove(CommonParams.DEBUG_QUERY);
     params.set(CommonParams.Q, "{!field f=id}44");
-    mltreq.close(); mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ(mltreq
         ,"//result/doc[1]/str[@name='id'][.='45']",
         "//lst[@name='interestingTerms']/lst[1][count(*)>0]",
         "//lst[@name='interestingTerms']/lst[1]/float[.>1.0]");
-    mltreq.close();
   }
 
   @Test
@@ -267,7 +254,7 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     params.set(CommonParams.Q, "id:44");
     params.set(CommonParams.DEBUG_QUERY, "true");
     
-    SolrQueryRequest mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    SolrQueryRequest mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ("morelike this - harrison ford",mltreq
         ,"//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='rawMLTQuery']"
         ,"//lst[@name='debug']/lst[@name='moreLikeThis']/lst[@name='44']/str[@name='boostedMLTQuery']"
@@ -279,7 +266,7 @@ public class MoreLikeThisComponentTest extends SolrTestCaseJ4 {
     params.remove(CommonParams.DEBUG_QUERY);
     params.set(CommonParams.Q, "{!field f=id}44");
 
-    mltreq = new LocalSolrQueryRequest(h.getCore(), params);
+    mltreq = new LocalSolrQueryRequest(h.getCore(), params, true);
     assertQ(mltreq
         ,"//result/doc[1]/str[@name='id'][.='45']",
         "//lst[@name='interestingTerms']/arr[@name='44'][count(*)>0]",
diff --git a/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
index 3e3032e..83fcb26 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
@@ -56,12 +56,10 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.util.hll.HLL;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 
 /**
  * Statistics Component Test (which also checks some equivalent json.facet functionality)
  */
-@Ignore // MRM-TEST TODO: finish closing things right
 public class StatsComponentTest extends SolrTestCaseJ4 {
 
   final static String XPRE = "/response/lst[@name='stats']/";
@@ -76,8 +74,13 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
   @Override
   public void setUp() throws Exception {
     super.setUp();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
     clearIndex();
     assertU(commit());
+    super.tearDown();
   }
 
   @Nightly
@@ -395,7 +398,6 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
   public void testFieldStatisticsResultsStringField() throws Exception {
     String f = "active_s";
 
-    SolrCore core = h.getCore();
     assertU(adoc("id", "1", f, "string1"));
     assertU(adoc("id", "2", f, "string2"));
     assertU(adoc("id", "3", f, "string3"));
@@ -476,7 +478,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     args.put(StatsParams.STATS_FIELD, "active_dt");
     args.put("f.active_dt.stats.calcdistinct","true");
     args.put("indent", "true");
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
 
     assertQ("test date statistics values", req,
             "//long[@name='count'][.='2']",
@@ -507,7 +509,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     args.put(StatsParams.STATS, "true");
     args.put(StatsParams.STATS_FIELD, "active_dt");
     args.put("indent", "true");
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
 
     assertQ("test date statistics values", req,
             "//long[@name='count'][.='1']",
@@ -522,6 +524,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     assertU(adoc("id", "3", "active_dt", "2215-12-14T09:00:00Z"));
     assertU(commit());
 
+    req = new LocalSolrQueryRequest(h.getCore(), new MapSolrParams(args), true);
     assertQ("test date statistics values", req,
         "//long[@name='count'][.='3']",
         "//date[@name='min'][.='2015-12-14T09:00:00Z']",
@@ -774,7 +777,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     args.put(StatsParams.STATS, "true");
     args.put(StatsParams.STATS_FIELD, "active_i");
     args.put("indent", "true");
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
 
     assertQ("test string statistics values", req
             ,"//lst[@name='active_i']/long[@name='count'][.='0']"
@@ -814,7 +817,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     args.put(StatsParams.STATS, "true");
     args.put(StatsParams.STATS_FIELD, "active_s");
     args.put("indent", "true");
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
 
     assertQ("test string statistics values", req
             ,"//lst[@name='active_s']/long[@name='count'][.='0']"
@@ -848,7 +851,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     args.put(StatsParams.STATS, "true");
     args.put(StatsParams.STATS_FIELD, "active_dt");
     args.put("indent", "true");
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
 
     assertQ("test string statistics values", req
             ,"//lst[@name='active_dt']/long[@name='count'][.='0']"
@@ -875,6 +878,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
   public void testStatsFacetMultivaluedErrorHandling() throws Exception {
     SolrCore core = h.getCore();
     SchemaField foo_ss = core.getLatestSchema().getField("foo_ss");
+    core.close();
 
     assertU(adoc("id", "1", "active_i", "1", "foo_ss", "aa" ));
     assertU(commit());
@@ -884,14 +888,12 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
 
     assertTrue("schema no longer satisfies test requirements: foo_ss no longer multivalued", foo_ss.multiValued());
     assertTrue("schema no longer satisfies test requirements: foo_ss's fieldtype no longer single valued", ! foo_ss.getType().isMultiValued());
-    
-    assertQEx("no failure trying to get stats facet on foo_ss",
-              req("q", "*:*", 
-                  "stats", "true",
-                  "stats.field", "active_i",
-                  "stats.facet", "foo_ss"),
-              400);
 
+    SolrQueryRequest req = req("q", "*:*",
+        "stats", "true",
+        "stats.field", "active_i",
+        "stats.facet", "foo_ss");
+      assertQEx("no failure trying to get stats facet on foo_ss", req, 400);
   }
 
   //SOLR-3177
@@ -908,7 +910,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     args.put(StatsParams.STATS, "true");
     args.put(StatsParams.STATS_FIELD, "{!ex=id}id_i");
     args.put("fq", "{!tag=id}id_i:[2 TO 3]");
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
 
     assertQ("test exluding filter query", req
             , "//lst[@name='id_i']/double[@name='min'][.='1.0']"
@@ -957,7 +959,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     args.put(StatsParams.STATS, "true");
     args.put(StatsParams.STATS_FIELD, "cat_docValues");
     args.put("indent", "true");
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
     
     assertQ("test min/max on docValues and multiValued", req
         , "//lst[@name='cat_docValues']/str[@name='min'][.='test']"
@@ -970,92 +972,85 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
   }
 
   public void testFieldStatisticsDocValuesAndMultiValuedInteger() throws Exception {
-      SolrCore core = h.getCore();
-      String fieldName = "cat_intDocValues";
-      // precondition for the test
-      SchemaField catDocValues = core.getLatestSchema().getField(fieldName);
-      assertTrue("schema no longer satisfies test requirements: cat_docValues no longer multivalued", catDocValues.multiValued());
-      assertTrue("schema no longer satisfies test requirements: cat_docValues fieldtype no longer single valued", !catDocValues.getType().isMultiValued());
-      assertTrue("schema no longer satisfies test requirements: cat_docValues no longer has docValues", catDocValues.hasDocValues());
+    SolrCore core = h.getCore();
+    String fieldName = "cat_intDocValues";
+    // precondition for the test
+    SchemaField catDocValues = core.getLatestSchema().getField(fieldName);
 
-      List<FldType> types = new ArrayList<>();
-      types.add(new FldType("id", ONE_ONE, new SVal('A', 'Z', 4, 4)));
-      types.add(new FldType(fieldName, ONE_ONE, new IRange(0, 0)));
+    assertTrue("schema no longer satisfies test requirements: cat_docValues no longer multivalued", catDocValues.multiValued());
+    assertTrue("schema no longer satisfies test requirements: cat_docValues fieldtype no longer single valued", !catDocValues.getType().isMultiValued());
+    assertTrue("schema no longer satisfies test requirements: cat_docValues no longer has docValues", catDocValues.hasDocValues());
 
-      Doc d1 = createDocValuesDocument(types, fieldName, "1", -1, 3, 5);
-      updateJ(toJSON(d1), null);
+    List<FldType> types = new ArrayList<>();
+    types.add(new FldType("id", ONE_ONE, new SVal('A', 'Z', 4, 4)));
+    types.add(new FldType(fieldName, ONE_ONE, new IRange(0, 0)));
 
-      Doc d2 = createDocValuesDocument(types, fieldName, "2", 3, -2, 6);
-      updateJ(toJSON(d2), null);
+    Doc d1 = createDocValuesDocument(types, fieldName, "1", -1, 3, 5);
+    updateJ(toJSON(d1), null);
 
-      Doc d3 = createDocValuesDocument(types, fieldName, "3", 16, -3, 11);
-      updateJ(toJSON(d3), null);
+    Doc d2 = createDocValuesDocument(types, fieldName, "2", 3, -2, 6);
+    updateJ(toJSON(d2), null);
 
-      assertU(commit());
+    Doc d3 = createDocValuesDocument(types, fieldName, "3", 16, -3, 11);
+    updateJ(toJSON(d3), null);
 
-      Map<String, String> args = new HashMap<>();
-      args.put(CommonParams.Q, "*:*");
-      args.put(StatsParams.STATS, "true");
-      args.put(StatsParams.STATS_FIELD, fieldName);
-      args.put(StatsParams.STATS_CALC_DISTINCT, "true");
-      args.put("indent", "true");
-
-      SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
-
-      assertQ("test min/max on docValues and multiValued", req
-          , "//lst[@name='" + fieldName + "']/double[@name='min'][.='-3.0']"
-          , "//lst[@name='" + fieldName + "']/double[@name='max'][.='16.0']"
-          , "//lst[@name='" + fieldName + "']/long[@name='count'][.='12']"
-          , "//lst[@name='" + fieldName + "']/long[@name='countDistinct'][.='9']"
-          , "//lst[@name='" + fieldName + "']/double[@name='sum'][.='38.0']"
-          , "//lst[@name='" + fieldName + "']/double[@name='mean'][.='3.1666666666666665']"
-          , "//lst[@name='" + fieldName + "']/double[@name='stddev'][.='5.638074031784151']"
-          , "//lst[@name='" + fieldName + "']/double[@name='sumOfSquares'][.='470.0']"
-          , "//lst[@name='" + fieldName + "']/long[@name='missing'][.='0']");
+    assertU(commit());
 
-    assertQ("cardinality", 
-            req("q","*:*", "stats", "true", "stats.field", "{!cardinality=true}" + fieldName)
-            , "//lst[@name='"+fieldName+"']/long[@name='cardinality'][.='9']");
+    Map<String,String> args = new HashMap<>();
+    args.put(CommonParams.Q, "*:*");
+    args.put(StatsParams.STATS, "true");
+    args.put(StatsParams.STATS_FIELD, fieldName);
+    args.put(StatsParams.STATS_CALC_DISTINCT, "true");
+    args.put("indent", "true");
+
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
+
+    assertQ("test min/max on docValues and multiValued", req, "//lst[@name='" + fieldName + "']/double[@name='min'][.='-3.0']", "//lst[@name='" + fieldName + "']/double[@name='max'][.='16.0']",
+        "//lst[@name='" + fieldName + "']/long[@name='count'][.='12']", "//lst[@name='" + fieldName + "']/long[@name='countDistinct'][.='9']",
+        "//lst[@name='" + fieldName + "']/double[@name='sum'][.='38.0']", "//lst[@name='" + fieldName + "']/double[@name='mean'][.='3.1666666666666665']",
+        "//lst[@name='" + fieldName + "']/double[@name='stddev'][.='5.638074031784151']", "//lst[@name='" + fieldName + "']/double[@name='sumOfSquares'][.='470.0']",
+        "//lst[@name='" + fieldName + "']/long[@name='missing'][.='0']");
+
+    assertQ("cardinality", req("q", "*:*", "stats", "true", "stats.field", "{!cardinality=true}" + fieldName), "//lst[@name='" + fieldName + "']/long[@name='cardinality'][.='9']");
 
   }
 
   public void testFieldStatisticsDocValuesAndMultiValuedIntegerFacetStats() throws Exception {
-       SolrCore core = h.getCore();
-       String fieldName = "cat_intDocValues";
-       // precondition for the test
-       SchemaField catDocValues = core.getLatestSchema().getField(fieldName);
-       assertTrue("schema no longer satisfies test requirements: cat_docValues no longer multivalued", catDocValues.multiValued());
-       assertTrue("schema no longer satisfies test requirements: cat_docValues fieldtype no longer single valued", !catDocValues.getType().isMultiValued());
-       assertTrue("schema no longer satisfies test requirements: cat_docValues no longer has docValues", catDocValues.hasDocValues());
-
-       List<FldType> types = new ArrayList<>();
-       types.add(new FldType("id", ONE_ONE, new SVal('A', 'Z', 4, 4)));
-       types.add(new FldType(fieldName, ONE_ONE, new IRange(0, 0)));
+    SolrCore core = h.getCore();
+    String fieldName = "cat_intDocValues";
+    // precondition for the test
+    SchemaField catDocValues = core.getLatestSchema().getField(fieldName);
 
-       Doc d1 = createDocValuesDocument(types, fieldName, "1", -1, 3, 5);
-       updateJ(toJSON(d1), null);
+    assertTrue("schema no longer satisfies test requirements: cat_docValues no longer multivalued", catDocValues.multiValued());
+    assertTrue("schema no longer satisfies test requirements: cat_docValues fieldtype no longer single valued", !catDocValues.getType().isMultiValued());
+    assertTrue("schema no longer satisfies test requirements: cat_docValues no longer has docValues", catDocValues.hasDocValues());
 
-       Doc d2 = createDocValuesDocument(types, fieldName, "2", 3, -2, 6);
-       updateJ(toJSON(d2), null);
+    List<FldType> types = new ArrayList<>();
+    types.add(new FldType("id", ONE_ONE, new SVal('A', 'Z', 4, 4)));
+    types.add(new FldType(fieldName, ONE_ONE, new IRange(0, 0)));
 
-       Doc d3 = createDocValuesDocument(types, fieldName, "3", 16, -3, 11);
-       updateJ(toJSON(d3), null);
+    Doc d1 = createDocValuesDocument(types, fieldName, "1", -1, 3, 5);
+    updateJ(toJSON(d1), null);
 
-       assertU(commit());
+    Doc d2 = createDocValuesDocument(types, fieldName, "2", 3, -2, 6);
+    updateJ(toJSON(d2), null);
 
-       Map<String, String> args = new HashMap<>();
-       args.put(CommonParams.Q, "*:*");
-       args.put(StatsParams.STATS, "true");
-       args.put(StatsParams.STATS_FIELD, fieldName);
-       args.put(StatsParams.STATS_FACET, fieldName);
-       args.put(StatsParams.STATS_CALC_DISTINCT, "true");
-       args.put("indent", "true");
+    Doc d3 = createDocValuesDocument(types, fieldName, "3", 16, -3, 11);
+    updateJ(toJSON(d3), null);
 
-       SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+    assertU(commit());
 
-       assertQEx("can not use FieldCache on multivalued field: cat_intDocValues", req, 400);
+    Map<String,String> args = new HashMap<>();
+    args.put(CommonParams.Q, "*:*");
+    args.put(StatsParams.STATS, "true");
+    args.put(StatsParams.STATS_FIELD, fieldName);
+    args.put(StatsParams.STATS_FACET, fieldName);
+    args.put(StatsParams.STATS_CALC_DISTINCT, "true");
+    args.put("indent", "true");
 
-     }
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args), true);
+    assertQEx("can not use FieldCache on multivalued field: cat_intDocValues", req, 400);
+  }
 
 
   public void testMiscQueryStats() throws Exception {
@@ -1151,6 +1146,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     String fieldName = "cat_floatDocValues";
     // precondition for the test
     SchemaField catDocValues = core.getLatestSchema().getField(fieldName);
+    core.close();
     assertTrue("schema no longer satisfies test requirements: cat_docValues no longer multivalued", catDocValues.multiValued());
     assertTrue("schema no longer satisfies test requirements: cat_docValues fieldtype no longer single valued", !catDocValues.getType().isMultiValued());
     assertTrue("schema no longer satisfies test requirements: cat_docValues no longer has docValues", catDocValues.hasDocValues());
@@ -1341,6 +1337,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
 
     SolrCore core = h.getCore();
     SchemaField field = core.getLatestSchema().getField("a_i");
+    core.close();
     HllOptions hllOpts = HllOptions.parseHllOptions(params("cardinality","true"), field);
     
     HLL hll = hllOpts.newHLL();
@@ -1630,7 +1627,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     // insure we have the same hasher a_l would use
     HashFunction hasher = HllOptions.parseHllOptions
       (params("cardinality","true"), core.getLatestSchema().getField("a_l")).getHasher();
-
+    core.close();
     String[] baseParams = new String[] { "q","*:*", "stats","true", "indent","true", "rows","0" };
     assertQ("empty cardinalities"
             , req(params("stats.field","{!key=a cardinality=true}a_l",
@@ -1701,7 +1698,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     SchemaField field_i = core.getLatestSchema().getField("field_i");
     SchemaField field_f = core.getLatestSchema().getField("field_f");
     SchemaField field_severity = core.getLatestSchema().getField("severity");
-
+    core.close();
     // simple cases that shouldn't use HLL
     assertNull(HllOptions.parseHllOptions(params(), field_l));
     assertNull(HllOptions.parseHllOptions(params("cardinality","false"), field_l));
@@ -1812,7 +1809,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     SolrCore core = h.getCore();
     SchemaField foo_s = core.getLatestSchema().getField("foo_s");
     SchemaField foo_i = core.getLatestSchema().getField("foo_i");
-
+    core.close();
     ignoreException("hllPreHashed");
     for (SchemaField field : new SchemaField[] { foo_s, foo_i }) {
       // whitebox - field
@@ -1825,7 +1822,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
       assertQEx("hllPreHashed " + field.getName(), "hllPreHashed is only supported with Long",
                 req(params("stats.field","{!cardinality=true hllPreHashed=true}" + field.getName()),
                     baseParams),
-                ErrorCode.BAD_REQUEST);
+                ErrorCode.BAD_REQUEST, true);
     }
 
     // whitebox - function
@@ -1839,7 +1836,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
     assertQEx("hllPreHashed function", "hllPreHashed is only supported with Long",
               req(params("stats.field","{!func cardinality=true hllPreHashed=true}sum(foo_i,foo_l)"),
                   baseParams),
-              ErrorCode.BAD_REQUEST);
+              ErrorCode.BAD_REQUEST, true);
 
 
     ignoreException("accuracy");
@@ -1853,7 +1850,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
       assertQEx("cardinality="+invalid, "number between 0 and 1",
                 req(params("stats.field","{!cardinality="+invalid+"}foo_s"),
                     baseParams),
-                ErrorCode.BAD_REQUEST);
+                ErrorCode.BAD_REQUEST, true);
     }
     
     ignoreException("hllLog2m must be");
@@ -1868,7 +1865,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
       assertQEx("hllLog2m="+invalid, "hllLog2m must be",
                 req(params("stats.field","{!cardinality=true hllLog2m="+invalid+"}foo_s"),
                     baseParams),
-                ErrorCode.BAD_REQUEST);
+                ErrorCode.BAD_REQUEST, true);
     }
 
     ignoreException("hllRegwidth must be");
@@ -1885,7 +1882,7 @@ public class StatsComponentTest extends SolrTestCaseJ4 {
       assertQEx("hllRegwidth="+invalid, "hllRegwidth must be",
                 req(params("stats.field","{!cardinality=true hllRegwidth="+invalid+"}foo_s"),
                     baseParams),
-                ErrorCode.BAD_REQUEST);
+                ErrorCode.BAD_REQUEST, true);
     }
   }
 
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 512b24c..78760b6 100644
--- a/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java
@@ -33,8 +33,10 @@ import org.junit.Test;
  * Test a whole slew of things related to PolyFields
  */
 public class PolyFieldTest extends SolrTestCaseJ4 {
+
   @BeforeClass
   public static void beforeClass() throws Exception {
+    System.setProperty(SolrTestCaseJ4.USE_NUMERIC_POINTS_SYSPROP, "false"); // Poly fields not supported as subtypes, PointType:73
     initCore("solrconfig.xml","schema.xml");
   }
 
diff --git a/solr/core/src/test/org/apache/solr/search/ReturnFieldsTest.java b/solr/core/src/test/org/apache/solr/search/ReturnFieldsTest.java
index cfd7150..f9f90bb 100644
--- a/solr/core/src/test/org/apache/solr/search/ReturnFieldsTest.java
+++ b/solr/core/src/test/org/apache/solr/search/ReturnFieldsTest.java
@@ -23,13 +23,13 @@ import org.apache.lucene.document.StringField;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrDocument;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.transform.*;
 import static org.apache.solr.response.DocsStreamer.convertLuceneDocToSolrDoc;
 import org.apache.solr.schema.IndexSchema;
 
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.lang.reflect.Method;
@@ -40,7 +40,6 @@ import java.util.Locale;
 import java.util.List;
 import java.util.Random;
 
-@Ignore // MRM-TEST TODO: finish closing things right
 public class ReturnFieldsTest extends SolrTestCaseJ4 {
 
   // :TODO: datatypes produced by the functions used may change
@@ -113,8 +112,8 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
         +",transformer=null,wantsScore=false,wantsAllFields=true)";
     assertEquals(rf1ToString, rf1.toString());
 
-    final ReturnFields rf2 = new SolrReturnFields(
-        req("fl", SolrReturnFields.SCORE));
+    SolrQueryRequest req = req("fl", SolrReturnFields.SCORE);
+    final ReturnFields rf2 = new SolrReturnFields(req);
     final String rf2ToStringA = "SolrReturnFields=(globs=[]"
         +",fields=["+SolrReturnFields.SCORE+"]"
         +",okFieldNames=[null, "+SolrReturnFields.SCORE+"]"
@@ -125,6 +124,7 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
         +",okFieldNames=["+SolrReturnFields.SCORE+", null]"
         +",reqFieldNames=["+SolrReturnFields.SCORE+"]"
         +",transformer=score,wantsScore=true,wantsAllFields=false)";
+    req.close();
     assertTrue(
         rf2ToStringA.equals(rf2.toString()) ||
         rf2ToStringB.equals(rf2.toString()));
@@ -187,20 +187,25 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
 
   @Test
   public void testWilcards() {
-    ReturnFields rf = new SolrReturnFields( req("fl", "*") );
+    SolrQueryRequest req = req("fl", "*");
+    ReturnFields rf = new SolrReturnFields( req );
     assertFalse( rf.wantsScore() );
     assertTrue( rf.wantsField( "xxx" ) );
     assertTrue( rf.wantsAllFields() );
     assertNull( rf.getTransformer() );
+    req.close();
 
-    rf = new SolrReturnFields( req("fl", " * ") );
+    req = req("fl", " * ");
+    rf = new SolrReturnFields( req );
     assertFalse( rf.wantsScore() );
     assertTrue( rf.wantsField( "xxx" ) );
     assertTrue( rf.wantsAllFields() );
     assertNull( rf.getTransformer() );
+    req.close();
 
     // Check that we want wildcards
-    rf = new SolrReturnFields( req("fl", "id,aaa*,*bbb") );
+    req = req("fl", "id,aaa*,*bbb");
+    rf = new SolrReturnFields( req);
     assertTrue( rf.wantsField( "id" ) );
     assertTrue( rf.wantsField( "aaaxxx" ) );
     assertFalse(rf.wantsField("xxxaaa"));
@@ -208,35 +213,40 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
     assertFalse(rf.wantsField("bbbxxx"));
     assertFalse( rf.wantsField( "aa" ) );
     assertFalse( rf.wantsField( "bb" ) );
+    req.close();
   }
 
   @Test
   public void testManyParameters() {
-    ReturnFields rf = new SolrReturnFields( req("fl", "id name", "fl", "test subject", "fl", "score") );
-    assertTrue( rf.wantsScore() );
-    assertTrue( rf.wantsField( "id" ) );
-    assertTrue( rf.wantsField( "name" ) );
-    assertTrue( rf.wantsField( "test" ) );
-    assertTrue( rf.wantsField( "subject" ) );
-    assertTrue( rf.wantsField( "score" ) );
-    assertFalse( rf.wantsAllFields() );
-    assertFalse( rf.wantsField( "xxx" ) );
-    assertTrue( rf.getTransformer() instanceof ScoreAugmenter);
+    try (SolrQueryRequest req = req("fl", "id name", "fl", "test subject", "fl", "score")) {
+      ReturnFields rf = new SolrReturnFields(req);
+      assertTrue(rf.wantsScore());
+      assertTrue(rf.wantsField("id"));
+      assertTrue(rf.wantsField("name"));
+      assertTrue(rf.wantsField("test"));
+      assertTrue(rf.wantsField("subject"));
+      assertTrue(rf.wantsField("score"));
+      assertFalse(rf.wantsAllFields());
+      assertFalse(rf.wantsField("xxx"));
+      assertTrue(rf.getTransformer() instanceof ScoreAugmenter);
+    }
   }
 
   @Test
   public void testFunctions() {
-    ReturnFields rf = new SolrReturnFields( req("fl", "exists(text),id,sum(1,1)") );
-    assertFalse(rf.wantsScore());
-    assertTrue( rf.wantsField( "id" ) );
-    assertTrue( rf.wantsField( "sum(1,1)" ));
-    assertTrue( rf.wantsField( "exists(text)" ));
-    assertFalse( rf.wantsAllFields() );
-    assertFalse( rf.wantsField( "xxx" ) );
-    assertTrue( rf.getTransformer() instanceof DocTransformers);
-    DocTransformers transformers = (DocTransformers)rf.getTransformer();
-    assertEquals("exists(text)", transformers.getTransformer(0).getName());
-    assertEquals("sum(1,1)", transformers.getTransformer(1).getName());
+    try (SolrQueryRequest req = req("fl", "exists(text),id,sum(1,1)")) {
+      ReturnFields rf = new SolrReturnFields(req);
+      assertFalse(rf.wantsScore());
+      assertTrue(rf.wantsField("id"));
+      assertTrue(rf.wantsField("sum(1,1)"));
+      assertTrue(rf.wantsField("exists(text)"));
+      assertFalse(rf.wantsAllFields());
+      assertFalse(rf.wantsField("xxx"));
+      assertTrue(rf.getTransformer() instanceof DocTransformers);
+      DocTransformers transformers = (DocTransformers) rf.getTransformer();
+      assertEquals("exists(text)", transformers.getTransformer(0).getName());
+      assertEquals("sum(1,1)", transformers.getTransformer(1).getName());
+    }
   }
 
   @Test
@@ -313,84 +323,101 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
 
   @Test
   public void testAliases() {
-    ReturnFields rf = new SolrReturnFields( req("fl", "newId:id newName:name newTest:test newSubject:subject") );
-    assertTrue(rf.wantsField("id"));
-    assertTrue(rf.wantsField("name"));
-    assertTrue(rf.wantsField("test"));
-    assertTrue(rf.wantsField("subject"));
-    assertTrue(rf.wantsField("newId"));
-    assertTrue(rf.wantsField("newName"));
-    assertTrue(rf.wantsField("newTest"));
-    assertTrue(rf.wantsField("newSubject"));
-    assertFalse(rf.wantsField("xxx"));
-    assertFalse(rf.wantsAllFields());
+    try (SolrQueryRequest req = req("fl", "newId:id newName:name newTest:test newSubject:subject")) {
+      ReturnFields rf = new SolrReturnFields(req);
+      assertTrue(rf.wantsField("id"));
+      assertTrue(rf.wantsField("name"));
+      assertTrue(rf.wantsField("test"));
+      assertTrue(rf.wantsField("subject"));
+      assertTrue(rf.wantsField("newId"));
+      assertTrue(rf.wantsField("newName"));
+      assertTrue(rf.wantsField("newTest"));
+      assertTrue(rf.wantsField("newSubject"));
+      assertFalse(rf.wantsField("xxx"));
+      assertFalse(rf.wantsAllFields());
+    }
 
-    rf = new SolrReturnFields( req("fl", "newId:id newName:name newTest:test newSubject:subject score") );
-    assertTrue(rf.wantsField("id"));
-    assertTrue(rf.wantsField("name"));
-    assertTrue(rf.wantsField("test"));
-    assertTrue(rf.wantsField("subject"));
-    assertTrue(rf.wantsField("newId"));
-    assertTrue(rf.wantsField("newName"));
-    assertTrue(rf.wantsField("newTest"));
-    assertTrue(rf.wantsField("newSubject"));
-    assertFalse(rf.wantsField("xxx"));
-    assertFalse(rf.wantsAllFields());
-    assertTrue( rf.getTransformer() instanceof DocTransformers);
-    assertEquals(5, ((DocTransformers)rf.getTransformer()).size());  // 4 rename and score
+    try (SolrQueryRequest req = req("fl", "newId:id newName:name newTest:test newSubject:subject score")) {
+      SolrReturnFields rf = new SolrReturnFields(req);
+      assertTrue(rf.wantsField("id"));
+      assertTrue(rf.wantsField("name"));
+      assertTrue(rf.wantsField("test"));
+      assertTrue(rf.wantsField("subject"));
+      assertTrue(rf.wantsField("newId"));
+      assertTrue(rf.wantsField("newName"));
+      assertTrue(rf.wantsField("newTest"));
+      assertTrue(rf.wantsField("newSubject"));
+      assertFalse(rf.wantsField("xxx"));
+      assertFalse(rf.wantsAllFields());
+      assertTrue(rf.getTransformer() instanceof DocTransformers);
+      assertEquals(5, ((DocTransformers) rf.getTransformer()).size());  // 4 rename and score
+    }
   }
 
   // hyphens in field names are not supported in all contexts, but we wanted
   // the simplest case of fl=foo-bar to work
   @Test
   public void testHyphenInFieldName() {
-    ReturnFields rf = new SolrReturnFields(req("fl", "id-test"));
-    assertFalse(rf.wantsScore());
-    assertTrue(rf.wantsField("id-test"));
-    assertFalse(rf.wantsField("xxx"));
-    assertFalse(rf.wantsAllFields());
+    try (SolrQueryRequest req = req("fl", "id-test")) {
+      ReturnFields rf = new SolrReturnFields(req);
+      assertFalse(rf.wantsScore());
+      assertTrue(rf.wantsField("id-test"));
+      assertFalse(rf.wantsField("xxx"));
+      assertFalse(rf.wantsAllFields());
+    }
   }
 
   @Test
   public void testTrailingDotInFieldName() {
-    ReturnFields rf = new SolrReturnFields(req("fl", "id.test"));
-    assertFalse(rf.wantsScore());
-    assertTrue(rf.wantsField("id.test"));
-    assertFalse(rf.wantsField("xxx"));
-    assertFalse(rf.wantsAllFields());
+    try (SolrQueryRequest req = req("fl", "id.test")) {
+      ReturnFields rf = new SolrReturnFields(req);
+      assertFalse(rf.wantsScore());
+      assertTrue(rf.wantsField("id.test"));
+      assertFalse(rf.wantsField("xxx"));
+      assertFalse(rf.wantsAllFields());
+    }
 
-    rf = new SolrReturnFields(req("fl", "test:id.test"));
-    assertFalse(rf.wantsScore());
-    assertTrue(rf.wantsField("id.test"));
-    assertTrue(rf.wantsField("test"));
-    assertFalse(rf.wantsField("xxx"));
-    assertFalse(rf.wantsAllFields());
+    SolrReturnFields rf;
+    try (SolrQueryRequest req = req("fl", "test:id.test")) {
+      rf = new SolrReturnFields(req);
+      assertFalse(rf.wantsScore());
+      assertTrue(rf.wantsField("id.test"));
+      assertTrue(rf.wantsField("test"));
+      assertFalse(rf.wantsField("xxx"));
+      assertFalse(rf.wantsAllFields());
+    }
 
-    rf = new SolrReturnFields(req("fl", "test.id:id.test"));
-    assertFalse(rf.wantsScore());
-    assertTrue(rf.wantsField("id.test"));
-    assertTrue(rf.wantsField("test.id"));
-    assertFalse(rf.wantsField("xxx"));
-    assertFalse(rf.wantsAllFields());
+    try (SolrQueryRequest req = req("fl", "test.id:id.test")) {
+      rf = new SolrReturnFields(req);
+      assertFalse(rf.wantsScore());
+      assertTrue(rf.wantsField("id.test"));
+      assertTrue(rf.wantsField("test.id"));
+      assertFalse(rf.wantsField("xxx"));
+      assertFalse(rf.wantsAllFields());
+    }
   }
 
   @Test
   public void testTrailingDollarInFieldName() {
-    ReturnFields rf = new SolrReturnFields(req("fl", "id$test"));
-    assertFalse(rf.wantsScore());
-    assertTrue(rf.wantsField("id$test"));
-    assertFalse(rf.wantsField("xxx"));
-    assertFalse(rf.wantsAllFields());
+    try (SolrQueryRequest req = req("fl", "id$test")) {
+      ReturnFields rf = new SolrReturnFields(req);
+      assertFalse(rf.wantsScore());
+      assertTrue(rf.wantsField("id$test"));
+      assertFalse(rf.wantsField("xxx"));
+      assertFalse(rf.wantsAllFields());
+    }
   }
 
   @Test
   public void testFunkyFieldNames() {
-    ReturnFields rf = new SolrReturnFields(req("fl", "#foo_s", "fl", "id"));
-    assertFalse(rf.wantsScore());
-    assertTrue(rf.wantsField("id"));
-    assertTrue(rf.wantsField("#foo_s"));
-    assertFalse(rf.wantsField("xxx"));
-    assertFalse(rf.wantsAllFields());
+    try (SolrQueryRequest req = req("fl", "#foo_s", "fl", "id")) {
+      ReturnFields rf = new SolrReturnFields(req);
+      assertFalse(rf.wantsScore());
+      assertTrue(rf.wantsField("id"));
+      assertTrue(rf.wantsField("#foo_s"));
+      assertFalse(rf.wantsField("xxx"));
+      assertFalse(rf.wantsAllFields());
+    }
 
     assertQ(req("q","id:1", "fl","#foo_s", "fl","id")
             ,"//*[@numFound='1'] "
@@ -409,7 +436,9 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
    * </p>
    */
   public void testWhiteboxSolrDocumentConversion() {
-    final IndexSchema schema = h.getCore().getLatestSchema();
+    SolrCore core = h.getCore();
+    final IndexSchema schema = core.getLatestSchema();
+    core.close();
     SolrDocument docOut = null;
 
     // a "mock" Document with a bunch of fields...
@@ -434,20 +463,25 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
     }
 
     // output should only have a single field
-    docOut = convertLuceneDocToSolrDoc(docIn, schema, new SolrReturnFields(req("fl","id")));
+    SolrQueryRequest req = req("fl", "id");
+    docOut = convertLuceneDocToSolrDoc(docIn, schema, new SolrReturnFields(req));
     assertEquals(docOut.toString(), 1, docOut.size());
     assertEquals(docOut.toString(),
                  Collections.singleton("id"),
                  docOut.getFieldNames());
     assertTrue(docOut.toString(), docOut.get("id") instanceof StringField);
+    req.close();
 
     // output should only have the few specified fields
     // behavior should be ultimately be consistent for all of these ReturnField instances
     // (aliasing, extra requested by transformer, or otherwise)
+    SolrQueryRequest req1 = req("fl", "id,subword,store_rpt,uniq,foo_2_s1");
+    SolrQueryRequest req2 = req("fl", "id,xxx:[geo f=store_rpt],uniq,foo_2_s1,subword");
+    SolrQueryRequest req3 = req("fl", "id,xxx:subword,uniq,yyy:foo_2_s1,[geo f=store_rpt]");
     for (ReturnFields rf : Arrays.asList
-           (new SolrReturnFields(req("fl","id,subword,store_rpt,uniq,foo_2_s1")),
-            new SolrReturnFields(req("fl","id,xxx:[geo f=store_rpt],uniq,foo_2_s1,subword")),
-            new SolrReturnFields(req("fl","id,xxx:subword,uniq,yyy:foo_2_s1,[geo f=store_rpt]")))) {
+           (new SolrReturnFields(req1),
+            new SolrReturnFields(req2),
+            new SolrReturnFields(req3))) {
       docOut = convertLuceneDocToSolrDoc(docIn, schema, rf);
       final String debug = rf.toString() + " => " +docOut.toString();
       assertEquals(debug, 5, docOut.size());
@@ -460,13 +494,15 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
       assertTrue(debug, docOut.get("subword") instanceof List);
       assertTrue(debug, docOut.get("uniq") instanceof List);
     }
-    
+    req1.close();
+    req2.close();
+    req3.close();
     // all source fields should be in the output
     // behavior should be ultimately be consistent for all of these ReturnField instances
     // (globbing or requesting more fields then doc has)
-    SolrQueryRequest req1 = req();
-    SolrQueryRequest req2 = req("fl","*");
-    SolrQueryRequest req3 = req("fl","*,score");
+    req1 = req();
+    req2 = req("fl","*");
+    req3 = req("fl","*,score");
     SolrQueryRequest req4 = req("fl","id,subword,uniq,foo_*,store_*");
     SolrQueryRequest req5 = req("fl",allFieldNames+",bogus1,bogus2,bogus3");
     for (ReturnFields rf : Arrays.asList
@@ -515,7 +551,8 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
         randomWhitespace(r, 0, 3);
 
       final String fl = id + (r.nextBoolean() ? "" : ",") + foo_i;
-      ReturnFields rf = new SolrReturnFields(req("fl", fl));
+      SolrQueryRequest req = req("fl", fl);
+      ReturnFields rf = new SolrReturnFields(req);
 
       assertFalse("score ("+fl+")", rf.wantsScore());
 
@@ -527,6 +564,7 @@ public class ReturnFieldsTest extends SolrTestCaseJ4 {
 
       assertFalse(rf.wantsField("xxx"));
       assertFalse(rf.wantsAllFields());
+      req.close();
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java b/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java
index 80d1c11..9523dde 100644
--- a/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java
+++ b/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java
@@ -39,12 +39,10 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 @Slow
 @SuppressTempFileChecks(bugUrl = "https://issues.apache.org/jira/browse/SOLR-1877 Spellcheck IndexReader leak bug?")
-@Ignore // MRM-TEST TODO: finish closing things right
 public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
   
   // if adding documents to this test, adjust me.
@@ -120,7 +118,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
       SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
       SolrQueryResponse rsp = new SolrQueryResponse();
       rsp.addResponseHeader(new SimpleOrderedMap());
-      SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+      SolrQueryRequest req = new LocalSolrQueryRequest(core, params, true);
       handler.handleRequest(req, rsp);
       req.close();
       NamedList values = rsp.getValues();
@@ -152,7 +150,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
       SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
       SolrQueryResponse rsp = new SolrQueryResponse();
       rsp.addResponseHeader(new SimpleOrderedMap());
-      SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+      SolrQueryRequest req = new LocalSolrQueryRequest(core, params, true);
       handler.handleRequest(req, rsp);
       req.close();
       NamedList values = rsp.getValues();
@@ -172,7 +170,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
       SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
       SolrQueryResponse rsp = new SolrQueryResponse();
       rsp.add("responseHeader", new SimpleOrderedMap());
-      SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+      SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params, true);
       handler.handleRequest(req, rsp);
       req.close();
       NamedList values = rsp.getValues();
@@ -246,7 +244,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
     SolrQueryResponse rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, params, true);
     handler.handleRequest(req, rsp);
     req.close();
     NamedList values = rsp.getValues();
@@ -281,7 +279,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
     SolrQueryResponse rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, params, true);
     handler.handleRequest(req, rsp);
     req.close();
     NamedList values = rsp.getValues();
@@ -328,7 +326,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
     SolrQueryResponse rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, params, true);
     handler.handleRequest(req, rsp);
     req.close();
     NamedList values = rsp.getValues();
@@ -345,7 +343,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     handler = core.getRequestHandler("/spellCheckCompRH");
     rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    req = new LocalSolrQueryRequest(core, params);
+    req = new LocalSolrQueryRequest(h.getCore(), params, true);
     handler.handleRequest(req, rsp);
     req.close();
     values = rsp.getValues();
@@ -363,7 +361,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     handler = core.getRequestHandler("/spellCheckCompRH");
     rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    req = new LocalSolrQueryRequest(core, params);
+    req = new LocalSolrQueryRequest(h.getCore(), params, true);
     handler.handleRequest(req, rsp);
     req.close();
     values = rsp.getValues();
@@ -382,7 +380,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     handler = core.getRequestHandler("/spellCheckCompRH");
     rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    req = new LocalSolrQueryRequest(core, params);
+    req = new LocalSolrQueryRequest(h.getCore(), params, true);
     handler.handleRequest(req, rsp);
     req.close();
     values = rsp.getValues();
@@ -435,7 +433,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
     SolrQueryResponse rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, params, true);
     handler.handleRequest(req, rsp);
     req.close();
     NamedList values = rsp.getValues();
@@ -610,7 +608,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
     SolrQueryResponse rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, params, true);
     handler.handleRequest(req, rsp);
     req.close();
     NamedList values = rsp.getValues();
@@ -640,7 +638,7 @@ public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
     SolrRequestHandler handler = core.getRequestHandler("/spellCheckCompRH");
     SolrQueryResponse rsp = new SolrQueryResponse();
     rsp.addResponseHeader(new SimpleOrderedMap());
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, params, true);
     handler.handleRequest(req, rsp);
     req.close();
     NamedList values = rsp.getValues();
diff --git a/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java b/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java
index 97566a6..ab489d5 100644
--- a/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java
+++ b/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java
@@ -17,9 +17,7 @@
 package org.apache.solr.spelling.suggest;
 
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 
-@Ignore // MRM-TEST TODO: finish closing things right
 public class SuggesterFSTTest extends SuggesterTest {
   @BeforeClass
   public static void beforeSuggesterFSTTest() throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java b/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java
index 58cb288..5ab9640 100644
--- a/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java
+++ b/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java
@@ -17,9 +17,7 @@
 package org.apache.solr.spelling.suggest;
 
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 
-@Ignore // MRM-TEST TODO: finish closing things right
 public class SuggesterTSTTest extends SuggesterTest {
   @BeforeClass
   public static void beforeSuggesterTSTTest() throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java b/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java
index 47543b1..ddf44b1 100644
--- a/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java
+++ b/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java
@@ -19,12 +19,12 @@ package org.apache.solr.spelling.suggest;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.SpellingParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
-@Ignore // MRM-TEST TODO: finish closing things right
 public class SuggesterTest extends SolrTestCaseJ4 {
   /**
    * Expected URI at which the given suggester will live.
@@ -102,7 +102,9 @@ public class SuggesterTest extends SolrTestCaseJ4 {
     NamedList params = new NamedList();
     params.add("field", "test_field");
     params.add("lookupImpl", "org.apache.solr.spelling.suggest.tst.TSTLookupFactory");
-    suggester.init(params, h.getCore());
+    SolrCore core = h.getCore();
+    suggester.init(params, core);
+    core.close();
     assertTrue(suggester.getQueryAnalyzer() != null);
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterWFSTTest.java b/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterWFSTTest.java
index b0ca691..fdb6acc 100644
--- a/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterWFSTTest.java
+++ b/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterWFSTTest.java
@@ -17,9 +17,7 @@
 package org.apache.solr.spelling.suggest;
 
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 
-@Ignore // MRM-TEST TODO: finish closing things right
 public class SuggesterWFSTTest extends SuggesterTest {
 
   @BeforeClass
diff --git a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
index 056a6be..8e05089 100644
--- a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
@@ -41,7 +41,6 @@ import org.apache.solr.search.SolrIndexSearcher;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,7 +51,6 @@ import static org.apache.solr.common.params.CommonParams.VERSION_FIELD;
  * 
  *
  */
-@Ignore // MRM-TEST TODO: finish closing things right
 public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -104,117 +102,113 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testBasics() throws Exception {
-
-    // get initial metrics
-    Map<String, Metric> metrics = h.getCoreContainer().getMetricManager()
-        .registry(h.getCore().getCoreMetricManager().getRegistryName()).getMetrics();
-
-    String PREFIX = "UPDATE.updateHandler.";
-
-    String commitsName = PREFIX + "commits";
-    assertTrue(metrics.containsKey(commitsName));
-    String addsName = PREFIX + "adds";
-    assertTrue(metrics.containsKey(addsName));
-    String cumulativeAddsName = PREFIX + "cumulativeAdds";
-    String delsIName = PREFIX + "deletesById";
-    String cumulativeDelsIName = PREFIX + "cumulativeDeletesById";
-    String delsQName = PREFIX + "deletesByQuery";
-    String cumulativeDelsQName = PREFIX + "cumulativeDeletesByQuery";
-    long commits = ((Meter) metrics.get(commitsName)).getCount();
-    long adds = ((Gauge<Number>) metrics.get(addsName)).getValue().longValue();
-    long cumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount();
-    long cumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount();
-    long cumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount();
-
-
-    assertNull("This test requires a schema that has no version field, " +
-               "it appears the schema file in use has been edited to violate " +
-               "this requirement",
-               h.getCore().getLatestSchema().getFieldOrNull(VERSION_FIELD));
-
-    assertU(adoc("id","5"));
-    assertU(adoc("id","6"));
-
-    // search - not committed - docs should not be found.
-    assertQ(req("q","id:5"), "//*[@numFound='0']");
-    assertQ(req("q","id:6"), "//*[@numFound='0']");
-
-    long newAdds = ((Gauge<Number>) metrics.get(addsName)).getValue().longValue();
-    long newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount();
-    assertEquals("new adds", 2, newAdds - adds);
-    assertEquals("new cumulative adds", 2, newCumulativeAdds - cumulativeAdds);
-
-    assertU(commit());
-
-    long newCommits = ((Meter) metrics.get(commitsName)).getCount();
-    assertEquals("new commits", 1, newCommits - commits);
-
-    newAdds = ((Gauge<Number>) metrics.get(addsName)).getValue().longValue();
-    newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount();
-    // adds should be reset to 0 after commit
-    assertEquals("new adds after commit", 0, newAdds);
-    // not so with cumulative ones!
-    assertEquals("new cumulative adds after commit", 2, newCumulativeAdds - cumulativeAdds);
-
-    // now they should be there
-    assertQ(req("q","id:5"), "//*[@numFound='1']");
-    assertQ(req("q","id:6"), "//*[@numFound='1']");
-
-    // now delete one
-    assertU(delI("5"));
-
-    long newDelsI = ((Gauge<Number>) metrics.get(delsIName)).getValue().longValue();
-    long newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount();
-    assertEquals("new delsI", 1, newDelsI);
-    assertEquals("new cumulative delsI", 1, newCumulativeDelsI - cumulativeDelsI);
-
-    // not committed yet
-    assertQ(req("q","id:5"), "//*[@numFound='1']");
-
-    assertU(commit());
-    // delsI should be reset to 0 after commit
-    newDelsI = ((Gauge<Number>) metrics.get(delsIName)).getValue().longValue();
-    newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount();
-    assertEquals("new delsI after commit", 0, newDelsI);
-    assertEquals("new cumulative delsI after commit", 1, newCumulativeDelsI - cumulativeDelsI);
-
-    // 5 should be gone
-    assertQ(req("q","id:5"), "//*[@numFound='0']");
-    assertQ(req("q","id:6"), "//*[@numFound='1']");
-
-    // now delete all
-    assertU(delQ("*:*"));
-
-    long newDelsQ = ((Gauge<Number>) metrics.get(delsQName)).getValue().longValue();
-    long newCumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount();
-    assertEquals("new delsQ", 1, newDelsQ);
-    assertEquals("new cumulative delsQ", 1, newCumulativeDelsQ - cumulativeDelsQ);
-
-    // not committed yet
-    assertQ(req("q","id:6"), "//*[@numFound='1']");
-
-    assertU(commit());
-
-    newDelsQ = ((Gauge<Number>) metrics.get(delsQName)).getValue().longValue();
-    newCumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount();
-    assertEquals("new delsQ after commit", 0, newDelsQ);
-    assertEquals("new cumulative delsQ after commit", 1, newCumulativeDelsQ - cumulativeDelsQ);
-
-    // 6 should be gone
-    assertQ(req("q","id:6"), "//*[@numFound='0']");
-
-    // verify final metrics
-    newCommits = ((Meter) metrics.get(commitsName)).getCount();
-    assertEquals("new commits", 3, newCommits - commits);
-    newAdds = ((Gauge<Number>) metrics.get(addsName)).getValue().longValue();
-    assertEquals("new adds", 0, newAdds);
-    newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount();
-    assertEquals("new cumulative adds", 2, newCumulativeAdds - cumulativeAdds);
-    newDelsI = ((Gauge<Number>) metrics.get(delsIName)).getValue().longValue();
-    assertEquals("new delsI", 0, newDelsI);
-    newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount();
-    assertEquals("new cumulative delsI", 1, newCumulativeDelsI - cumulativeDelsI);
-
+    try (SolrCore core = h.getCore()) {
+      // get initial metrics
+      Map<String,Metric> metrics = h.getCoreContainer().getMetricManager().registry(core.getCoreMetricManager().getRegistryName()).getMetrics();
+
+      String PREFIX = "UPDATE.updateHandler.";
+
+      String commitsName = PREFIX + "commits";
+      assertTrue(metrics.containsKey(commitsName));
+      String addsName = PREFIX + "adds";
+      assertTrue(metrics.containsKey(addsName));
+      String cumulativeAddsName = PREFIX + "cumulativeAdds";
+      String delsIName = PREFIX + "deletesById";
+      String cumulativeDelsIName = PREFIX + "cumulativeDeletesById";
+      String delsQName = PREFIX + "deletesByQuery";
+      String cumulativeDelsQName = PREFIX + "cumulativeDeletesByQuery";
+      long commits = ((Meter) metrics.get(commitsName)).getCount();
+      long adds = ((Gauge<Number>) metrics.get(addsName)).getValue().longValue();
+      long cumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount();
+      long cumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount();
+      long cumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount();
+
+      assertNull("This test requires a schema that has no version field, " + "it appears the schema file in use has been edited to violate " + "this requirement",
+          core.getLatestSchema().getFieldOrNull(VERSION_FIELD));
+
+      assertU(adoc("id", "5"));
+      assertU(adoc("id", "6"));
+
+      // search - not committed - docs should not be found.
+      assertQ(req("q", "id:5"), "//*[@numFound='0']");
+      assertQ(req("q", "id:6"), "//*[@numFound='0']");
+
+      long newAdds = ((Gauge<Number>) metrics.get(addsName)).getValue().longValue();
+      long newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount();
+      assertEquals("new adds", 2, newAdds - adds);
+      assertEquals("new cumulative adds", 2, newCumulativeAdds - cumulativeAdds);
+
+      assertU(commit());
+
+      long newCommits = ((Meter) metrics.get(commitsName)).getCount();
+      assertEquals("new commits", 1, newCommits - commits);
+
+      newAdds = ((Gauge<Number>) metrics.get(addsName)).getValue().longValue();
+      newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount();
+      // adds should be reset to 0 after commit
+      assertEquals("new adds after commit", 0, newAdds);
+      // not so with cumulative ones!
+      assertEquals("new cumulative adds after commit", 2, newCumulativeAdds - cumulativeAdds);
+
+      // now they should be there
+      assertQ(req("q", "id:5"), "//*[@numFound='1']");
+      assertQ(req("q", "id:6"), "//*[@numFound='1']");
+
+      // now delete one
+      assertU(delI("5"));
+
+      long newDelsI = ((Gauge<Number>) metrics.get(delsIName)).getValue().longValue();
+      long newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount();
+      assertEquals("new delsI", 1, newDelsI);
+      assertEquals("new cumulative delsI", 1, newCumulativeDelsI - cumulativeDelsI);
+
+      // not committed yet
+      assertQ(req("q", "id:5"), "//*[@numFound='1']");
+
+      assertU(commit());
+      // delsI should be reset to 0 after commit
+      newDelsI = ((Gauge<Number>) metrics.get(delsIName)).getValue().longValue();
+      newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount();
+      assertEquals("new delsI after commit", 0, newDelsI);
+      assertEquals("new cumulative delsI after commit", 1, newCumulativeDelsI - cumulativeDelsI);
+
+      // 5 should be gone
+      assertQ(req("q", "id:5"), "//*[@numFound='0']");
+      assertQ(req("q", "id:6"), "//*[@numFound='1']");
+
+      // now delete all
+      assertU(delQ("*:*"));
+
+      long newDelsQ = ((Gauge<Number>) metrics.get(delsQName)).getValue().longValue();
+      long newCumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount();
+      assertEquals("new delsQ", 1, newDelsQ);
+      assertEquals("new cumulative delsQ", 1, newCumulativeDelsQ - cumulativeDelsQ);
+
+      // not committed yet
+      assertQ(req("q", "id:6"), "//*[@numFound='1']");
+
+      assertU(commit());
+
+      newDelsQ = ((Gauge<Number>) metrics.get(delsQName)).getValue().longValue();
+      newCumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount();
+      assertEquals("new delsQ after commit", 0, newDelsQ);
+      assertEquals("new cumulative delsQ after commit", 1, newCumulativeDelsQ - cumulativeDelsQ);
+
+      // 6 should be gone
+      assertQ(req("q", "id:6"), "//*[@numFound='0']");
+
+      // verify final metrics
+      newCommits = ((Meter) metrics.get(commitsName)).getCount();
+      assertEquals("new commits", 3, newCommits - commits);
+      newAdds = ((Gauge<Number>) metrics.get(addsName)).getValue().longValue();
+      assertEquals("new adds", 0, newAdds);
+      newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount();
+      assertEquals("new cumulative adds", 2, newCumulativeAdds - cumulativeAdds);
+      newDelsI = ((Gauge<Number>) metrics.get(delsIName)).getValue().longValue();
+      assertEquals("new delsI", 0, newDelsI);
+      newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount();
+      assertEquals("new cumulative delsI", 1, newCumulativeDelsI - cumulativeDelsI);
+    }
   }
 
 
@@ -229,6 +223,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
     // commit "A"
     SolrCore core = h.getCore();
     UpdateHandler updater = core.getUpdateHandler();
+    core.close();
     assertTrue( updater instanceof DirectUpdateHandler2 );
     DirectUpdateHandler2 duh2 = (DirectUpdateHandler2)updater;
     SolrQueryRequest ureq = req();
@@ -291,6 +286,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
     UpdateHandler updater = core.getUpdateHandler();
     assertTrue( updater instanceof DirectUpdateHandler2 );
     DirectUpdateHandler2 duh2 = (DirectUpdateHandler2)updater;
+    core.close();
     SolrQueryRequest ureq = req();
     CommitUpdateCommand cmtCmd = new CommitUpdateCommand(ureq, false);
     cmtCmd.waitSearcher = true;
@@ -441,6 +437,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
     MySolrEventListener listener = new MySolrEventListener();
     core.registerNewSearcherListener(listener);
     updater.registerSoftCommitCallback(listener);
+    core.close();
     assertU(adoc("id", "999"));
     assertU(commit("softCommit", "true"));
     assertEquals("newSearcher was called more than once", 1, listener.newSearcherCount.get());
diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
index 5ed7396..cb3e400 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
@@ -24,16 +24,15 @@ import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
 /**
  * Test proper registration and collection of index and directory metrics.
  */
-@Ignore // MRM-TEST TODO: finish closing things right
 public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
 
   @Before
@@ -52,19 +51,20 @@ public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
   }
 
   private void addDocs() throws Exception {
-    SolrQueryRequest req = lrf.makeRequest();
-    UpdateHandler uh = req.getCore().getUpdateHandler();
-    AddUpdateCommand add = new AddUpdateCommand(req);
-    for (int i = 0; i < (TEST_NIGHTLY ? 1000 : 100); i++) {
-      add.clear();
-      add.solrDoc = new SolrInputDocument();
-      add.solrDoc.addField("id", "" + i);
-      add.solrDoc.addField("foo_s", "foo-" + i);
-      uh.addDoc(add);
+    try (SolrQueryRequest req = lrf.makeRequest()) {
+      UpdateHandler uh = req.getCore().getUpdateHandler();
+      AddUpdateCommand add = new AddUpdateCommand(req);
+      for (int i = 0; i < (TEST_NIGHTLY ? 1000 : 100); i++) {
+        add.clear();
+        add.solrDoc = new SolrInputDocument();
+        add.solrDoc.addField("id", "" + i);
+        add.solrDoc.addField("foo_s", "foo-" + i);
+        uh.addDoc(add);
+      }
+      uh.commit(new CommitUpdateCommand(req, false));
+      // make sure all merges are finished
+      h.reload();
     }
-    uh.commit(new CommitUpdateCommand(req, false));
-    // make sure all merges are finished
-    h.reload();
   }
 
   @Test
@@ -75,7 +75,9 @@ public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
 
     addDocs();
 
-    MetricRegistry registry = h.getCoreContainer().getMetricManager().registry(h.getCore().getCoreMetricManager().getRegistryName());
+    SolrCore core = h.getCore();
+    MetricRegistry registry = h.getCoreContainer().getMetricManager().registry(core.getCoreMetricManager().getRegistryName());
+    core.close();
     assertNotNull(registry);
 
     Map<String, Metric> metrics = registry.getMetrics();
@@ -101,7 +103,9 @@ public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
 
     addDocs();
 
-    MetricRegistry registry = h.getCoreContainer().getMetricManager().registry(h.getCore().getCoreMetricManager().getRegistryName());
+    SolrCore core = h.getCore();
+    MetricRegistry registry = h.getCoreContainer().getMetricManager().registry(core.getCoreMetricManager().getRegistryName());
+    core.close();
     assertNotNull(registry);
 
     Map<String, Metric> metrics = registry.getMetrics();
@@ -116,8 +120,9 @@ public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
     initCore("solrconfig-indexmetrics.xml", "schema.xml");
 
     addDocs();
-
-    MetricRegistry registry = h.getCoreContainer().getMetricManager().registry(h.getCore().getCoreMetricManager().getRegistryName());
+    SolrCore core = h.getCore();
+    MetricRegistry registry = h.getCoreContainer().getMetricManager().registry(core.getCoreMetricManager().getRegistryName());
+    core.close();
     assertNotNull(registry);
 
     Map<String, Metric> metrics = registry.getMetrics();
diff --git a/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java
index 5a156e8..60e80e4 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java
@@ -34,7 +34,6 @@ import org.junit.Ignore;
  *
  * TODO: This test, to run from an IDE, requires a working directory of &lt;path-to&gt;/solr/core/src/test-files.  Fix!
  */
-@Ignore // MRM-TEST TODO: finish closing things right
 public class StatelessScriptUpdateProcessorFactoryTest extends UpdateProcessorTestBase {
 
   @BeforeClass
@@ -70,6 +69,7 @@ public class StatelessScriptUpdateProcessorFactoryTest extends UpdateProcessorTe
   public void testSingleScript() throws Exception {
     SolrCore core = h.getCore();
     UpdateRequestProcessorChain chained = core.getUpdateProcessingChain("single-script");
+    core.close();
     final StatelessScriptUpdateProcessorFactory factory = ((StatelessScriptUpdateProcessorFactory) chained.getProcessors().get(0));
     final List<String> functionMessages = new ArrayList<>();
     factory.setScriptEngineCustomizer(new ScriptEngineCustomizer() {
@@ -108,76 +108,59 @@ public class StatelessScriptUpdateProcessorFactoryTest extends UpdateProcessorTe
   }
 
   public void testMultipleScripts() throws Exception {
-    SolrCore core = h.getCore();
+    try (SolrCore core = h.getCore()) {
 
-    for (final String chain : new String[] {"dual-scripts-arr", 
-                                            "dual-scripts-strs"}) {
-    
-      UpdateRequestProcessorChain chained = core.getUpdateProcessingChain(chain);
-      final StatelessScriptUpdateProcessorFactory factory = 
-        ((StatelessScriptUpdateProcessorFactory) chained.getProcessors().get(0));
-      final List<String> functionMessages = new ArrayList<>();
-      ScriptEngineCustomizer customizer = new ScriptEngineCustomizer() {
+      for (final String chain : new String[] {"dual-scripts-arr", "dual-scripts-strs"}) {
+
+        UpdateRequestProcessorChain chained = core.getUpdateProcessingChain(chain);
+        final StatelessScriptUpdateProcessorFactory factory = ((StatelessScriptUpdateProcessorFactory) chained.getProcessors().get(0));
+        final List<String> functionMessages = new ArrayList<>();
+        ScriptEngineCustomizer customizer = new ScriptEngineCustomizer() {
           @Override
           public void customize(ScriptEngine engine) {
             engine.put("functionMessages", functionMessages);
           }
         };
-      factory.setScriptEngineCustomizer(customizer);
-      assertNotNull(chained);
+        factory.setScriptEngineCustomizer(customizer);
 
-      SolrInputDocument d = processAdd(chain,
-                                       doc(f("id", "2"),
-                                           f("name", " foo "),
-                                           f("subject", "bar")));
-      
-      assertEquals(chain + " didn't add Double field", 
-                   42.3d, d.getFieldValue("script_added_d"));
-      assertEquals(chain + " didn't add integer field",
-          42, d.getFieldValue("script_added_i"));
-      
-      processCommit("run-no-scripts");
+        assertNotNull(chained);
 
-      assertQ(chain + ": couldn't find doc by id",
-              req("q","id:2")
-              , "//result[@numFound=1]");
+        SolrInputDocument d = processAdd(chain, doc(f("id", "2"), f("name", " foo "), f("subject", "bar")));
 
-      processDeleteById(chain, "2");
-      processCommit(chain);
-      
-      assertEquals(chain, 6, functionMessages.size());
-      assertTrue(chain, functionMessages.contains("processAdd0"));
-      assertTrue(chain, functionMessages.contains("processAdd1"));
-      assertTrue(chain + ": script order doesn't match conf order",
-                 functionMessages.indexOf("processAdd0") 
-                 < functionMessages.indexOf("processAdd1"));
-
-      assertTrue(chain, functionMessages.contains("processDelete0"));
-      assertTrue(chain, functionMessages.contains("processDelete1"));
-      assertTrue(chain + ": script order doesn't match conf order",
-                 functionMessages.indexOf("processDelete0") 
-                 < functionMessages.indexOf("processDelete1"));
-
-      assertTrue(chain, functionMessages.contains("processCommit0"));
-      assertTrue(chain, functionMessages.contains("processCommit1"));
-      assertTrue(chain + ": script order doesn't match conf order",
-                 functionMessages.indexOf("processCommit0") 
-                 < functionMessages.indexOf("processCommit1"));
-
-      finish(chain);
-    
-      assertEquals(chain, 8, functionMessages.size());
+        assertEquals(chain + " didn't add Double field", 42.3d, d.getFieldValue("script_added_d"));
+        assertEquals(chain + " didn't add integer field", 42, d.getFieldValue("script_added_i"));
 
-      assertTrue(chain, functionMessages.contains("finish0"));
-      assertTrue(chain, functionMessages.contains("finish1"));
-      assertTrue(chain + ": script order doesn't match conf order",
-                 functionMessages.indexOf("finish0") 
-                 < functionMessages.indexOf("finish1"));
+        processCommit("run-no-scripts");
 
-      assertQ(chain + ": found deleted doc",
-              req("q","id:2")
-              , "//result[@numFound=0]");
-      
+        assertQ(chain + ": couldn't find doc by id", req("q", "id:2"), "//result[@numFound=1]");
+
+        processDeleteById(chain, "2");
+        processCommit(chain);
+
+        assertEquals(chain, 6, functionMessages.size());
+        assertTrue(chain, functionMessages.contains("processAdd0"));
+        assertTrue(chain, functionMessages.contains("processAdd1"));
+        assertTrue(chain + ": script order doesn't match conf order", functionMessages.indexOf("processAdd0") < functionMessages.indexOf("processAdd1"));
+
+        assertTrue(chain, functionMessages.contains("processDelete0"));
+        assertTrue(chain, functionMessages.contains("processDelete1"));
+        assertTrue(chain + ": script order doesn't match conf order", functionMessages.indexOf("processDelete0") < functionMessages.indexOf("processDelete1"));
+
+        assertTrue(chain, functionMessages.contains("processCommit0"));
+        assertTrue(chain, functionMessages.contains("processCommit1"));
+        assertTrue(chain + ": script order doesn't match conf order", functionMessages.indexOf("processCommit0") < functionMessages.indexOf("processCommit1"));
+
+        finish(chain);
+
+        assertEquals(chain, 8, functionMessages.size());
+
+        assertTrue(chain, functionMessages.contains("finish0"));
+        assertTrue(chain, functionMessages.contains("finish1"));
+        assertTrue(chain + ": script order doesn't match conf order", functionMessages.indexOf("finish0") < functionMessages.indexOf("finish1"));
+
+        assertQ(chain + ": found deleted doc", req("q", "id:2"), "//result[@numFound=0]");
+
+      }
     }
   }
 
@@ -251,6 +234,7 @@ public class StatelessScriptUpdateProcessorFactoryTest extends UpdateProcessorTe
         0 < e.getMessage().indexOf("processAdd"));
   }
 
+  @Ignore // MRM-TEST TODO: investigate
   public void testJavaScriptCompatibility() throws Exception  {
     final String chain = "javascript-compatibility";
     SolrInputDocument d = processAdd(chain,
diff --git a/solr/core/src/test/org/apache/solr/util/AuthToolTest.java b/solr/core/src/test/org/apache/solr/util/AuthToolTest.java
index 557c9c5..b35dc12 100644
--- a/solr/core/src/test/org/apache/solr/util/AuthToolTest.java
+++ b/solr/core/src/test/org/apache/solr/util/AuthToolTest.java
@@ -21,12 +21,10 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 
 import org.apache.commons.cli.CommandLine;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import static org.apache.solr.util.SolrCLI.findTool;
@@ -35,7 +33,6 @@ import static org.apache.solr.util.SolrCLI.parseCmdLine;
 /**
  * Unit test for SolrCLI's AuthTool
  */
-@Ignore // MRM-TEST TODO:
 public class AuthToolTest extends SolrCloudTestCase {
   private Path dir;
 
diff --git a/solr/server/build.gradle b/solr/server/build.gradle
index 1960f2e..a99be09 100644
--- a/solr/server/build.gradle
+++ b/solr/server/build.gradle
@@ -74,6 +74,7 @@ dependencies {
   libExt 'org.apache.logging.log4j:log4j-core'
   libExt 'org.apache.logging.log4j:log4j-slf4j-impl'
   libExt 'org.apache.logging.log4j:log4j-web'
+  libExt 'com.vlkan.log4j2:log4j2-logstash-layout'
 
   libExt(project(":solr:core"), {
     exclude module: "server"
diff --git a/solr/server/resources/jetty-logging.properties b/solr/server/resources/jetty-logging.properties
deleted file mode 100644
index 55b0c37..0000000
--- a/solr/server/resources/jetty-logging.properties
+++ /dev/null
@@ -1 +0,0 @@
-org.eclipse.jetty.util.log.class=org.eclipse.jetty.util.log.Slf4jLog
\ No newline at end of file
diff --git a/solr/server/resources/log4j2.xml b/solr/server/resources/log4j2.xml
index 9105008..def3f9b 100644
--- a/solr/server/resources/log4j2.xml
+++ b/solr/server/resources/log4j2.xml
@@ -18,145 +18,82 @@
 
 <!-- Configuration for asynchronous logging -->
 <Configuration>
-  <Appenders>
-
-    <Console name="STDOUT" target="SYSTEM_OUT">
-      <PatternLayout>
-        <Pattern>
-          %maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{node_name} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
-        </Pattern>
-      </PatternLayout>
-      <!-- <LogstashLayout dateTimeFormatPattern="yyyy-MM-dd'T'HH:mm:ss.SSSZZZ"
-                      eventTemplateUri="classpath:LogstashJsonEventLayoutV1.json"
-                      prettyPrintEnabled="true"
-                      stackTraceEnabled="true"/> -->
-    </Console>
-
-    <RollingRandomAccessFile
-        name="MainLogFile"
-        fileName="${sys:solr.log.dir}/solr.log"
-        filePattern="${sys:solr.log.dir}/solr.log.%i" >
-      <PatternLayout>
-        <Pattern>
-          %maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{node_name} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
-        </Pattern>
-      </PatternLayout>
-      <!-- <LogstashLayout dateTimeFormatPattern="yyyy-MM-dd'T'HH:mm:ss.SSSZZZ"
-                      eventTemplateUri="classpath:LogstashJsonEventLayoutV1.json"
-                      prettyPrintEnabled="true"
-                      stackTraceEnabled="true"/> -->
-      <Policies>
-        <OnStartupTriggeringPolicy />
-        <SizeBasedTriggeringPolicy size="64 MB"/>
-      </Policies>
-      <DefaultRolloverStrategy max="20"/>
-    </RollingRandomAccessFile>
-
-    <RollingRandomAccessFile
-        name="SlowLogFile"
-        fileName="${sys:solr.log.dir}/solr_slow_requests.log"
-        filePattern="${sys:solr.log.dir}/solr_slow_requests.log.%i" >
-      <PatternLayout>
-        <Pattern>
-          %maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{node_name} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
-        </Pattern>
-      </PatternLayout>
-      <!-- <LogstashLayout dateTimeFormatPattern="yyyy-MM-dd'T'HH:mm:ss.SSSZZZ"
-                      eventTemplateUri="classpath:LogstashJsonEventLayoutV1.json"
-                      prettyPrintEnabled="true"
-                      stackTraceEnabled="true"/> -->
-      <Policies>
-        <OnStartupTriggeringPolicy />
-        <SizeBasedTriggeringPolicy size="64 MB"/>
-      </Policies>
-      <DefaultRolloverStrategy max="20"/>
-    </RollingRandomAccessFile>
-
-  </Appenders>
-  <Loggers>
-
-    <AsyncLogger name="org.eclipse.jetty.servlets" level="WARN"/>
-    <AsyncLogger name="org.eclipse.jetty" level="WARN"/>
-    <AsyncLogger name="org.eclipse.jetty.server.Server" level="WARN"/>
-    <AsyncLogger name="org.apache.hadoop" level="WARN"/>
-    <AsyncLogger name="org.apache.zookeeper" level="WARN"/>
-    <AsyncLogger name="org.apache.zookeeper.ClientCnxn" level="ERROR"/>
-    <AsyncLogger name="org.apache.zookeeper.server.ZooKeeperCriticalThread" level="OFF"/>
-
-    <AsyncLogger name="org.apache.solr.update.processor.LogUpdateProcessorFactory" level="INFO"/>
-    <AsyncLogger name="org.apache.solr.update.LoggingInfoStream" level="OFF"/>
-    <AsyncLogger name="org.apache.solr.core.SolrCore.SlowRequest" level="INFO" additivity="false">
-      <AppenderRef ref="SlowLogFile"/>
-    </AsyncLogger>
-
-    <AsyncRoot level="INFO">
-      <AppenderRef ref="MainLogFile"/>
-      <AppenderRef ref="STDOUT"/>
-    </AsyncRoot>
-  </Loggers>
+    <Properties>
+        <Property name="appenderToUse">${sys:LOG4J_APPENDER:-MainLogFile}</Property>
+    </Properties>
+
+    <Appenders>
+
+        <Console name="STDOUT" target="SYSTEM_OUT">
+            <PatternLayout>
+                <Pattern>
+                    %maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{node_name} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
+                </Pattern>
+            </PatternLayout>
+            <!-- <LogstashLayout dateTimeFormatPattern="yyyy-MM-dd'T'HH:mm:ss.SSSZZZ"
+                            eventTemplateUri="classpath:LogstashJsonEventLayoutV1.json"
+                            prettyPrintEnabled="true"
+                            stackTraceEnabled="true"/> -->
+        </Console>
+
+        <RollingRandomAccessFile
+                name="MainLogFile"
+                fileName="${sys:solr.log.dir}/solr.log"
+                filePattern="${sys:solr.log.dir}/solr.log.%i">
+            <PatternLayout>
+                <Pattern>
+                    %maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{node_name} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
+                </Pattern>
+            </PatternLayout>
+            <!-- <LogstashLayout dateTimeFormatPattern="yyyy-MM-dd'T'HH:mm:ss.SSSZZZ"
+                            eventTemplateUri="classpath:LogstashJsonEventLayoutV1.json"
+                            prettyPrintEnabled="true"
+                            stackTraceEnabled="true"/> -->
+            <Policies>
+                <OnStartupTriggeringPolicy/>
+                <SizeBasedTriggeringPolicy size="64 MB"/>
+            </Policies>
+            <DefaultRolloverStrategy max="20"/>
+        </RollingRandomAccessFile>
+
+        <RollingRandomAccessFile
+                name="SlowLogFile"
+                fileName="${sys:solr.log.dir}/solr_slow_requests.log"
+                filePattern="${sys:solr.log.dir}/solr_slow_requests.log.%i">
+            <PatternLayout>
+                <Pattern>
+                    %maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{node_name} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
+                </Pattern>
+            </PatternLayout>
+            <!-- <LogstashLayout dateTimeFormatPattern="yyyy-MM-dd'T'HH:mm:ss.SSSZZZ"
+                            eventTemplateUri="classpath:LogstashJsonEventLayoutV1.json"
+                            prettyPrintEnabled="true"
+                            stackTraceEnabled="true"/> -->
+            <Policies>
+                <OnStartupTriggeringPolicy/>
+                <SizeBasedTriggeringPolicy size="64 MB"/>
+            </Policies>
+            <DefaultRolloverStrategy max="20"/>
+        </RollingRandomAccessFile>
+
+    </Appenders>
+    <Loggers>
+        <AsyncLogger name="org.eclipse.jetty.servlets" level="WARN"/>
+        <AsyncLogger name="org.eclipse.jetty" level="WARN"/>
+        <AsyncLogger name="org.eclipse.jetty.server.Server" level="WARN"/>
+        <AsyncLogger name="org.apache.hadoop" level="WARN"/>
+        <AsyncLogger name="org.apache.zookeeper" level="WARN"/>
+        <AsyncLogger name="org.apache.zookeeper.ClientCnxn" level="ERROR"/>
+        <AsyncLogger name="org.apache.zookeeper.server.ZooKeeperCriticalThread" level="OFF"/>
+
+        <AsyncLogger name="org.apache.solr.update.processor.LogUpdateProcessorFactory" level="INFO"/>
+        <AsyncLogger name="org.apache.solr.update.LoggingInfoStream" level="OFF"/>
+        <AsyncLogger name="org.apache.solr.core.SolrCore.SlowRequest" level="INFO" additivity="false">
+            <AppenderRef ref="SlowLogFile"/>
+        </AsyncLogger>
+
+        <AsyncRoot level="INFO">
+            <AppenderRef ref="${appenderToUse}"/>
+        </AsyncRoot>
+    </Loggers>
 </Configuration>
-
-<!-- Configuration for synchronous logging
-     there _may_ be a very small window where log messages will not be flushed
-     to the log file on abnormal shutdown. If even this risk is unacceptable, use
-     the configuration below
--->
-<!--Configuration>
-  <Appenders>
-
-    <Console name="STDOUT" target="SYSTEM_OUT">
-      <PatternLayout>
-        <Pattern>
-          %d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%n
-        </Pattern>
-      </PatternLayout>
-    </Console>
-
-    <RollingFile
-        name="RollingFile"
-        fileName="${sys:solr.log.dir}/solr.log"
-        filePattern="${sys:solr.log.dir}/solr.log.%i" >
-      <PatternLayout>
-        <Pattern>
-          %d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%n
-        </Pattern>
-      </PatternLayout>
-      <Policies>
-        <OnStartupTriggeringPolicy />
-        <SizeBasedTriggeringPolicy size="32 MB"/>
-      </Policies>
-      <DefaultRolloverStrategy max="10"/>
-    </RollingFile>
-
-    <RollingFile
-        name="SlowFile"
-        fileName="${sys:solr.log.dir}/solr_slow_requests.log"
-        filePattern="${sys:solr.log.dir}/solr_slow_requests.log.%i" >
-      <PatternLayout>
-        <Pattern>
-          %d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%n
-        </Pattern>
-      </PatternLayout>
-      <Policies>
-        <OnStartupTriggeringPolicy />
-        <SizeBasedTriggeringPolicy size="32 MB"/>
-      </Policies>
-      <DefaultRolloverStrategy max="10"/>
-    </RollingFile>
-
-  </Appenders>
-  <Loggers>
-    <Logger name="org.apache.hadoop" level="warn"/>
-    <Logger name="org.apache.solr.update.LoggingInfoStream" level="off"/>
-    <Logger name="org.apache.zookeeper" level="warn"/>
-    <Logger name="org.apache.solr.core.SolrCore.SlowRequest" level="info" additivity="false">
-      <AppenderRef ref="SlowFile"/>
-    </Logger>
-
-    <Root level="info">
-      <AppenderRef ref="RollingFile"/>
-      <AppenderRef ref="STDOUT"/>
-    </Root>
-  </Loggers>
-</Configuration-->
-
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
index a3973db..59f6c80 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
@@ -1136,8 +1136,6 @@ public abstract class BaseCloudSolrClient extends SolrClient {
 
     final List<String> theUrlList = new ArrayList<>(); // we populate this as follows...
 
-    System.out.println("path:" + request.getPath());
-
     if (request instanceof V2Request) {
       if (!liveNodes.isEmpty()) {
         List<String> liveNodesList = new ArrayList<>(liveNodes);
@@ -1218,7 +1216,6 @@ public abstract class BaseCloudSolrClient extends SolrClient {
             "Could not find a healthy node to handle the request, collection names: " + collectionNames);
       }
     }
-    System.out.println("urllist:" + theUrlList);
     LBSolrClient.Req req = new LBSolrClient.Req(request, theUrlList);
     LBSolrClient.Rsp rsp = getLbClient().request(req);
     return rsp.getResponse();
diff --git a/solr/solrj/src/java/org/apache/solr/common/ParWork.java b/solr/solrj/src/java/org/apache/solr/common/ParWork.java
index b3be9cb..422d9d3 100644
--- a/solr/solrj/src/java/org/apache/solr/common/ParWork.java
+++ b/solr/solrj/src/java/org/apache/solr/common/ParWork.java
@@ -421,17 +421,11 @@ public class ParWork implements Closeable {
 
       if (exception.get() != null) {
         Throwable exp = exception.get();
+        exp.fillInStackTrace();
         if (exp instanceof Error) {
           throw (Error) exp;
         }
         if (exp instanceof RuntimeException) {
-          //exp.fillInStackTrace();
-          Throwable rootCause = exp;
-          while (rootCause.getCause() != null) {
-            rootCause = rootCause.getCause();
-          }
-          rootCause.initCause(new SolrException(SolrException.ErrorCode.SERVER_ERROR, ""));
-
           throw (RuntimeException) exp;
         }
         throw new RuntimeException(exp);
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
index 3ba61b7..a9977f7 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
@@ -264,7 +264,7 @@ public class Aliases {
       }
     }
     if (uniqueResult == null) {
-      return level1;
+      return Collections.unmodifiableList(level1);
     } else {
       return Collections.unmodifiableList(new ArrayList<>(uniqueResult));
     }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index 3a895cb..d644752 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -923,7 +923,7 @@ public class SolrZkClient implements Closeable {
 
   public void printLayout(String path, int indent, StringBuilder output) {
     try {
-      printLayout(path, "", indent, new StringBuilder(64), output);
+      printLayout(path, "", indent, output);
     } catch (Exception e) {
       log.error("Exception printing layout", e);
     }
@@ -932,7 +932,7 @@ public class SolrZkClient implements Closeable {
   /**
    * Fills string with printout of current ZooKeeper layout.
    */
-  public void printLayout(String path, String node, int indent, StringBuilder internal, StringBuilder output) {
+  public void printLayout(String path, String node, int indent, StringBuilder output) {
     try {
       //log.info("path={} node={} indext={}", path, node, indent);
 
@@ -968,7 +968,6 @@ public class SolrZkClient implements Closeable {
         } else {
           childrenString = "";
         }
-        internal.append(path).append(children.size() > 0 ? node : "► " + node).append(" (").append(childrenString).append("v=" + (stat == null ? "?" : stat.getVersion()) + ")");
         output.append(dent.toString()).append(children.size() == 0 ? node : "+" + node).append(" [").append(childrenString).append("v=").append ((stat == null ? "?" : stat.getVersion()) + "]");
         StringBuilder dataBuilder = new StringBuilder();
         String dataString;
@@ -994,11 +993,9 @@ public class SolrZkClient implements Closeable {
         } else {
           output.append(NEWL);
         }
-        internal.append(dataBuilder);
         output.append(dataBuilder);
         indent += 2;
       } else {
-        internal.append("/");
         output.append("/");
       }
       if (children == null) {
@@ -1016,8 +1013,8 @@ public class SolrZkClient implements Closeable {
       }
       if (children != null) {
         for (String child : children) {
-          if (!child.equals("quota") && !(internal.toString() + child).equals("/zookeeper")) {
-            printLayout(path.equals("/") ? "/" + child : path + "/" + child, child, indent, internal, output);
+          if (!child.equals("quota") && !child.equals("/zookeeper")) {
+            printLayout(path.equals("/") ? "/" + child : path + "/" + child, child, indent, output);
           }
         }
       }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 57eb20b..8ed397c 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -72,6 +72,7 @@ import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.zookeeper.AddWatchMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.WatchedEvent;
@@ -92,7 +93,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final byte[] emptyJson = "{}".getBytes(StandardCharsets.UTF_8);
+  public static final byte[] emptyJson = Utils.toJSON(EMPTY_MAP);
 
   public static final String BASE_URL_PROP = "base_url";
   public static final String NODE_NAME_PROP = "node_name";
@@ -166,7 +167,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
   private static final String SOLR_ENVIRONMENT = "environment";
 
   public static final String REPLICA_TYPE = "type";
-  public static final byte[] EMPTY_ARRAY = new byte[0];
+
   private CloseTracker closeTracker;
 
   /**
@@ -319,8 +320,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
   private volatile boolean closed = false;
 
-  private final Set<CountDownLatch> waitLatches = ConcurrentHashMap.newKeySet(64);
-
   public ZkStateReader(SolrZkClient zkClient) {
     this(zkClient, null);
   }
@@ -374,13 +373,13 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
   public void forciblyRefreshAllClusterStateSlow() {
     // No need to set watchers because we should already have watchers registered for everything.
     try {
-      refreshCollectionList(null);
-      refreshLiveNodes(null);
+      refreshCollectionList();
+      refreshLiveNodes();
       // Need a copy so we don't delete from what we're iterating over.
       Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
       Set<String> updatedCollections = new HashSet<>();
       for (String coll : safeCopy) {
-        DocCollection newState = fetchCollectionState(coll, null);
+        DocCollection newState = fetchCollectionState(coll);
         if (updateWatchedCollection(coll, newState)) {
           updatedCollections.add(coll);
         }
@@ -399,13 +398,13 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
   public void forciblyRefreshClusterStateSlow(String name) {
     try {
-      refreshCollectionList(null);
-      refreshLiveNodes(null);
+      refreshCollectionList();
+      refreshLiveNodes();
       // Need a copy so we don't delete from what we're iterating over.
 
       Set<String> updatedCollections = new HashSet<>();
 
-      DocCollection newState = fetchCollectionState(name, null);
+      DocCollection newState = fetchCollectionState(name);
 
       if (updateWatchedCollection(name, newState)) {
         updatedCollections.add(name);
@@ -426,7 +425,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
    * Refresh the set of live nodes.
    */
   public void updateLiveNodes() throws KeeperException, InterruptedException {
-    refreshLiveNodes(null);
+    refreshLiveNodes();
   }
 
   public Integer compareStateVersions(String coll, int version) {
@@ -526,10 +525,21 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       // on reconnect of SolrZkClient force refresh and re-add watches.
       loadClusterProperties();
 
-      this.liveNodesWatcher = new LiveNodeWatcher();
-      refreshLiveNodes(this.liveNodesWatcher);
-      this.collectionsChildWatcher = new CollectionsChildWatcher();
-      refreshCollectionList(collectionsChildWatcher);
+      if (this.liveNodesWatcher == null) {
+        this.liveNodesWatcher = new LiveNodeWatcher();
+      } else {
+        this.liveNodesWatcher.removeWatch();
+      }
+      this.liveNodesWatcher.createWatch();
+      this.liveNodesWatcher.refresh();
+
+      if (this.collectionsChildWatcher == null) {
+        this.collectionsChildWatcher = new CollectionsChildWatcher();
+      } else {
+        this.collectionsChildWatcher.removeWatch();
+      }
+      this.collectionsChildWatcher.createWatch();
+      this.collectionsChildWatcher.refresh();
 
       refreshAliases(aliasesManager);
 
@@ -608,12 +618,10 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
   private void constructState(Set<String> changedCollections, String caller) {
     if (log.isDebugEnabled()) log.debug("construct new cluster state on structure change {} {}", caller, changedCollections);
 
+    Map<String,ClusterState.CollectionRef> result = new LinkedHashMap<>(watchedCollectionStates.size() + lazyCollectionStates.size());
+
     clusterStateLock.lock();
     try {
-      Set<String> liveNodes = this.liveNodes; // volatile read
-
-      Map<String,ClusterState.CollectionRef> result = new LinkedHashMap<>();
-
       // Add collections
       watchedCollectionStates.forEach((s, slices) -> {
         result.put(s, new ClusterState.CollectionRef(slices));
@@ -625,24 +633,17 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       });
 
       this.clusterState = new ClusterState(result, -1);
-
     } finally {
       clusterStateLock.unlock();
     }
 
     if (log.isDebugEnabled()) {
-      log.debug("clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
-          collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
-          lazyCollectionStates.keySet().size(),
+      log.debug("clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]", collectionWatches.keySet().size(), watchedCollectionStates.keySet().size(), lazyCollectionStates.keySet().size(),
           clusterState.getCollectionStates().size());
     }
 
     if (log.isTraceEnabled()) {
-      log.trace("clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
-          collectionWatches.keySet(),
-          watchedCollectionStates.keySet(),
-          lazyCollectionStates.keySet(),
+      log.trace("clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]", collectionWatches.keySet(), watchedCollectionStates.keySet(), lazyCollectionStates.keySet(),
           clusterState.getCollectionStates());
     }
 
@@ -660,13 +661,10 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
   /**
    * Search for any lazy-loadable collections.
    */
-  private void refreshCollectionList(CollectionsChildWatcher watcher) throws KeeperException, InterruptedException {
-    if (watcher != null) {
-      IOUtils.closeQuietly(watcher);
-    }
+  private void refreshCollectionList() throws KeeperException, InterruptedException {
     List<String> children = null;
     try {
-      children = zkClient.getChildren(COLLECTIONS_ZKNODE, watcher, true);
+      children = zkClient.getChildren(COLLECTIONS_ZKNODE, null, true);
     } catch (KeeperException.NoNodeException e) {
       log.warn("Error fetching collection names: [{}]", e.getMessage());
       // fall through
@@ -802,7 +800,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
   /**
    * Refresh live_nodes.
    */
-  private void refreshLiveNodes(LiveNodeWatcher watcher) throws KeeperException, InterruptedException {
+  private void refreshLiveNodes() throws KeeperException, InterruptedException {
     SortedSet<String> oldLiveNodes;
     SortedSet<String> newLiveNodes = null;
     liveNodesLock.lock();
@@ -810,7 +808,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       try {
 
         Stat stat = new Stat();
-        List<String> nodeList = zkClient.getChildren(LIVE_NODES_ZKNODE, watcher, stat, true);
+        List<String> nodeList = zkClient.getChildren(LIVE_NODES_ZKNODE, null, stat, true);
         this.liveNodesVersion = stat.getCversion();
         newLiveNodes = new TreeSet<>(nodeList);
       } catch (KeeperException.NoNodeException e) {
@@ -839,6 +837,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         });
       });
 
+    } catch (AlreadyClosedException e) {
+
     } finally {
       liveNodesLock.unlock();
     }
@@ -892,7 +892,10 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (cpc != null) {
         cpc.cancel(true);
       }
-      stateWatchersMap.forEach((s, stateWatcher) -> IOUtils.closeQuietly(stateWatcher));
+      stateWatchersMap.forEach((s, stateWatcher) -> {
+        IOUtils.closeQuietly(stateWatcher);
+        stateWatcher.removeWatch();
+      });
       stateWatchersMap.clear();
 
       IOUtils.closeQuietly(this.liveNodesWatcher);
@@ -1347,8 +1350,12 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     private final String coll;
     private volatile StateUpdateWatcher stateUpdateWatcher;
 
+    private final ReentrantLock collectionStateLock = new ReentrantLock(true);
+
     CollectionStateWatcher(String coll) {
       this.coll = coll;
+      String stateUpdatesPath = ZkStateReader.getCollectionStateUpdatesPath(coll);
+      stateUpdateWatcher = new StateUpdateWatcher(stateUpdatesPath);
     }
 
     @Override
@@ -1375,7 +1382,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         log.info("A cluster state change: [{}] for collection [{}] has occurred - updating... (live nodes size: [{}])", event, coll, liveNodes.size());
       }
 
-      refreshAndWatch(true);
+      refresh();
     }
 
     /**
@@ -1383,35 +1390,63 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
      * As a side effect, updates {@link #clusterState} and {@link #watchedCollectionStates}
      * with the results of the refresh.
      */
-    public void refreshAndWatch(boolean createWatcher) {
+    public void refresh() {
+      // MRM TODO: per collection lock, put it on the watcher object
+      collectionStateLock.lock();
       try {
-
-        Watcher watcher = (createWatcher ? this : null);
-        DocCollection newState = fetchCollectionState(coll, watcher);
-
+        DocCollection newState = fetchCollectionState(coll);
         updateWatchedCollection(coll, newState);
-
         constructState(Collections.singleton(coll), "state.json watcher");
-
       } catch (KeeperException e) {
         log.error("Unwatched collection: [{}]", coll, e);
         throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         log.error("Unwatched collection: [{}]", coll, e);
+      } finally {
+        collectionStateLock.unlock();
       }
     }
 
-    public void watchStateUpdates(boolean createWatcher) {
+    public void createWatch() {
+      String collectionCSNPath = getCollectionSCNPath(coll);
+      zkClient.getSolrZooKeeper().addWatch(collectionCSNPath, this, AddWatchMode.PERSISTENT, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception creating watch for " + path, ex);
+        }
+      }, "collectionStateWatcher:" + coll);
+
+      zkClient.getSolrZooKeeper().addWatch(stateUpdateWatcher.stateUpdatesPath, stateUpdateWatcher, AddWatchMode.PERSISTENT, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception creating watch for " + path, ex);
+        }
+      }, "collectionStateUpdatesWatcher:" + coll);
+    }
+
+    public void removeWatch() {
+      String collectionCSNPath = getCollectionSCNPath(coll);
+      zkClient.getSolrZooKeeper().removeWatches(collectionCSNPath, this, WatcherType.Any, true, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception removing watch for " + path, ex);
+        }
+      }, "collectionStateWatcher:" + coll);
+
+      zkClient.getSolrZooKeeper().removeWatches(stateUpdateWatcher.stateUpdatesPath, stateUpdateWatcher, WatcherType.Any, true, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception removing watch for " + path, ex);
+        }
+      }, "collectionStateUpdatesWatcher:" + coll);
+    }
+
+    public void refreshStateUpdates() {
       if (log.isDebugEnabled()) log.debug("watch for additional state updates {}", coll);
 
-      String stateUpdatesPath = ZkStateReader.getCollectionStateUpdatesPath(coll);
-      if (createWatcher) {
-        IOUtils.closeQuietly(stateUpdateWatcher);
-        stateUpdateWatcher = new StateUpdateWatcher(stateUpdatesPath);
-      }
       try {
-        processStateUpdates(stateUpdatesPath);
+        processStateUpdates(stateUpdateWatcher.stateUpdatesPath);
       } catch (Exception e) {
         log.error("Unwatched collection: [{}]", coll, e);
       }
@@ -1421,10 +1456,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
       byte[] data = null;
 
-      IOUtils.closeQuietly(stateUpdateWatcher);
-
       try {
-        data = getZkClient().getData(stateUpdatesPath, stateUpdateWatcher, null, true);
+        data = getZkClient().getData(stateUpdatesPath, null, null, true);
       } catch (NoNodeException e) {
         log.info("No node found for {}", stateUpdatesPath);
         return;
@@ -1447,7 +1480,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       m.remove("_cs_ver_");
 
       // TODO per collection?
-      clusterStateLock.lock();
+      collectionStateLock.lock();
       try {
         Set<Entry<String,Object>> entrySet = m.entrySet();
         DocCollection docCollection = clusterState.getCollectionOrNull(coll);
@@ -1525,28 +1558,29 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
             }
           }
           if (changedCollections.size() > 0) {
-            Set<String> liveNodes = ZkStateReader.this.liveNodes; // volatile read
-            // Add collections
-            for (Map.Entry<String,DocCollection> entry : watchedCollectionStates.entrySet()) {
-
-            }
-            watchedCollectionStates.forEach((s, slices) -> {
-              if (!s.equals(coll)) {
-                result.put(s, new ClusterState.CollectionRef(slices));
-              }
-            });
+            clusterStateLock.lock();
+            ClusterState cs;
+            try {
+              watchedCollectionStates.forEach((s, slices) -> {
+                if (!s.equals(coll)) {
+                  result.put(s, new ClusterState.CollectionRef(slices));
+                }
+              });
 
-            // Finally, add any lazy collections that aren't already accounted for.
-            lazyCollectionStates.forEach((s, lazyCollectionRef) -> {
-              if (!s.equals(coll)) {
-                result.putIfAbsent(s, lazyCollectionRef);
-              }
+              // Finally, add any lazy collections that aren't already accounted for.
+              lazyCollectionStates.forEach((s, lazyCollectionRef) -> {
+                if (!s.equals(coll)) {
+                  result.putIfAbsent(s, lazyCollectionRef);
+                }
 
-            });
+              });
 
-            ClusterState cs = new ClusterState(result, -2);
-            if (log.isDebugEnabled()) log.debug("Set a new clusterstate based on update diff {}", cs);
-            ZkStateReader.this.clusterState = cs;
+              cs = new ClusterState(result, -2);
+              if (log.isDebugEnabled()) log.debug("Set a new clusterstate based on update diff {}", cs);
+              ZkStateReader.this.clusterState = cs;
+            } finally {
+              clusterStateLock.unlock();
+            }
 
             notifyCloudCollectionsListeners(true);
 
@@ -1567,29 +1601,24 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       } catch (Exception e) {
         log.error("exeption trying to process additional updates", e);
       } finally {
-        clusterStateLock.unlock();
+        collectionStateLock.unlock();
       }
 
     }
 
     @Override
     public void close() throws IOException {
-      try {
-        SolrZooKeeper zk = zkClient.getSolrZooKeeper();
-        if (zk != null) {
-          if (stateUpdateWatcher != null) {
-            try {
-              zk.removeWatches(getCollectionStateUpdatesPath(coll), stateUpdateWatcher, WatcherType.Any, true);
-            } catch (KeeperException.NoWatcherException e) {
-
-            } catch (Exception e) {
-              if (log.isDebugEnabled()) log.debug("could not remove watch {} {}", e.getClass().getSimpleName(), e.getMessage());
-            }
-          }
-        }
-      } finally {
-        IOUtils.closeQuietly(stateUpdateWatcher);
-      }
+//      SolrZooKeeper zk = zkClient.getSolrZooKeeper();
+//      if (zk != null) {
+//        try {
+//          zk.removeWatches(getCollectionSCNPath(coll), this, WatcherType.Any, true);
+//        } catch (KeeperException.NoWatcherException e) {
+//
+//        } catch (Exception e) {
+//          if (log.isDebugEnabled()) log.debug("could not remove watch {} {}", e.getClass().getSimpleName(), e.getMessage());
+//        }
+//      }
+//      IOUtils.closeQuietly(stateUpdateWatcher);
     }
 
     private class StateUpdateWatcher implements Watcher, Closeable {
@@ -1601,23 +1630,23 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
       @Override
       public void close() throws IOException {
-        SolrZooKeeper zk = zkClient.getSolrZooKeeper();
-        if (zk != null) {
-          try {
-            zk.removeWatches(getCollectionSCNPath(coll), this, WatcherType.Any, true);
-          } catch (KeeperException.NoWatcherException e) {
-
-          } catch (Exception e) {
-            if (log.isDebugEnabled()) log.debug("could not remove watch {} {}", e.getClass().getSimpleName(), e.getMessage());
-          }
-        }
+//        SolrZooKeeper zk = zkClient.getSolrZooKeeper();
+//        if (zk != null) {
+//          if (stateUpdateWatcher != null) {
+//            try {
+//              zk.removeWatches(getCollectionStateUpdatesPath(coll), stateUpdateWatcher, WatcherType.Any, true);
+//            } catch (KeeperException.NoWatcherException e) {
+//
+//            } catch (Exception e) {
+//              if (log.isDebugEnabled()) log.debug("could not remove watch {} {}", e.getClass().getSimpleName(), e.getMessage());
+//            }
+//          }
+//        }
       }
 
       @Override
       public void process(WatchedEvent event) {
-        if (isClosed()) {
-          return;
-        }
+        if (closed || zkClient.isClosed()) return;
         if (log.isDebugEnabled()) log.debug("_statupdates event {}", event);
 
         try {
@@ -1627,6 +1656,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
           processStateUpdates(stateUpdatesPath);
           //            }
 
+        } catch (AlreadyClosedException e) {
+
         } catch (Exception e) {
           log.error("Unwatched collection: [{}]", coll, e);
         }
@@ -1663,6 +1694,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (EventType.None.equals(event.getType())) {
         return;
       }
+      if (closed) return;
       if (node != null) {
         MDCLoggingContext.setNode(node);
       }
@@ -1756,9 +1788,9 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (EventType.None.equals(event.getType())) {
         return;
       }
-      log.debug("A collections change: [{}], has occurred - updating...", event);
+      if (log.isDebugEnabled()) log.debug("A collections change: [{}], has occurred - updating...", event);
       try {
-        refreshAndWatch(this);
+        refresh();
       } catch (Exception e) {
         log.error("An error has occurred", e);
         return;
@@ -1767,9 +1799,11 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       constructState(Collections.emptySet(), "collection child watcher");
     }
 
-    public void refreshAndWatch(CollectionsChildWatcher watcher) {
+    public void refresh() {
       try {
-        refreshCollectionList(watcher);
+        refreshCollectionList();
+      } catch (AlreadyClosedException e) {
+
       } catch (KeeperException e) {
         log.error("A ZK error has occurred", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
@@ -1780,19 +1814,27 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       }
     }
 
-    @Override
-    public void close() throws IOException {
-      SolrZooKeeper zk = zkClient.getSolrZooKeeper();
-      if (zk != null) {
-
-        try {
-          zk.removeWatches(COLLECTIONS_ZKNODE, this, WatcherType.Any, true);
-        } catch (KeeperException.NoWatcherException e) {
+    public void createWatch() {
+      zkClient.getSolrZooKeeper().addWatch(COLLECTIONS_ZKNODE, this, AddWatchMode.PERSISTENT, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception creating watch for " + path, ex);
+        }
+      }, "collectionsChildWatcher");
+    }
 
-        } catch (Exception e) {
-          if (log.isDebugEnabled()) log.debug("could not remove watch {} {}", e.getClass().getSimpleName(), e.getMessage());
+    public void removeWatch() {
+      zkClient.getSolrZooKeeper().removeWatches(COLLECTIONS_ZKNODE, this, WatcherType.Any, true, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception removing watch for " + path, ex);
         }
-      }
+      }, "collectionsChildWatcher");
+    }
+
+    @Override
+    public void close() throws IOException {
+      removeWatch();
     }
   }
 
@@ -1807,6 +1849,9 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (EventType.None.equals(event.getType())) {
         return;
       }
+      if (closed || zkClient.isClosed()) {
+        return;
+      }
       if (node != null) {
         MDCLoggingContext.setNode(node);
       }
@@ -1814,12 +1859,12 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (log.isDebugEnabled()) {
         log.debug("A live node change: [{}], has occurred - updating... (previous live nodes size: [{}])", event, liveNodes.size());
       }
-      refreshAndWatch();
+      refresh();
     }
 
-    public void refreshAndWatch() {
+    public void refresh() {
       try {
-        refreshLiveNodes(this);
+        refreshLiveNodes();
       } catch (KeeperException e) {
         log.error("A ZK error has occurred", e);
       } catch (InterruptedException e) {
@@ -1827,26 +1872,34 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       }
     }
 
+    public void createWatch() {
+      zkClient.getSolrZooKeeper().addWatch(LIVE_NODES_ZKNODE, this, AddWatchMode.PERSISTENT, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception creating watch for " + path, ex);
+        }
+      }, "liveNodesWatcher");
+    }
 
-    @Override
-    public void close() throws IOException {
-      SolrZooKeeper zk = zkClient.getSolrZooKeeper();
-      if (zk != null) {
+    public void removeWatch() {
+      zkClient.getSolrZooKeeper().removeWatches(LIVE_NODES_ZKNODE, this, WatcherType.Any, true, (rc, path, ctx) -> {
+        if (rc != 0) {
+          KeeperException ex = KeeperException.create(KeeperException.Code.get(rc), path);
+          log.error("Exception removing watch for " + path, ex);
+        }
+      }, "liveNodesWatcher");
+    }
 
-        try {
-          zk.removeWatches(ZkStateReader.LIVE_NODES_ZKNODE, this, WatcherType.Any, true);
-        } catch (KeeperException.NoWatcherException e) {
 
-        } catch (Exception e) {
-          if (log.isDebugEnabled()) log.debug("could not remove watch {} {}", e.getClass().getSimpleName(), e.getMessage());
-        }
-      }
+    @Override
+    public void close() throws IOException {
+      removeWatch();
     }
   }
 
   public static DocCollection getCollectionLive(ZkStateReader zkStateReader, String coll) {
     try {
-      return zkStateReader.fetchCollectionState(coll, null);
+      return zkStateReader.fetchCollectionState(coll);
     } catch (KeeperException.SessionExpiredException | InterruptedException e) {
       ParWork.propagateInterrupt(e);
       throw new AlreadyClosedException("Could not load collection from ZK: " + coll, e);
@@ -1856,43 +1909,47 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     }
   }
 
-  private DocCollection fetchCollectionState(String coll, Watcher watcher) throws KeeperException, InterruptedException {
-    String collectionPath = getCollectionPath(coll);
-    String collectionCSNPath = getCollectionSCNPath(coll);
-    if (log.isDebugEnabled()) log.debug("Looking at fetching full clusterstate");
-    Stat exists = zkClient.exists(collectionCSNPath, watcher, true);
-    int version = 0;
-    if (exists != null) {
-
-      Stat stateStat = zkClient.exists(collectionPath, null, true);
-      if (stateStat != null) {
-        version = stateStat.getVersion();
-        if (log.isDebugEnabled()) log.debug("version for cs is {}", version);
-        // version we would get
-        DocCollection docCollection = watchedCollectionStates.get(coll);
-        if (docCollection != null) {
-          int localVersion = docCollection.getZNodeVersion();
-          if (log.isDebugEnabled()) log.debug("found version {}, our local version is {}, has updates {}", version, localVersion, docCollection.hasStateUpdates());
-          if (docCollection.hasStateUpdates()) {
-            if (localVersion > version) {
-              return docCollection;
-            }
-          } else {
-            if (localVersion >= version) {
-              return docCollection;
+  private DocCollection fetchCollectionState(String coll) throws KeeperException, InterruptedException {
+    try {
+      String collectionPath = getCollectionPath(coll);
+      String collectionCSNPath = getCollectionSCNPath(coll);
+      if (log.isDebugEnabled()) log.debug("Looking at fetching full clusterstate");
+      Stat exists = zkClient.exists(collectionCSNPath, null, true);
+      int version = 0;
+      if (exists != null) {
+
+        Stat stateStat = zkClient.exists(collectionPath, null, true);
+        if (stateStat != null) {
+          version = stateStat.getVersion();
+          if (log.isDebugEnabled()) log.debug("version for cs is {}", version);
+          // version we would get
+          DocCollection docCollection = watchedCollectionStates.get(coll);
+          if (docCollection != null) {
+            int localVersion = docCollection.getZNodeVersion();
+            if (log.isDebugEnabled()) log.debug("found version {}, our local version is {}, has updates {}", version, localVersion, docCollection.hasStateUpdates());
+            if (docCollection.hasStateUpdates()) {
+              if (localVersion > version) {
+                return docCollection;
+              }
+            } else {
+              if (localVersion >= version) {
+                return docCollection;
+              }
             }
           }
         }
+        if (log.isDebugEnabled()) log.debug("getting latest state.json knowing it's at least {}", version);
+        Stat stat = new Stat();
+        byte[] data = zkClient.getData(collectionPath, null, stat, true);
+        if (data == null) return null;
+        ClusterState state = ClusterState.createFromJson(this, stat.getVersion(), data);
+        ClusterState.CollectionRef collectionRef = state.getCollectionStates().get(coll);
+        return collectionRef == null ? null : collectionRef.get();
       }
-      if (log.isDebugEnabled()) log.debug("getting latest state.json knowing it's at least {}", version);
-      Stat stat = new Stat();
-      byte[] data = zkClient.getData(collectionPath, null, stat, true);
-      if (data == null) return null;
-      ClusterState state = ClusterState.createFromJson(this, stat.getVersion(), data);
-      ClusterState.CollectionRef collectionRef = state.getCollectionStates().get(coll);
-      return collectionRef == null ? null : collectionRef.get();
-    }
 
+    } catch (AlreadyClosedException e) {
+
+    }
     return null;
   }
 
@@ -1935,16 +1992,15 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (v == null) {
         reconstructState.set(true);
         v = new CollectionWatch<>();
+        CollectionStateWatcher sw = new CollectionStateWatcher(collection);
+        stateWatchersMap.put(collection, sw);
+        sw.createWatch();
+        sw.refresh();
+        sw.refreshStateUpdates();
       }
       v.coreRefCount.incrementAndGet();
       return v;
     });
-    if (reconstructState.get()) {
-      CollectionStateWatcher sw = new CollectionStateWatcher(collection);
-      stateWatchersMap.put(collection, sw);
-      sw.refreshAndWatch(true);
-      sw.watchStateUpdates(true);
-    }
 
   }
 
@@ -1974,7 +2030,11 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         v.coreRefCount.decrementAndGet();
       if (v.canBeRemoved()) {
         watchedCollectionStates.remove(collection);
-        IOUtils.closeQuietly(stateWatchersMap.remove(collection));
+        CollectionStateWatcher watcher = stateWatchersMap.remove(collection);
+        if (watcher != null) {
+          IOUtils.closeQuietly(watcher);
+          watcher.removeWatch();
+        }
         lazyCollectionStates.put(collection, new LazyCollectionRef(collection));
         reconstructState.set(true);
         return null;
@@ -2035,23 +2095,19 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       throw new IllegalArgumentException("Collection cannot be null");
     }
 
-    AtomicBoolean watchSet = new AtomicBoolean(false);
-
     collectionWatches.compute(collection, (k, v) -> {
       if (v == null) {
         v = new CollectionWatch<>();
-        watchSet.set(true);
+        CollectionStateWatcher sw = new CollectionStateWatcher(collection);
+        stateWatchersMap.put(collection, sw);
+        sw.createWatch();
+        sw.refresh();
+        sw.refreshStateUpdates();
       }
       v.stateWatchers.add(stateWatcher);
       return v;
     });
 
-    if (watchSet.get()) {
-      CollectionStateWatcher sw = new CollectionStateWatcher(collection);
-      sw.refreshAndWatch(true);
-      sw.watchStateUpdates(true);
-    }
-
     DocCollection state = clusterState.getCollectionOrNull(collection);
     if (stateWatcher.onStateChanged(state) == true) {
       removeDocCollectionWatcher(collection, stateWatcher);
@@ -2089,7 +2145,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       return;
     }
     final CountDownLatch latch = new CountDownLatch(1);
-    waitLatches.add(latch);
     AtomicReference<DocCollection> docCollection = new AtomicReference<>();
     org.apache.solr.common.cloud.CollectionStateWatcher watcher = (n, c) -> {
       // if (isClosed()) return true;
@@ -2115,7 +2170,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       }
     } finally {
       removeCollectionStateWatcher(collection, watcher);
-      waitLatches.remove(latch);
     }
   }
 
@@ -2168,7 +2222,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     }
 
     final CountDownLatch latch = new CountDownLatch(1);
-    waitLatches.add(latch);
 
     LiveNodesListener listener = (n) -> {
       boolean matches = predicate.matches(n);
@@ -2189,7 +2242,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
     } finally {
       removeLiveNodesListener(listener);
-      waitLatches.remove(latch);
     }
   }
 
@@ -2243,6 +2295,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         CollectionStateWatcher stateWatcher = stateWatchersMap.remove(collection);
         if (stateWatcher != null) {
           IOUtils.closeQuietly(stateWatcher);
+          stateWatcher.removeWatch();
         }
         reconstructState.set(true);
         return null;
@@ -2859,7 +2912,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     long num = -1;
     long lastNum = -1;
     String failMessage = null;
-    if (verbose) System.err.println("check const of " + shard);
+    if (verbose) System.err.println("\nCheck consistency of shard: " + shard);
+    if (verbose) System.err.println("__________________________\n");
     int cnt = 0;
 
     DocCollection coll = getClusterState().getCollection(collection);
@@ -2869,8 +2923,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     Replica lastReplica = null;
     for (Replica replica : replicas) {
 
-      if (verbose) System.err.println("client" + cnt++);
-      if (verbose) System.err.println("Replica:" + replica);
+      //if (verbose) System.err.println("client" + cnt++);
+      if (verbose) System.err.println("Replica: " + replica);
       try (SolrClient client = getHttpClient(replica.getCoreUrl())) {
         try {
           SolrParams query = params("q","*:*", "rows","0", "distrib","false", "tests","checkShardConsistency"); // "tests" is just a tag that won't do anything except be echoed in logs
@@ -2886,8 +2940,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         if (isNodeLive(nodeName)) {
           live = true;
         }
-        if (verbose) System.err.println(" live:" + live);
-        if (verbose) System.err.println(" num:" + num + "\n");
+        if (verbose) System.err.println(" Live:" + live);
+        if (verbose) System.err.println(" Count:" + num + "\n");
 
         boolean active = replica.getState() == Replica.State.ACTIVE;
         if (active && live) {
@@ -2988,6 +3042,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (EventType.None.equals(event.getType())) {
         return;
       }
+      if (closed) return;
       loadClusterProperties();
     }
 
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
index 9a54493..25613ae 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
@@ -92,7 +92,7 @@ public class SolrCloudTestCase extends SolrTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final int DEFAULT_TIMEOUT = 30; // this is SECONDS, not MILLIS
+  public static final int DEFAULT_TIMEOUT = 10; // this is SECONDS, not MILLIS
   public static final TimeUnit DEFAULT_TIMEOUT_UNIT = TimeUnit.SECONDS;
 
   protected static volatile SolrQueuedThreadPool qtp;
diff --git a/solr/test-framework/src/resources/logconf/log4j2-std-debug.xml b/solr/test-framework/src/resources/logconf/log4j2-std-debug.xml
index 91acf82..7eb4338 100644
--- a/solr/test-framework/src/resources/logconf/log4j2-std-debug.xml
+++ b/solr/test-framework/src/resources/logconf/log4j2-std-debug.xml
@@ -39,8 +39,8 @@
         <AsyncLogger name="org.eclipse.jetty" level="INFO"/>
 
         <AsyncLogger name="org.apache.solr.core.SolrCore" level="DEBUG"/>
-        <AsyncLogger name="org.apache.solr.core.DefaultSolrCore" level="DEBUG"/>
-        <AsyncLogger name="org.apache.solr.core.SolrCoreState" level="DEBUG"/>
+        <AsyncLogger name="org.apache.solr.core.DefaultSolrCore" level="INFO"/>
+        <AsyncLogger name="org.apache.solr.core.SolrCoreState" level="INFO"/>
 
         <AsyncLogger name="org.apache.solr.handler.admin.CollectionsHandler" level="INFO"/>
         <AsyncLogger name="org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler" level="INFO"/>