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

[lucene-solr] branch reference_impl_dev updated: @1421 Clean up the last commit.

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 0cd05a2  @1421 Clean up the last commit.
0cd05a2 is described below

commit 0cd05a2bf58c3f6a0d5887f69de6ba631994a650
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Wed Mar 3 15:47:29 2021 -0600

    @1421 Clean up the last commit.
    
    Took 7 hours 11 minutes
---
 .../apache/solr/cloud/OverseerElectionContext.java |   2 +-
 .../solr/cloud/ShardLeaderElectionContext.java     |  11 +-
 .../solr/cloud/ShardLeaderElectionContextBase.java |   5 +-
 .../java/org/apache/solr/cloud/StatePublisher.java |  12 +-
 .../java/org/apache/solr/cloud/ZkController.java   |  29 +++--
 .../apache/solr/cloud/ZkSolrResourceLoader.java    |  32 +++++-
 .../solr/cloud/api/collections/AddReplicaCmd.java  |   1 +
 .../cloud/api/collections/CreateCollectionCmd.java |  11 +-
 .../solr/cloud/api/collections/RestoreCmd.java     |   2 +-
 .../solr/cloud/api/collections/SplitShardCmd.java  |   2 +-
 .../solr/cloud/overseer/ClusterStateMutator.java   |   6 +-
 .../solr/cloud/overseer/CollectionMutator.java     |   2 +-
 .../apache/solr/cloud/overseer/NodeMutator.java    |   4 +-
 .../apache/solr/cloud/overseer/ReplicaMutator.java |  17 ++-
 .../apache/solr/cloud/overseer/SliceMutator.java   |  16 +--
 .../apache/solr/cloud/overseer/ZkStateWriter.java  |  16 +--
 .../apache/solr/core/CachingDirectoryFactory.java  |   6 +-
 .../java/org/apache/solr/core/CoreDescriptor.java  |   4 +
 .../apache/solr/core/CorePropertiesLocator.java    |   4 +-
 .../src/java/org/apache/solr/core/PluginBag.java   |   2 +-
 .../src/java/org/apache/solr/core/SolrConfig.java  |  11 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |  59 ++++------
 .../java/org/apache/solr/core/SolrTinyBuilder.java |   1 -
 .../java/org/apache/solr/core/XmlConfigFile.java   | 124 ++++++++-------------
 .../org/apache/solr/handler/SolrConfigHandler.java |   1 -
 .../solr/handler/admin/RequestSyncShardOp.java     |   4 +-
 .../handler/component/RealTimeGetComponent.java    |   3 +-
 .../org/apache/solr/handler/loader/CSVLoader.java  |   2 +-
 .../org/apache/solr/handler/loader/JsonLoader.java |   8 +-
 .../org/apache/solr/request/SolrRequestInfo.java   |   3 -
 .../org/apache/solr/schema/AbstractEnumField.java  |  45 +++++---
 .../apache/solr/servlet/DirectSolrConnection.java  |   4 +
 .../AddSchemaFieldsUpdateProcessorFactory.java     |  25 ++---
 .../src/java/org/apache/solr/util/FileUtils.java   |   2 +-
 .../java/org/apache/solr/util/SafeXMLParsing.java  |   1 -
 .../org/apache/solr/BasicFunctionalityTest.java    |  92 ++++++++-------
 .../embedded/TestEmbeddedSolrServerSchemaAPI.java  |   1 -
 .../apache/solr/cloud/ClusterStateMockUtil.java    |   7 +-
 .../org/apache/solr/cloud/ClusterStateTest.java    |  13 ++-
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |   4 +-
 .../org/apache/solr/cloud/LeaderElectionTest.java  |   4 +-
 .../test/org/apache/solr/cloud/OverseerTest.java   |   9 +-
 .../org/apache/solr/cloud/TestHashPartitioner.java |   7 +-
 .../org/apache/solr/cloud/TestPullReplica.java     |   2 +-
 .../test/org/apache/solr/core/CoreSorterTest.java  |   4 +-
 .../solr/handler/component/SearchHandlerTest.java  |   9 +-
 .../apache/solr/request/TestIntervalFaceting.java  |  30 +++--
 .../AddSchemaFieldsUpdateProcessorFactoryTest.java |   7 +-
 .../processor/AtomicUpdateRemovalJavabinTest.java  |  10 +-
 .../processor/PreAnalyzedUpdateProcessorTest.java  |   9 +-
 .../client/solrj/impl/CloudHttp2SolrClient.java    |   3 +-
 .../solr/client/solrj/impl/CloudSolrClient.java    |   1 +
 .../solrj/impl/ZkClientClusterStateProvider.java   |  22 ++--
 .../org/apache/solr/common/cloud/ClusterState.java |   5 +-
 .../apache/solr/common/cloud/DocCollection.java    |  66 +++++------
 .../java/org/apache/solr/common/cloud/Replica.java |  42 ++++---
 .../java/org/apache/solr/common/cloud/Slice.java   |  16 ++-
 .../org/apache/solr/common/cloud/SolrZkClient.java |  24 +++-
 .../solr/common/cloud/ZkMaintenanceUtils.java      |   2 +-
 .../org/apache/solr/common/cloud/ZkNodeProps.java  |   8 ++
 .../apache/solr/common/cloud/ZkStateReader.java    |  80 +++++++------
 .../ref_guide_examples/JsonRequestApiTest.java     |   8 +-
 .../solrj/impl/BaseSolrClientWireMockTest.java     |   5 +-
 .../solrj/impl/CloudHttp2SolrClientRetryTest.java  |  16 ++-
 .../solrj/impl/CloudSolrClientRetryTest.java       |  11 +-
 .../solrj/impl/TestCloudSolrClientConnections.java |  69 ++----------
 .../routing/NodePreferenceRulesComparatorTest.java |  20 ++--
 .../solrj/routing/ReplicaListTransformerTest.java  |   3 +-
 ...RequestReplicaListTransformerGeneratorTest.java |  25 +++--
 .../ShufflingReplicaListTransformerTest.java       |   3 +-
 .../src/java/org/apache/solr/SolrTestCase.java     |  13 ++-
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |   1 +
 .../update/processor/UpdateProcessorTestBase.java  |   1 +
 .../src/java/org/apache/solr/util/TestHarness.java |  19 ++--
 .../src/resources/logconf/log4j2-std.xml           |  14 ++-
 75 files changed, 635 insertions(+), 527 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
index d294266..a7e3391 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
@@ -38,7 +38,7 @@ final class OverseerElectionContext extends ShardLeaderElectionContextBase {
   private final Overseer overseer;
 
   public OverseerElectionContext(final String zkNodeName, SolrZkClient zkClient, Overseer overseer) {
-    super(zkNodeName, Overseer.OVERSEER_ELECT, Overseer.OVERSEER_ELECT + "/leader", new Replica("overseer:" + overseer.getZkController().getNodeName(), getIDMap(zkNodeName, overseer), "overseer", "overseer", overseer.getZkStateReader()), null, zkClient);
+    super(zkNodeName, Overseer.OVERSEER_ELECT, Overseer.OVERSEER_ELECT + "/leader", new Replica("overseer:" + overseer.getZkController().getNodeName(), getIDMap(zkNodeName, overseer), "overseer", -1l, "overseer", overseer.getZkStateReader()), null, zkClient);
     this.overseer = overseer;
     this.zkClient = zkClient;
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
index 923a6ce..3cbdd73 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
@@ -58,8 +58,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
 
   public ShardLeaderElectionContext(LeaderElector leaderElector,
                                     final String shardId, final String collection,
-                                    final String coreNodeName, Replica props, ZkController zkController, CoreContainer cc, CoreDescriptor cd) {
-    super(coreNodeName, ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection
+                                    Replica props, ZkController zkController, CoreContainer cc, CoreDescriptor cd) {
+    super(cd.getName(), ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection
                     + "/leader_elect/" + shardId,  ZkStateReader.getShardLeadersPath(
             collection, shardId), props, cd,
             zkController.getZkClient());
@@ -78,11 +78,9 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
 
   @Override
   public ElectionContext copy() {
-    return new ShardLeaderElectionContext(leaderElector, shardId, collection, id, leaderProps, zkController, cc, cd);
+    return new ShardLeaderElectionContext(leaderElector, shardId, collection, leaderProps, zkController, cc, cd);
   }
 
-
-
   public LeaderElector getLeaderElector() {
     return leaderElector;
   }
@@ -270,7 +268,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         }
 
         ZkNodeProps zkNodes = ZkNodeProps
-            .fromKeyVals(StatePublisher.OPERATION, OverseerAction.STATE.toLower(), ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.CORE_NAME_PROP, leaderProps.getName(), "id", leaderProps.getId(),
+            .fromKeyVals(StatePublisher.OPERATION, OverseerAction.STATE.toLower(), ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.CORE_NAME_PROP, leaderProps.getName(),
+                "id", cd.getCoreProperties().get("collId") + "-" + cd.getCoreProperties().get("id"),
                 ZkStateReader.STATE_PROP, "leader");
 
         log.info("I am the new leader, publishing as active: " + leaderProps.getCoreUrl() + " " + shardId);
diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
index f8c5752..3e3896b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
@@ -52,21 +52,20 @@ class ShardLeaderElectionContextBase extends ElectionContext {
   protected volatile Integer leaderZkNodeParentVersion;
 
   public ShardLeaderElectionContextBase(final String coreNodeName, String electionPath, String leaderPath,
-                                        Replica props, CoreDescriptor cd,SolrZkClient zkClient) {
+                                        Replica props, CoreDescriptor cd, SolrZkClient zkClient) {
     super(coreNodeName, electionPath, leaderPath, props, cd);
     this.zkClient = zkClient;
   }
 
   @Override
   protected void cancelElection() throws InterruptedException, KeeperException {
-
+    if (!zkClient.isAlive()) return;
     if (log.isTraceEnabled()) log.trace("cancelElection");
     //    if (!zkClient.isConnected()) {
     //      log.info("Can't cancel, zkClient is not connected");
     //      return;
     //    }
     super.cancelElection();
-  //  if (zkClient.isAlive()) {
       try {
         if (leaderZkNodeParentVersion != null) {
           try {
diff --git a/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java b/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
index fa66fe0..84e4a94 100644
--- a/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
+++ b/solr/core/src/java/org/apache/solr/cloud/StatePublisher.java
@@ -196,7 +196,7 @@ public class StatePublisher implements Closeable {
     try {
       if (stateMessage != TERMINATE_OP) {
         String operation = stateMessage.getStr(OPERATION);
-        String id = stateMessage.getStr("id");
+        String id = null;
         if (operation.equals("state")) {
           String core = stateMessage.getStr(ZkStateReader.CORE_NAME_PROP);
           String collection = stateMessage.getStr(ZkStateReader.COLLECTION_PROP);
@@ -205,7 +205,11 @@ public class StatePublisher implements Closeable {
           DocCollection coll = zkStateReader.getClusterState().getCollectionOrNull(collection);
           if (coll != null) {
             Replica replica = coll.getReplica(core);
-            id = replica.getId();
+            if (replica != null) {
+              id = replica.getId();
+            } else {
+              id = stateMessage.getStr("id");
+            }
             String lastState = stateCache.get(id);
             if (collection != null && !state.equals(Replica.State.ACTIVE) && state.equals(lastState) && replica.getState().toString().equals(state)) {
               log.info("Skipping publish state as {} for {}, because it was the last state published", state, core);
@@ -218,6 +222,10 @@ public class StatePublisher implements Closeable {
             throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Nulls in published state " + stateMessage);
           }
 
+          if (id == null) {
+            id = stateMessage.getStr("id");
+          }
+
           stateMessage.getProperties().put("id", id);
 
           stateCache.put(id, state);
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 9c8fe19..ca16fcb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1344,7 +1344,7 @@ public class ZkController implements Closeable, Runnable {
           break;
         }
         try {
-          Replica leader = zkStateReader.getLeaderRetry(collection, shardId, 3000, true);
+          Replica leader = zkStateReader.getLeaderRetry(collection, shardId, 6000, true);
           leaderName = leader.getName();
 
         } catch (TimeoutException timeoutException) {
@@ -1492,9 +1492,9 @@ public class ZkController implements Closeable, Runnable {
    * Get leader props directly from zk nodes.
    * @throws SessionExpiredException on zk session expiration.
    */
-  public Replica getLeaderProps(final String collection,
+  public Replica getLeaderProps(final String collection, long id,
                                         final String slice, int timeoutms) throws InterruptedException, SessionExpiredException {
-    return getLeaderProps(collection, slice, timeoutms, true);
+    return getLeaderProps(collection, id, slice, timeoutms, true);
   }
 
   /**
@@ -1503,7 +1503,7 @@ public class ZkController implements Closeable, Runnable {
    * @return leader props
    * @throws SessionExpiredException on zk session expiration.
    */
-  public Replica getLeaderProps(final String collection, final String slice, int timeoutms, boolean failImmediatelyOnExpiration)
+  public Replica getLeaderProps(final String collection, long id, final String slice, int timeoutms, boolean failImmediatelyOnExpiration)
       throws InterruptedException, SessionExpiredException { // MRM TODO: look at failImmediatelyOnExpiration
 
     try {
@@ -1512,7 +1512,7 @@ public class ZkController implements Closeable, Runnable {
       byte[] data = zkClient.getData(ZkStateReader.getShardLeadersPath(collection, slice), null, null);
       ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(ZkNodeProps.load(data));
 
-      return new Replica(leaderProps.getNodeProps().getStr(CORE_NAME_PROP), leaderProps.getNodeProps().getProperties(), collection, slice, zkStateReader);
+      return new Replica(leaderProps.getNodeProps().getStr(CORE_NAME_PROP), leaderProps.getNodeProps().getProperties(), collection, id, slice, zkStateReader);
 
     } catch (Exception e) {
       SolrZkClient.checkInterrupted(e);
@@ -1533,9 +1533,20 @@ public class ZkController implements Closeable, Runnable {
     // we only put a subset of props into the leader node
     props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
     props.put(CORE_NAME_PROP, cd.getName());
-    props.put("id", cd.getCoreProperty("id", "-1"));
 
-    Replica replica = new Replica(cd.getName(), props, collection, shardId, zkStateReader);
+    String id = cd.getCoreProperty("id", "-1");
+    if (id.equals("-1")) {
+      throw new IllegalArgumentException("no id found props=" + cd.getCoreProperties());
+    }
+
+    props.put("id", id);
+
+    String collId = cd.getCoreProperty("collId", "-1");
+    if (collId.equals("-1")) {
+      throw new IllegalArgumentException("no id found props=" + cd.getCoreProperties());
+    }
+
+    Replica replica = new Replica(cd.getName(), props, collection, Long.parseLong(collId), shardId, zkStateReader);
     LeaderElector leaderElector;
 
     if (isDcCalled() || isClosed) {
@@ -1551,7 +1562,7 @@ public class ZkController implements Closeable, Runnable {
     }
 
     ElectionContext context = new ShardLeaderElectionContext(leaderElector, shardId,
-        collection, cd.getName(), replica, this, cc, cd);
+        collection, replica, this, cc, cd);
 
 
     leaderElector.setup(context);
@@ -1600,7 +1611,7 @@ public class ZkController implements Closeable, Runnable {
 
       props.put(Overseer.QUEUE_OPERATION, "state");
       props.put(ZkStateReader.STATE_PROP, state.toString());
-      props.put("id", cd.getCoreProperty("id", "-1"));
+      props.put("id", cd.getCoreProperty("collId", "-1") + "-" + cd.getCoreProperty("id", "-1"));
       //  props.put(ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles());
       props.put(CORE_NAME_PROP, cd.getName());
       //  props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
index 981fac6..9449813 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
@@ -21,7 +21,10 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
+import java.util.Collections;
+import java.util.Map;
 
+import org.apache.commons.collections.map.ReferenceMap;
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
@@ -45,6 +48,15 @@ public class ZkSolrResourceLoader extends SolrResourceLoader implements Resource
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final SolrZkClient zkClient;
 
+  private static Map CONFIG_CACHE = new ReferenceMap(ReferenceMap.HARD, ReferenceMap.WEAK) {
+    {
+      purgeBeforeRead();
+      purgeBeforeWrite();
+    }
+  };
+
+  private static Map<String, ZkSolrResourceLoader.ZkByteArrayInputStream> SYNC_CONFIG_CACHE = Collections.synchronizedMap(CONFIG_CACHE);
+
   /**
    * <p>
    * This loader will first attempt to load resources from ZooKeeper, but if not found
@@ -72,6 +84,22 @@ public class ZkSolrResourceLoader extends SolrResourceLoader implements Resource
 
     try {
 
+      boolean usedCached = false;
+
+      ZkSolrResourceLoader.ZkByteArrayInputStream cached = SYNC_CONFIG_CACHE.get(file);
+      if (cached != null) {
+        Stat checkStat;
+        try {
+          checkStat = zkClient.exists(file, null);
+        } catch (KeeperException | InterruptedException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+        }
+        if (checkStat != null && checkStat.getVersion() <= cached.getStat().getVersion()) {
+          return new ZkSolrResourceLoader.ZkByteArrayInputStream(cached.getBytes(), cached.getStat());
+        }
+
+      }
+
       Stat stat = new Stat();
       byte[] bytes = zkClient.getData(file, null, stat);
       if (bytes == null) {
@@ -80,7 +108,9 @@ public class ZkSolrResourceLoader extends SolrResourceLoader implements Resource
                 + "' in classpath or '" + configSetZkPath + "', cwd="
                 + System.getProperty("user.dir"));
       }
-      return new ZkByteArrayInputStream(bytes, stat);
+      ZkByteArrayInputStream is = new ZkByteArrayInputStream(bytes, stat);
+      SYNC_CONFIG_CACHE.put(file, (ZkSolrResourceLoader.ZkByteArrayInputStream) is);
+      return is;
     } catch (InterruptedException e) {
       ParWork.propagateInterrupt(e);
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Interrupted while opening " + file, e);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
index 0a66b8e..e314348 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
@@ -320,6 +320,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       params.set(CoreAdminParams.PROPERTY_PREFIX + "bufferOnStart", "true");
     }
     params.set(CoreAdminParams.PROPERTY_PREFIX + "id", Long.toString(createReplica.id));
+    params.set(CoreAdminParams.PROPERTY_PREFIX + "collId", Long.toString(collection.getId()));
 
     log.info("Creating SolrCore with name={}", createReplica.coreName);
     if (createReplica.sliceName != null) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
index 1ef5690..a8e4d0f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
@@ -217,7 +217,8 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
 
       OverseerCollectionMessageHandler.createConfNode(cloudManager.getDistribStateManager(), configName, collectionName);
 
-      DocCollection docCollection = buildDocCollection(cloudManager, ocmh.overseer.getZkStateWriter().getHighestId(), message, true);
+      long id = ocmh.overseer.getZkStateWriter().getHighestId();
+      DocCollection docCollection = buildDocCollection(cloudManager, id, message, true);
       clusterState = clusterState.copyWith(collectionName, docCollection);
       try {
         replicaPositions = buildReplicaPositions(cloudManager, message, shardNames);
@@ -280,6 +281,8 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
         params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.CREATE.toString());
 
         params.set(CoreAdminParams.NAME, coreName);
+        params.set(CoreAdminParams.PROPERTY_PREFIX + "id",  Long.toString(docCollection.getHighestReplicaId()));
+        params.set(CoreAdminParams.PROPERTY_PREFIX + "collId", Long.toString(id));
         params.set(COLL_CONF, configName);
         params.set(CoreAdminParams.COLLECTION, collectionName);
         params.set(CoreAdminParams.SHARD, replicaPosition.shard);
@@ -516,7 +519,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
     }
   }
 
-  public static DocCollection buildDocCollection(SolrCloudManager cloudManager, long id, ZkNodeProps message, boolean withDocRouter) {
+  public static DocCollection buildDocCollection(SolrCloudManager cloudManager, Long id, ZkNodeProps message, boolean withDocRouter) {
     if (log.isDebugEnabled()) log.debug("buildDocCollection {}", message);
     String cName = message.getStr(NAME);
     DocRouter router = null;
@@ -530,7 +533,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
 
     Map<String,Slice> slices;
     if (messageShardsObj instanceof Map) { // we are being explicitly told the slice data (e.g. coll restore)
-      slices = Slice.loadAllFromMap((Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider(), message.getStr(ZkStateReader.COLLECTION_PROP), (Map<String,Object>) messageShardsObj);
+      slices = Slice.loadAllFromMap((Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider(), message.getStr(ZkStateReader.COLLECTION_PROP), id, (Map<String,Object>) messageShardsObj);
     } else {
       List<String> shardNames = new ArrayList<>();
       if (withDocRouter) {
@@ -559,7 +562,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
           sliceProps.put(Slice.RANGE, ranges == null ? null : ranges.get(i));
         }
 
-        slices.put(sliceName, new Slice(sliceName, null, sliceProps, message.getStr(ZkStateReader.COLLECTION_PROP), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider()));
+        slices.put(sliceName, new Slice(sliceName, null, sliceProps, message.getStr(ZkStateReader.COLLECTION_PROP), id, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider()));
 
       }
     }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
index de2466d..5fe41c5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
@@ -202,7 +202,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
         Map<String, Slice> newSlices = new LinkedHashMap<>(backupSlices.size());
         for (Slice backupSlice : backupSlices) {
           newSlices.put(backupSlice.getName(),
-              new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties(),restoreCollectionName, ocmh.zkStateReader));
+              new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties(), restoreCollectionName, -1l, ocmh.zkStateReader)); // MRM TODO
         }
         propMap.put(OverseerCollectionMessageHandler.SHARDS_PROP, newSlices);
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
index a76cc56..7fb2e1c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
@@ -685,7 +685,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
           ClusterState completeCs = finalClusterState;
           for (Map<String,Object> replica : replicas) {
              completeCs = checkAndCompleteShardSplit(completeCs, coll, replica.get("name").toString(), replica.get("shard").toString(),
-                new Replica(replica.get("name").toString(), replica, replica.get("collection").toString(), replica.get("shard").toString(), ocmh.zkStateReader));
+                new Replica(replica.get("name").toString(), replica, replica.get("collection").toString(), -1l, replica.get("shard").toString(), ocmh.zkStateReader));
           }
 
           AddReplicaCmd.Response response = new AddReplicaCmd.Response();
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
index dcaddc0..17ea6ca 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
@@ -68,7 +68,7 @@ public class ClusterStateMutator {
 
     Map<String, Slice> slices;
     if (messageShardsObj instanceof Map) { // we are being explicitly told the slice data (e.g. coll restore)
-      slices = Slice.loadAllFromMap((Replica.NodeNameToBaseUrl) dataProvider.getClusterStateProvider(), cName, (Map<String, Object>)messageShardsObj);
+      slices = Slice.loadAllFromMap((Replica.NodeNameToBaseUrl) dataProvider.getClusterStateProvider(), cName,-1l,  (Map<String, Object>)messageShardsObj);
     } else {
       List<String> shardNames = new ArrayList<>();
 
@@ -89,7 +89,7 @@ public class ClusterStateMutator {
         Map<String, Object> sliceProps = new LinkedHashMap<>(1);
         sliceProps.put(Slice.RANGE, ranges == null ? null : ranges.get(i));
 
-        slices.put(sliceName, new Slice(sliceName, null, sliceProps,cName, (Replica.NodeNameToBaseUrl) dataProvider.getClusterStateProvider()));
+        slices.put(sliceName, new Slice(sliceName, null, sliceProps,cName, -1l, (Replica.NodeNameToBaseUrl) dataProvider.getClusterStateProvider()));
       }
     }
 
@@ -108,7 +108,7 @@ public class ClusterStateMutator {
     if (message.getStr("fromApi") == null) {
       collectionProps.put("autoCreated", "true");
     }
-
+    collectionProps.put("id", 1l);
     DocCollection newCollection = new DocCollection(cName,
             slices, collectionProps, router, 0, false);
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
index 85dd0bd..d8ce923 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
@@ -78,7 +78,7 @@ public class CollectionMutator {
       if (shardParentNode != null)  {
         sliceProps.put("shard_parent_node", shardParentNode);
       }
-      collection = updateSlice(collectionName, collection, new Slice(shardId, replicas, sliceProps, collectionName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider()));
+      collection = updateSlice(collectionName, collection, new Slice(shardId, replicas, sliceProps, collectionName, collection.getId(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider()));
 
 
       // TODO - fix, no makePath (ensure every path part exists), async, single node
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
index 4441f75..438f7ea 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
@@ -63,13 +63,13 @@ public class NodeMutator {
             log.debug("Update replica state for {} to {}", replica, Replica.State.DOWN);
             Map<String, Object> props = replica.shallowCopy();
             props.put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
-            Replica newReplica = new Replica(replica.getName(), props, collection, slice.getName(), nodeNameToBaseUrl);
+            Replica newReplica = new Replica(replica.getName(), props, collection, replica.getCollectionId(), slice.getName(), nodeNameToBaseUrl);
             newReplicas.put(replica.getName(), newReplica);
             needToUpdateCollection = true;
           }
         }
 
-        Slice newSlice = new Slice(slice.getName(), newReplicas, slice.shallowCopy(),collection, nodeNameToBaseUrl);
+        Slice newSlice = new Slice(slice.getName(), newReplicas, slice.shallowCopy(), collection, docCollection.getId(), nodeNameToBaseUrl);
         slicesCopy.put(slice.getName(), newSlice);
       }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
index 21d6dba..7664960 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
@@ -61,7 +61,7 @@ public class ReplicaMutator {
 
     Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
     replicaProps.put(key, value);
-    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice(), replica.getBaseUrl());
+    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getCollectionId(), replica.getSlice(), replica.getBaseUrl());
   }
 
   protected Replica setProperty(Replica replica, String key, String value, String key2, String value2) {
@@ -74,7 +74,7 @@ public class ReplicaMutator {
     Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
     replicaProps.put(key, value);
     replicaProps.put(key2, value2);
-    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getCollectionId(), replica.getSlice(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
   }
 
   protected Replica unsetProperty(Replica replica, String key) {
@@ -83,7 +83,7 @@ public class ReplicaMutator {
     if (!replica.containsKey(key)) return replica;
     Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
     replicaProps.remove(key);
-    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getCollectionId(), replica.getSlice(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
   }
 
   protected Replica setLeader(Replica replica) {
@@ -162,7 +162,8 @@ public class ReplicaMutator {
         }
       }
     }
-    Slice newSlice = new Slice(sliceName, replicas, collection.getSlice(sliceName).shallowCopy(),collectionName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+    Slice newSlice = new Slice(sliceName, replicas, collection.getSlice(sliceName).shallowCopy(), collectionName,
+        replica.getCollectionId(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
     DocCollection newCollection = CollectionMutator.updateSlice(collectionName, collection,
         newSlice);
     return clusterState.copyWith(collectionName, newCollection);
@@ -278,14 +279,18 @@ public class ReplicaMutator {
 
 
     Slice slice = collection != null ?  collection.getSlice(sliceName) : null;
+    Long id = -1l;
 
     Map<String, Object> replicaProps = new LinkedHashMap<>(message.getProperties());
     if (slice != null) {
       Replica oldReplica = slice.getReplica(coreName);
+
       if (oldReplica != null) {
+        id = oldReplica.getCollectionId();
         if (oldReplica.containsKey(ZkStateReader.LEADER_PROP)) {
           replicaProps.put(ZkStateReader.LEADER_PROP, oldReplica.get(ZkStateReader.LEADER_PROP));
         }
+
         replicaProps.put(ZkStateReader.REPLICA_TYPE, oldReplica.getType().toString());
         // Move custom props over.
         for (Map.Entry<String, Object> ent : oldReplica.getProperties().entrySet()) {
@@ -321,7 +326,7 @@ public class ReplicaMutator {
     String shardParent = (String) replicaProps.remove(ZkStateReader.SHARD_PARENT_PROP);
 
 
-    Replica replica = new Replica(coreName, replicaProps, collectionName, sliceName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+    Replica replica = new Replica(coreName, replicaProps, collectionName, id, sliceName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
 
     log.debug("Will update state for replica: {}", replica);
 
@@ -336,7 +341,7 @@ public class ReplicaMutator {
       replicas = slice.getReplicasCopy();
 
     replicas.put(replica.getName(), replica);
-    slice = new Slice(sliceName, replicas, sliceProps, collectionName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+    slice = new Slice(sliceName, replicas, sliceProps, collectionName, collection.getId(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
 
     DocCollection newCollection = CollectionMutator.updateSlice(collectionName, collection, slice);
     log.debug("Collection is now: {}", newCollection);
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
index 3d41dc6..af2bcd1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
@@ -70,12 +70,12 @@ public class SliceMutator {
       coreName = Assign.buildSolrCoreName(collection, slice, Replica.Type.get(message.getStr(ZkStateReader.REPLICA_TYPE)));
     }
     Replica replica = new Replica(coreName,
-        Utils.makeNonNullMap("id", String.valueOf(collection.getHighestReplicaId() + 1), "collId", String.valueOf(collection.getId()),
+        Utils.makeNonNullMap("id", String.valueOf(collection.getHighestReplicaId()),
                     ZkStateReader.STATE_PROP, Replica.State.DOWN,
                     ZkStateReader.NODE_NAME_PROP, message.getStr(ZkStateReader.NODE_NAME_PROP),
                     ZkStateReader.NUM_SHARDS_PROP, message.getStr(ZkStateReader.NUM_SHARDS_PROP),
                     "shards", message.getStr("shards"),
-                    ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)), coll, slice, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+                    ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)), coll, collection.getId(), slice, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
 
     if (log.isDebugEnabled()) {
       log.debug("addReplica(ClusterState, ZkNodeProps) - end");
@@ -105,7 +105,7 @@ public class SliceMutator {
       if (replica != null) {
         Map<String, Replica> newReplicas = slice.getReplicasCopy();
         newReplicas.remove(coreName);
-        slice = new Slice(slice.getName(), newReplicas, slice.getProperties(), collection, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+        slice = new Slice(slice.getName(), newReplicas, slice.getProperties(), collection, coll.getId(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
       }
       newSlices.put(slice.getName(), slice);
     }
@@ -158,7 +158,7 @@ public class SliceMutator {
 
     Map<String, Object> newSliceProps = slice.shallowCopy();
     newSliceProps.put(Slice.REPLICAS, newReplicas);
-    slice = new Slice(slice.getName(), newReplicas, slice.getProperties(), collectionName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+    slice = new Slice(slice.getName(), newReplicas, slice.getProperties(), collectionName, coll.getId(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
 
     clusterState = clusterState.copyWith(collectionName, CollectionMutator.updateSlice(collectionName, coll, slice));
     if (log.isDebugEnabled()) log.debug("setShardLeader {} {}", sliceName, clusterState);
@@ -195,7 +195,7 @@ public class SliceMutator {
       props.put(ZkStateReader.STATE_PROP, message.getStr(key));
       // we need to use epoch time so that it's comparable across Overseer restarts
       props.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs()));
-      Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props, collectionName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+      Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props, collectionName, collection.getId(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
 
       slicesCopy.put(slice.getName(), newSlice);
     }
@@ -246,7 +246,7 @@ public class SliceMutator {
     Map<String, Object> props = slice.shallowCopy();
     props.put("routingRules", routingRules);
 
-    Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props, collectionName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+    Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props, collectionName, collection.getId(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
     return clusterState.copyWith( collectionName,
         CollectionMutator.updateSlice(collectionName, collection, newSlice));
   }
@@ -275,7 +275,7 @@ public class SliceMutator {
       routingRules.remove(routeKeyStr); // no rules left
       Map<String, Object> props = slice.shallowCopy();
       props.put("routingRules", routingRules);
-      Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props, collectionName, (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
+      Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props, collectionName, collection.getId(), (Replica.NodeNameToBaseUrl) cloudManager.getClusterStateProvider());
 
       if (log.isDebugEnabled()) {
         log.debug("removeRoutingRule(ClusterState, ZkNodeProps) - end");
@@ -305,7 +305,7 @@ public class SliceMutator {
     } else {
       replicasCopy.put(replica.getName(), replica);
     }
-    Slice newSlice = new Slice(slice.getName(), replicasCopy, slice.getProperties(), collection.getName(), nodeNameToBaseUrl);
+    Slice newSlice = new Slice(slice.getName(), replicasCopy, slice.getProperties(), collection.getName(), collection.getId(),  nodeNameToBaseUrl);
     if (log.isDebugEnabled()) {
       log.debug("Old Slice: {}", slice);
       log.debug("New Slice: {}", newSlice);
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 475ab1a..732330f 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
@@ -80,7 +80,7 @@ public class ZkStateWriter {
 
   protected final ReentrantLock ourLock = new ReentrantLock();
 
-  private final ActionThrottle throttle = new ActionThrottle("ZkStateWriter", Integer.getInteger("solr.zkstatewriter.throttle", 10), new TimeSource.NanoTimeSource(){
+  private final ActionThrottle throttle = new ActionThrottle("ZkStateWriter", Integer.getInteger("solr.zkstatewriter.throttle", 100), new TimeSource.NanoTimeSource(){
     public void sleep(long ms) throws InterruptedException {
       ourLock.newCondition().await(ms, TimeUnit.MILLISECONDS);
     }
@@ -475,12 +475,12 @@ public class ZkStateWriter {
               lastVersion.set(version);
               if (log.isDebugEnabled()) log.debug("Write state.json prevVersion={} bytes={} col={}", version, data.length, collection);
 
-              reader.getZkClient().setData(path, data, version, true);
+              reader.getZkClient().setData(path, data, version, true, false);
               trackVersions.put(collection.getName(), version + 1);
               if (dirtyStructure.contains(collection.getName())) {
                 if (log.isDebugEnabled()) log.debug("structure change in {}", collection.getName());
                 dirtyStructure.remove(collection.getName());
-                reader.getZkClient().setData(pathSCN, null, -1, true);
+                reader.getZkClient().setData(pathSCN, null, -1, true, false);
 
                 ZkNodeProps updates = stateUpdates.get(collection.getName());
                 if (updates != null) {
@@ -488,7 +488,7 @@ public class ZkStateWriter {
                  String stateUpdatesPath = ZkStateReader.getCollectionStateUpdatesPath(collection.getName());
                  if (log.isDebugEnabled()) log.debug("write state updates for collection {} {}", collection.getName(), updates);
                   try {
-                    reader.getZkClient().setData(stateUpdatesPath, Utils.toJSON(updates), -1, true);
+                    reader.getZkClient().setData(stateUpdatesPath, Utils.toJSON(updates), -1, true, false);
                   } catch (KeeperException.NoNodeException e) {
                     if (log.isDebugEnabled()) log.debug("No node found for " + stateUpdatesPath, e);
                     lastVersion.set(-1);
@@ -510,7 +510,7 @@ public class ZkStateWriter {
               //failedUpdates.put(collection.getName(), collection);
              // Stat estate = reader.getZkClient().exists(path, null);
               trackVersions.remove(collection.getName());
-              Stat stat = reader.getZkClient().exists(path, null);
+              Stat stat = reader.getZkClient().exists(path, null, false, false);
               log.info("Tried to update state.json ({}) with bad version {} \n {}", collection, version, stat != null ? stat.getVersion() : "null");
 
               if (!overseer.isClosed() && stat != null) {
@@ -567,7 +567,7 @@ public class ZkStateWriter {
     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);
+      reader.getZkClient().setData(stateUpdatesPath, Utils.toJSON(updates), -1, true, false);
     } catch (KeeperException.NoNodeException e) {
       if (log.isDebugEnabled()) log.debug("No node found for state.json", e);
       lastVersion.set(-1);
@@ -630,8 +630,8 @@ public class ZkStateWriter {
       stateUpdates.remove(collection);
       cs.getCollectionStates().remove(collection);
       trackVersions.remove(collection);
-      reader.getZkClient().delete(ZkStateReader.getCollectionSCNPath(collection), -1);
-      reader.getZkClient().delete(ZkStateReader.getCollectionStateUpdatesPath(collection), -1);
+      reader.getZkClient().deleteAsync(ZkStateReader.getCollectionSCNPath(collection), -1);
+      reader.getZkClient().deleteAsync(ZkStateReader.getCollectionStateUpdatesPath(collection), -1);
     } catch (Exception e) {
       log.error("", e);
     } finally {
diff --git a/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
index cea225b..4c6ced7 100644
--- a/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
@@ -154,6 +154,7 @@ public abstract class CachingDirectoryFactory extends DirectoryFactory {
       CacheValue cacheValue = byDirectoryCache.get(directory);
       if (cacheValue == null) {
         log.warn("done with an unknown directory, {}", directory);
+        org.apache.solr.common.util.IOUtils.closeQuietly(directory);
         return;
       }
       cacheValue.doneWithDir = true;
@@ -432,7 +433,10 @@ public abstract class CachingDirectoryFactory extends DirectoryFactory {
     synchronized (this) {
       CacheValue cacheValue = byDirectoryCache.get(directory);
       if (cacheValue == null) {
-        throw new IllegalArgumentException("Unknown directory: " + directory);
+        org.apache.solr.common.util.IOUtils.closeQuietly(directory);
+        log.warn("Unknown directory: " + directory
+            + " " + byDirectoryCache);
+        return;
       }
 
       cacheValue.refCnt++;
diff --git a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
index 42613b3..f8f5ac6 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
@@ -352,6 +352,10 @@ public class CoreDescriptor {
     return coreProperties.getProperty(prop, defVal);
   }
 
+  public Properties getCoreProperties() {
+    return coreProperties;
+  }
+
   /**
    * Returns all substitutable properties defined on this CoreDescriptor
    * @return all substitutable properties defined on this CoreDescriptor
diff --git a/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java b/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
index 1ecd6b3..e3497fa 100644
--- a/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
+++ b/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.core;
 
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
@@ -82,7 +83,8 @@ public class CorePropertiesLocator implements CoresLocator {
   private void writePropertiesFile(CoreDescriptor cd, Path propfile)  {
     Properties p = buildCoreProperties(cd);
     try {
-      FileUtils.createDirectories(propfile.getParent()); // Handling for symlinks.
+      File parentDir = propfile.getParent().toFile();
+      parentDir.mkdirs();
       try (Writer os = new OutputStreamWriter(Files.newOutputStream(propfile), StandardCharsets.UTF_8)) {
         p.store(os, "Written by CorePropertiesLocator");
       }
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index 9023a3c..15e8ee8 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -286,7 +286,7 @@ public class PluginBag<T> implements AutoCloseable {
    */
   void init(Map<String, T> defaults, SolrCore solrCore, Collection<PluginInfo> infos) {
     core = solrCore;
-    try (ParWork parWork = new ParWork(this, false, true)) {
+    try (ParWork parWork = new ParWork(this, false, false)) {
       for (PluginInfo info : infos) {
         parWork.collect("", new CreateAndPutRequestHandler(info));
       }
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index 26315eb..f5c5ec4 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -189,7 +189,6 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     super(loader, name, (InputSource) null, "/config/", substitutableProperties == null ? new Properties() : substitutableProperties);
 
     getOverlay();//just in case it is not initialized
-    getRequestParams();
     initLibs(loader, isConfigsetTrusted);
     luceneMatchVersion = SolrConfig.parseLuceneVersionString(getVal(loader.configXpathExpressions.luceneMatchVersionExp, ConfigXpathExpressions.luceneMatchVersionPath, true));
     log.info("Using Lucene MatchVersion: {}", luceneMatchVersion);
@@ -992,7 +991,7 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
 
   @Override
   public Properties getSubstituteProperties() {
-    Map<String, Object> p = getOverlay().getUserProps();
+    Map<String, Object> p = overlay.getUserProps();
     if (p == null || p.isEmpty()) return super.getSubstituteProperties();
     Properties result = new Properties(super.getSubstituteProperties());
     result.putAll(p);
@@ -1013,13 +1012,17 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
   }
 
   public RequestParams getRequestParams() {
+
     if (requestParams == null) {
-      return refreshRequestParams();
+      synchronized (this) {
+        if (requestParams == null) {
+          return refreshRequestParams();
+        }
+      }
     }
     return requestParams;
   }
 
-
   public RequestParams refreshRequestParams() {
     requestParams = RequestParams.getFreshRequestParams(getResourceLoader(), requestParams);
     if (log.isDebugEnabled()) {
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 faab326..39e8a49 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -32,7 +32,6 @@ import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.LockObtainFailedException;
-import org.apache.lucene.store.NativeFSLockFactory;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.RecoveryStrategy;
@@ -799,9 +798,6 @@ public final class SolrCore implements SolrInfoBean, Closeable {
       this.indexReaderFactory = indexReaderFactory;
     }
 
-    // protect via synchronized(SolrCore.class)
-    private static Set<String> dirs = new HashSet<>();
-
     /**
      * Returns <code>true</code> iff the index in the named directory is
      * currently locked.
@@ -820,39 +816,12 @@ public final class SolrCore implements SolrInfoBean, Closeable {
       }
     }
 
-    void initIndex ( boolean passOnPreviousState, boolean reload) throws IOException {
+    void initIndex (boolean passOnPreviousState, boolean reload) throws IOException {
       String indexDir = getNewIndexDir();
       boolean indexExists = getDirectoryFactory().exists(indexDir);
-      boolean firstTime;
-      synchronized (SolrCore.class) {
-        firstTime = dirs.add(getDirectoryFactory().normalize(indexDir));
-      }
 
       initIndexReaderFactory();
 
-      if (indexExists && firstTime && !passOnPreviousState) {
-        final String lockType = getSolrConfig().indexConfig.lockType;
-        Directory dir = directoryFactory.get(indexDir, DirContext.DEFAULT, lockType);
-        try {
-          if (isWriterLocked(dir)) {
-            log.error("{}Solr index directory '{}' is locked (lockType={}).  Throwing exception.", logid, indexDir, lockType);
-            throw new LockObtainFailedException("Index dir '" + indexDir + "' of core '" + name + "' is already locked. "
-                + "The most likely cause is another Solr server (or another solr core in this server) "
-                + "also configured to use this directory; other possible causes may be specific to lockType: " + lockType);
-          }
-        } finally {
-          directoryFactory.release(dir);
-        }
-      }
-
-      if (!indexExists) {
-        try {
-          NativeFSLockFactory.clearLockHeld(Paths.get(indexDir));
-        } catch (org.apache.lucene.store.AlreadyClosedException e) {
-
-        }
-      }
-
       // Create the index if it doesn't exist.
       if (!indexExists) {
         log.debug("{}Solr index directory '{}' doesn't exist. Creating new index...", logid, indexDir);
@@ -866,7 +835,9 @@ public final class SolrCore implements SolrInfoBean, Closeable {
         }
       }
 
-      cleanupOldIndexDirectories(reload);
+      ParWork.getRootSharedExecutor().submit(()->{
+        cleanupOldIndexDirectories(reload);
+      });
     }
 
     public static <T > T createInstance(String className, Class < T > cast, String msg, SolrCore core, ResourceLoader resourceLoader) {
@@ -1724,13 +1695,29 @@ public final class SolrCore implements SolrInfoBean, Closeable {
 
       if (count == 0) {
         try {
-          coreContainer.solrCoreExecutor.submit(() -> {
+          if (!coreContainer.solrCoreExecutor.isShutdown()) {
+            coreContainer.solrCoreExecutor.submit(() -> {
+              try {
+                doClose();
+              } catch (Exception e1) {
+                log.error("Exception closing SolrCore", e1);
+              }
+            });
+          } else if (!ParWork.getRootSharedExecutor().isShutdown()) {
+            ParWork.getRootSharedExecutor().submit(() -> {
+              try {
+                doClose();
+              } catch (Exception e1) {
+                log.error("Exception closing SolrCore", e1);
+              }
+            });
+          } else {
             try {
               doClose();
             } catch (Exception e1) {
               log.error("Exception closing SolrCore", e1);
             }
-          });
+          }
         } catch (RejectedExecutionException e) {
           try {
             doClose();
@@ -3422,7 +3409,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
 
     }
 
-    public static Runnable getConfListener (SolrCore core, ZkSolrResourceLoader zkSolrResourceLoader){
+    public static Runnable getConfListener(SolrCore core, ZkSolrResourceLoader zkSolrResourceLoader){
       final String coreName = core.getName();
       final CoreContainer cc = core.getCoreContainer();
       final String overlayPath = zkSolrResourceLoader.getConfigSetZkPath() + "/" + ConfigOverlay.RESOURCE_NAME;
diff --git a/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java b/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
index 58c08867..f9ebada 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrTinyBuilder.java
@@ -21,7 +21,6 @@ import net.sf.saxon.om.AttributeInfo;
 import net.sf.saxon.tree.tiny.TinyBuilder;
 import org.apache.solr.util.PropertiesUtil;
 
-import java.util.HashMap;
 import java.util.Properties;
 
 public class SolrTinyBuilder extends TinyBuilder {
diff --git a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
index 3f6036f..74adcad 100644
--- a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
+++ b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
@@ -19,11 +19,9 @@ package org.apache.solr.core;
 import net.sf.saxon.event.PipelineConfiguration;
 import net.sf.saxon.event.Sender;
 import net.sf.saxon.lib.ParseOptions;
-import net.sf.saxon.lib.Validation;
 import net.sf.saxon.om.NodeInfo;
 import net.sf.saxon.trans.XPathException;
 import net.sf.saxon.tree.tiny.TinyDocumentImpl;
-import org.apache.commons.collections.map.ReferenceMap;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
@@ -31,8 +29,6 @@ import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.StopWatch;
 import org.apache.solr.common.util.XMLErrorLogger;
 import org.apache.solr.util.DOMUtil;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Element;
@@ -50,7 +46,6 @@ import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -70,15 +65,6 @@ public class XmlConfigFile { // formerly simply "Config"
 
   public static final XMLErrorLogger xmllog = new XMLErrorLogger(log);
 
-  private static Map CONFIG_CACHE = new ReferenceMap(ReferenceMap.HARD, ReferenceMap.WEAK) {
-    {
-      purgeBeforeRead();
-      purgeBeforeWrite();
-    }
-  };
-
-  private static Map<String, ZkSolrResourceLoader.ZkByteArrayInputStream> SYNC_CONFIG_CACHE = Collections.synchronizedMap(CONFIG_CACHE);
-
   protected final String prefix;
   private final String name;
   protected final SolrResourceLoader loader;
@@ -135,76 +121,22 @@ public class XmlConfigFile { // formerly simply "Config"
 
       InputStream in = null;
 
-      boolean usedCached = false;
-      if (loader instanceof  ZkSolrResourceLoader) {
-        ZkSolrResourceLoader.ZkByteArrayInputStream cached = SYNC_CONFIG_CACHE.get(name);
-        if (cached != null) {
-          Stat checkStat;
-          try {
-            checkStat = ((ZkSolrResourceLoader) loader).getZkClient().exists(loader.getConfigDir() + "/" + name, null);
-          } catch (KeeperException | InterruptedException e) {
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-          }
-          if (checkStat != null && checkStat.getVersion() <= cached.getStat().getVersion()) {
-            in = new ZkSolrResourceLoader.ZkByteArrayInputStream(cached.getBytes(), cached.getStat());
-            zkVersion = cached.getStat().getVersion();
-            usedCached = true;
-          }
-        }
-      }
-      if (in == null) {
-        in = loader.openResource(name);
-      }
+      in = loader.openResource(name);
 
-      if (!usedCached) {
-        if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
-          SYNC_CONFIG_CACHE.put(name, (ZkSolrResourceLoader.ZkByteArrayInputStream) in);
-          zkVersion = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
-          if (log.isDebugEnabled()) {
-            log.debug("loaded config {} with version {} ", name, zkVersion);
-          }
+      if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
+
+        zkVersion = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
+        if (log.isDebugEnabled()) {
+          log.debug("loaded config {} with version {} ", name, zkVersion);
         }
       }
+
       is = new InputSource(in);
-   //   is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
+      //   is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
     }
 
     try {
-      SAXSource source = new SAXSource(is);
-      source.setXMLReader(loader.getXmlReader());
-      PipelineConfiguration plc = getResourceLoader().getConf().makePipelineConfiguration();
-      //      if (is.getSystemId() != null) {
-      //     plc.setURIResolver(loader.getSysIdResolver().asURIResolver());
-      //      }
-
-      TinyDocumentImpl docTree;
-      SolrTinyBuilder builder = new SolrTinyBuilder(plc, substituteProps);
-      try {
-        //builder.setStatistics(conf2.getTreeStatistics().SOURCE_DOCUMENT_STATISTICS);
-        builder.open();
-        ParseOptions po = plc.getParseOptions();
-        if (is.getSystemId() != null) {
-          po.setEntityResolver(loader.getSysIdResolver());
-        } else {
-          po.setEntityResolver(null);
-        }
-        // Set via conf already
-        //   po.setXIncludeAware(true);
-        //  po.setCheckEntityReferences(false);
-        // po.setExpandAttributeDefaults(false);
-        po.setDTDValidationMode(Validation.STRIP);
-        po.setPleaseCloseAfterUse(true);
-        Sender.send(source, builder, po);
-        docTree = (TinyDocumentImpl) builder.getCurrentRoot();
-      } catch (Exception e) {
-        log.error("Exception handling xml doc", e);
-        throw e;
-      }  finally {
-        //builder.close();
-        //builder.reset();
-      }
-
-      this.tree = docTree;
+      this.tree = parseXml(loader, is, substituteProps);
 
       this.substituteProperties = substituteProps;
     } catch (XPathException e) {
@@ -217,7 +149,43 @@ public class XmlConfigFile { // formerly simply "Config"
 
   }
 
-    /*
+  public static TinyDocumentImpl parseXml(SolrResourceLoader loader, InputSource is, Properties substituteProps) throws XPathException {
+    SAXSource source = new SAXSource(is);
+    source.setXMLReader(loader.getXmlReader());
+    PipelineConfiguration plc = loader.getConf().makePipelineConfiguration();
+    //      if (is.getSystemId() != null) {
+    //     plc.setURIResolver(loader.getSysIdResolver().asURIResolver());
+    //      }
+    TinyDocumentImpl docTree;
+    SolrTinyBuilder builder = new SolrTinyBuilder(plc, substituteProps);
+    try {
+      //builder.setStatistics(conf2.getTreeStatistics().SOURCE_DOCUMENT_STATISTICS);
+      builder.open();
+      ParseOptions po = plc.getParseOptions();
+      if (is.getSystemId() != null) {
+        po.setEntityResolver(loader.getSysIdResolver());
+      } else {
+        po.setEntityResolver(null);
+      }
+      // Set via conf already
+      //   po.setXIncludeAware(true);
+      //  po.setCheckEntityReferences(false);
+      // po.setExpandAttributeDefaults(false);
+      po.setPleaseCloseAfterUse(true);
+      Sender.send(source, builder, po);
+      docTree = (TinyDocumentImpl) builder.getCurrentRoot();
+    } catch (Exception e) {
+      log.error("Exception handling xml doc", e);
+      throw e;
+    }  finally {
+      //builder.close();
+      //builder.reset();
+    }
+
+    return docTree;
+  }
+
+  /*
      * Assert that assertCondition is true.
      * If not, prints reason as log warning.
      * If failCondition is true, then throw exception instead of warning
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
index d0fd94b..795dff9 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -61,7 +61,6 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.ConfigOverlay;
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/RequestSyncShardOp.java b/solr/core/src/java/org/apache/solr/handler/admin/RequestSyncShardOp.java
index fb2422c..1374559 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/RequestSyncShardOp.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/RequestSyncShardOp.java
@@ -46,6 +46,7 @@ class RequestSyncShardOp implements CoreAdminHandler.CoreAdminOp {
     log.info("I have been requested to sync up my shard");
 
     String cname = params.required().get(CoreAdminParams.CORE);
+    String id = params.required().get("id");
 
     ZkController zkController = it.handler.coreContainer.getZkController();
     if (zkController == null) {
@@ -61,10 +62,11 @@ class RequestSyncShardOp implements CoreAdminHandler.CoreAdminOp {
         Map<String, Object> props = new HashMap<>();
         props.put(ZkStateReader.CORE_NAME_PROP, cname);
         props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());
+        props.put("id", core.getCoreDescriptor().getCoreProperty("id", "-1"));
         String collection = params.get("collection");
         String shard = params.get("shard");
 
-        Replica replica = new Replica(cname, props, collection, shard, zkController.zkStateReader);
+        Replica replica = new Replica(cname, props, collection, Long.parseLong(core.getCoreDescriptor().getCoreProperty("collId", "-1")), shard, zkController.zkStateReader);
 
         boolean success = syncStrategy.sync(zkController, core, replica, true).isSuccess();
         // solrcloud_debug
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
index fe30f00..5f108ee 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
@@ -641,8 +641,7 @@ public class RealTimeGetComponent extends SearchComponent
 
         SolrDocumentFetcher docFetcher = searcher.getDocFetcher();
         if (onlyTheseNonStoredDVs != null) {
-          sid = SolrInputDocument.THREAD_LOCAL_SolrInputDocument.get();
-          sid.clear();
+          sid = new SolrInputDocument();
         } else {
           Document luceneDocument = docFetcher.doc(docid);
           sid = toSolrInputDocument(luceneDocument, schema);
diff --git a/solr/core/src/java/org/apache/solr/handler/loader/CSVLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/CSVLoader.java
index d04ca82..377d507 100644
--- a/solr/core/src/java/org/apache/solr/handler/loader/CSVLoader.java
+++ b/solr/core/src/java/org/apache/solr/handler/loader/CSVLoader.java
@@ -51,7 +51,7 @@ class SingleThreadedCSVLoader extends CSVLoaderBase {
     templateAdd.overwrite = overwrite;
     templateAdd.commitWithin = commitWithin;
     SolrInputDocument doc = SolrInputDocument.THREAD_LOCAL_SolrInputDocument.get();
-    doc.clear();
     doAdd(line, vals, doc, templateAdd);
+    doc.clear();
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
index 1163d41..c00ead4 100644
--- a/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
+++ b/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
@@ -233,9 +233,7 @@ public class JsonLoader extends ContentStreamLoader {
             changeChildDoc(copy);
             docs.add(copy);
           } else {
-            AddUpdateCommand cmd = AddUpdateCommand.THREAD_LOCAL_AddUpdateCommand.get();
-            cmd.clear();
-            cmd.setReq(req);
+            AddUpdateCommand cmd = new AddUpdateCommand(req);
             cmd.commitWithin = commitWithin;
             cmd.overwrite = overwrite;
             cmd.solrDoc = buildDoc(copy);
@@ -488,9 +486,7 @@ public class JsonLoader extends ContentStreamLoader {
 
     void handleAdds() throws IOException {
       while (true) {
-        AddUpdateCommand cmd = AddUpdateCommand.THREAD_LOCAL_AddUpdateCommand.get();
-        cmd.clear();
-        cmd.setReq(req);
+        AddUpdateCommand cmd = new AddUpdateCommand(req);
         cmd.commitWithin = commitWithin;
         cmd.overwrite = overwrite;
 
diff --git a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
index e7aa2ee..f0c860d 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
@@ -27,7 +27,6 @@ import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.handler.component.ResponseBuilder;
@@ -84,8 +83,6 @@ public class SolrRequestInfo {
     } finally {
       threadLocal.remove();
       AddUpdateCommand.THREAD_LOCAL_AddUpdateCommand.get().clear();
-      AddUpdateCommand.THREAD_LOCAL_AddUpdateCommand_TLOG.get().clear();
-      SolrInputDocument.THREAD_LOCAL_SolrInputDocument.get().clear();
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java b/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
index b2d20e7..03ee98a 100644
--- a/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
+++ b/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
@@ -21,30 +21,34 @@ import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 
+import net.sf.saxon.om.NodeInfo;
+import net.sf.saxon.trans.XPathException;
+import net.sf.saxon.tree.tiny.TinyDocumentImpl;
+import net.sf.saxon.tree.tiny.TinyElementImpl;
 import org.apache.commons.lang3.math.NumberUtils;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.EnumFieldSource;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.EnumFieldValue;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.core.XmlConfigFile;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
-import org.apache.solr.util.SafeXMLParsing;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
+import org.xml.sax.InputSource;
 
 /***
  * Abstract Field type for support of string values with custom sort order.
@@ -108,11 +112,24 @@ public abstract class AbstractEnumField extends PrimitiveFieldType {
       final SolrResourceLoader loader = schema.getResourceLoader(); 
       try {
         log.debug("Reloading enums config file from {}", enumsConfigFile);
-        Document doc = SafeXMLParsing.parseConfigXML(log, loader, enumsConfigFile);
+        InputStream in = null;
+
+        in = loader.openResource(enumsConfigFile);
+
+        if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
+
+          int zkVersion = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
+          if (log.isDebugEnabled()) {
+            log.debug("loaded config {} with version {} ", enumsConfigFile, zkVersion);
+          }
+        }
+
+        InputSource is = new InputSource(in);
+        TinyDocumentImpl doc = XmlConfigFile.parseXml(loader, is, null);
         final XPath xpath = schema.loader.getXPath();
         final String xpathStr = String.format(Locale.ROOT, "/enumsConfig/enum[@name='%s']", enumName);
-        final NodeList nodes = (NodeList) xpath.evaluate(xpathStr, doc, XPathConstants.NODESET);
-        final int nodesLength = nodes.getLength();
+        ArrayList<NodeInfo> nodes = (ArrayList<NodeInfo>) xpath.evaluate(xpathStr, doc, XPathConstants.NODESET);
+        final int nodesLength = nodes.size();
         if (nodesLength == 0) {
           String exceptionMessage = String.format
             (Locale.ENGLISH, "%s: No enum configuration found for enum '%s' in %s.",
@@ -123,11 +140,11 @@ public abstract class AbstractEnumField extends PrimitiveFieldType {
           log.warn("{}: More than one enum configuration found for enum '{}' in {}. The last one was taken."
               , ftName, enumName, enumsConfigFile);
         }
-        final Node enumNode = nodes.item(nodesLength - 1);
-        final NodeList valueNodes = (NodeList) xpath.evaluate("value", enumNode, XPathConstants.NODESET);
-        for (int i = 0; i < valueNodes.getLength(); i++) {
-          final Node valueNode = valueNodes.item(i);
-          final String valueStr = valueNode.getTextContent();
+        TinyElementImpl enumNode = (TinyElementImpl) nodes.get(nodesLength - 1);
+        final ArrayList<NodeInfo>  valueNodes = (ArrayList<NodeInfo>) xpath.evaluate("value", enumNode, XPathConstants.NODESET);
+        for (int i = 0; i < valueNodes.size(); i++) {
+          NodeInfo valueNode = valueNodes.get(i);
+          final String valueStr = valueNode.getStringValue();
           if ((valueStr == null) || (valueStr.length() == 0)) {
             final String exceptionMessage = String.format
               (Locale.ENGLISH, "%s: A value was defined with an no value in enum '%s' in %s.",
@@ -143,7 +160,7 @@ public abstract class AbstractEnumField extends PrimitiveFieldType {
           enumIntToStringMap.put(i, valueStr);
           enumStringToIntMap.put(valueStr, i);
         }
-      } catch (IOException | SAXException | XPathExpressionException e) {
+      } catch (IOException | XPathExpressionException | XPathException e) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
                                 ftName + ": Error while parsing enums config.", e);
       }
diff --git a/solr/core/src/java/org/apache/solr/servlet/DirectSolrConnection.java b/solr/core/src/java/org/apache/solr/servlet/DirectSolrConnection.java
index 7237a0f..216551c 100644
--- a/solr/core/src/java/org/apache/solr/servlet/DirectSolrConnection.java
+++ b/solr/core/src/java/org/apache/solr/servlet/DirectSolrConnection.java
@@ -151,4 +151,8 @@ public class DirectSolrConnection implements Closeable
   public void close() {
     core.close();
   }
+
+  public SolrCore getCore() {
+    return core;
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
index a7aca6e..c9ef32f 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
@@ -24,6 +24,8 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.stream.Collectors;
 
@@ -306,7 +308,7 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
     public final String fieldTypeName;
     public final Collection<String> valueClassNames;
     public final Collection<CopyFieldDef> copyFieldDefs;
-    public volatile List<Class<?>> valueClasses;
+    public final Set<Class<?>> valueClasses = ConcurrentHashMap.newKeySet(32);
     public final Boolean isDefault;
 
     public TypeMapping(String fieldTypeName, Collection<String> valueClassNames, boolean isDefault,
@@ -324,15 +326,14 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
       if (null == schema.getFieldTypeByName(fieldTypeName, schema.getFieldTypes())) {
         throw new SolrException(SERVER_ERROR, "fieldType '" + fieldTypeName + "' not found in the schema");
       }
-      valueClasses = Collections.synchronizedList(new ArrayList<>());
-      synchronized (valueClasses) {
-        for (String valueClassName : valueClassNames) {
-          try {
-            valueClasses.add(loader.loadClass(valueClassName));
-          } catch (ClassNotFoundException e) {
-            throw new SolrException(SERVER_ERROR, "valueClass '" + valueClassName + "' not found for fieldType '" + fieldTypeName + "'");
-          }
+
+      for (String valueClassName : valueClassNames) {
+        try {
+          valueClasses.add(loader.loadClass(valueClassName));
+        } catch (ClassNotFoundException e) {
+          throw new SolrException(SERVER_ERROR, "valueClass '" + valueClassName + "' not found for fieldType '" + fieldTypeName + "'");
         }
+
       }
     }
 
@@ -561,11 +562,9 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
     private TypeMapping mapValueClassesToFieldType(List<SolrInputField> fields) {
       NEXT_TYPE_MAPPING: for (TypeMapping typeMapping : typeMappings) {
         for (SolrInputField field : fields) {
-          //We do a assert and a null check because even after SOLR-12710 is addressed
-          //older SolrJ versions can send null values causing an NPE
-          assert field.getValues() != null;
           if (field.getValues() != null) {
-            NEXT_FIELD_VALUE: for (Object fieldValue : field.getValues()) {
+            NEXT_FIELD_VALUE:
+            for (Object fieldValue : field.getValues()) {
               for (Class<?> valueClass : typeMapping.valueClasses) {
                 if (valueClass.isInstance(fieldValue)) {
                   continue NEXT_FIELD_VALUE;
diff --git a/solr/core/src/java/org/apache/solr/util/FileUtils.java b/solr/core/src/java/org/apache/solr/util/FileUtils.java
index e6956f4..b493bf5 100644
--- a/solr/core/src/java/org/apache/solr/util/FileUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/FileUtils.java
@@ -106,7 +106,7 @@ public class FileUtils {
     if (Files.exists(path) && Files.isSymbolicLink(path)) {
       Path real = path.toRealPath();
       if (Files.isDirectory(real)) return real;
-      throw new FileExistsException("Tried to create a directory at to an existing non-directory symlink: " + path.toString());
+      throw new FileExistsException("Tried to create a directory to an existing non-directory symlink: " + path.toString());
     }
     return Files.createDirectories(path);
   }
diff --git a/solr/core/src/java/org/apache/solr/util/SafeXMLParsing.java b/solr/core/src/java/org/apache/solr/util/SafeXMLParsing.java
index f9db583..82563a7 100644
--- a/solr/core/src/java/org/apache/solr/util/SafeXMLParsing.java
+++ b/solr/core/src/java/org/apache/solr/util/SafeXMLParsing.java
@@ -21,7 +21,6 @@ import org.apache.solr.common.EmptyEntityResolver;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.common.util.XMLErrorLogger;
-import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.rest.schema.FieldTypeXmlAdapter;
 import org.slf4j.Logger;
 import org.w3c.dom.Document;
diff --git a/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java b/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
index e82a5ea..964fa4a 100644
--- a/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
+++ b/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
@@ -16,19 +16,6 @@
  */
 package org.apache.solr;
 
-import javax.xml.parsers.DocumentBuilder;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Metric;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -39,8 +26,6 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.TimeOut;
-import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.metrics.SolrMetricManager;
@@ -59,6 +44,17 @@ import org.apache.solr.search.DocList;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import javax.xml.parsers.DocumentBuilder;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Tests some basic functionality of Solr while demonstrating good
  * Best Practices for using SolrTestCaseJ4
@@ -103,7 +99,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     System.out.println("ret=" + ret + " time="+ (end-start));
   }
   ***/
-  
+
   @Test
   public void testIgnoredFields() throws Exception {
     lrf.args.put(CommonParams.VERSION,"2.2");
@@ -111,7 +107,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
             adoc("id", "42", "foo_ignored", "blah blah"));
     assertU("commit",
             commit());
-    
+
     // :TODO: the behavior of querying on an unindexed field should be better specified in the future.
     assertQ("query with ignored field",
             req("bar_ignored:yo id:42")
@@ -119,7 +115,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
             ,"//str[@name='id'][.='42']"
             );
   }
-  
+
   @Test
   public void testSomeStuff() throws Exception {
     clearIndex();
@@ -246,7 +242,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     assertQ(req("id:[100 TO 110]")
             ,"//*[@numFound='0']"
             );
-    
+
     assertU(h.simpleTag("rollback"));
     assertU(commit());
     core.close();
@@ -259,7 +255,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
    */
   public void testNonTrivialDeleteByQuery() throws Exception {
     clearIndex();
-    
+
     // setup
     assertU( add(doc("id","101", "text", "red apple" )) );
     assertU( add(doc("id","102", "text", "purple grape" )) );
@@ -457,7 +453,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
           msg3.contains(BAD_VALUE));
     }
   }
-  
+
   @Test
   public void testRequestHandlerBaseException() {
     final String tmp = "BOO! ignore_exception";
@@ -477,7 +473,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
                         req,
                         rsp);
     assertNotNull("should have found an exception", rsp.getException());
-    req.close();                    
+    req.close();
   }
 
   @Test
@@ -496,9 +492,9 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
   @Test
   public void testDocBoost() throws Exception {
     String res = h.update("<add>" + "<doc><field name=\"id\">1</field>"+
-                                          "<field name=\"text\">hello</field></doc>" + 
+                                          "<field name=\"text\">hello</field></doc>" +
                           "<doc boost=\"2.0\"><field name=\"id\">2</field>" +
-                                          "<field name=\"text\">hello</field></doc>" + 
+                                          "<field name=\"text\">hello</field></doc>" +
                           "</add>");
 
     // assertEquals("<result status=\"0\"></result>", res);
@@ -515,9 +511,9 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
   @Test
   public void testFieldBoost() throws Exception {
     String res = h.update("<add>" + "<doc><field name=\"id\">1</field>"+
-                                      "<field name=\"text\">hello</field></doc>" + 
+                                      "<field name=\"text\">hello</field></doc>" +
                                     "<doc><field name=\"id\">2</field>" +
-                                      "<field boost=\"2.0\" name=\"text\">hello</field></doc>" + 
+                                      "<field boost=\"2.0\" name=\"text\">hello</field></doc>" +
                           "</add>");
 
     // assertEquals("<result status=\"0\"></result>", res);
@@ -585,7 +581,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
 
   @Test
   public void testTermVectorFields() {
-    
+
     IndexSchema ischema = IndexSchemaFactory.buildIndexSchema(getSchemaFile(), solrConfig);
     SchemaField f; // Solr field type
     IndexableField luf; // Lucene field
@@ -698,7 +694,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
             ,"*[count(//doc)=3]"
             ,"//date[@name='timestamp']"
             );
-    
+
     assertQ("2 docs should have the default for multiDefault",
             req("multiDefault:muLti-Default")
             ,"*[count(//doc)=2]"
@@ -717,7 +713,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
             req("intDefault:[3 TO 5]")
             ,"*[count(//doc)=1]"
             );
-    
+
   }
 
   @Test
@@ -778,7 +774,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
       sb.append((char)(65 + i%26));
     }
     return new String(sb);
-  }   
+  }
 
   @Test
   public void testNotLazyField() throws IOException {
@@ -789,7 +785,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
 
     assertU(commit());
     SolrCore core = h.getCore();
-   
+
     SolrQueryRequest req = req("q", "id:7777", "fl", "id,title,test_hlt");
     SolrQueryResponse rsp = new SolrQueryResponse();
     core.execute(core.getRequestHandler(req.getParams().get(CommonParams.QT)), req, rsp);
@@ -814,7 +810,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
 
     assertU(commit());
     SolrCore core = h.getCore();
-    
+
     // initial request
     SolrQueryRequest req = req("q", "id:7777", "fl", "id,title");
     SolrQueryResponse rsp = new SolrQueryResponse();
@@ -822,7 +818,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     core.close();
 
     DocList dl = ((ResultContext) rsp.getResponse()).getDocList();
-    DocIterator di = dl.iterator();    
+    DocIterator di = dl.iterator();
     Document d1 = req.getSearcher().doc(di.nextDoc());
     IndexableField[] values1 = null;
 
@@ -844,18 +840,18 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     core.execute(core.getRequestHandler(req.getParams().get(CommonParams.QT)), req, rsp);
 
     dl = ((ResultContext) rsp.getResponse()).getDocList();
-    di = dl.iterator();    
+    di = dl.iterator();
     Document d2 = req.getSearcher().doc(di.nextDoc());
     // ensure same doc, same lazy field now
     assertTrue("Doc was not cached", d1 == d2);
     IndexableField[] values2 = d2.getFields("test_hlt");
     assertEquals(values1.length, values2.length);
     for (int i = 0; i < values1.length; i++) {
-      assertSame("LazyField wasn't reused", 
+      assertSame("LazyField wasn't reused",
                  values1[i], values2[i]);
       LazyDocument.LazyField f = (LazyDocument.LazyField) values1[i];
       // still not a real boy, no response writer in play
-      assertFalse(f.hasBeenLoaded()); 
+      assertFalse(f.hasBeenLoaded());
     }
 
     assertNotNull(values2[0].stringValue()); // actuallize one value
@@ -866,8 +862,8 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     }
 
     req.close();
-  } 
-            
+  }
+
 
   /** @see org.apache.solr.util.DateMathParserTest */
   @Test
@@ -889,7 +885,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     assertU(adoc("id", "5",  "bday", "NOW+30MINUTES"));
     assertU(adoc("id", "6",  "bday", "NOW+2YEARS"));
     assertU(commit());
-    
+
     // a ridiculoulsy long date math expression that's still equivalent to july4
     final StringBuilder july4Long = new StringBuilder(july4);
     final int iters = SolrTestUtil.atLeast(10);
@@ -899,14 +895,14 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     }
 
     // term queries using date math (all of these should match doc#1)
-    for (String q : 
+    for (String q :
            new String[] {
              "bday:1976-07-04T12\\:08\\:56.45Z/SECOND+235MILLIS",
              "bday:1976-07-04T12\\:08\\:56.123Z/MINUTE+56SECONDS+235MILLIS",
              "bday:\"1976-07-04T12:08:56.45Z/SECOND+235MILLIS\"",
              "bday:\"1976-07-04T12:08:56.123Z/MINUTE+56SECONDS+235MILLIS\"",
              "{!term f=bday}1976-07-04T12:08:56.45Z/SECOND+235MILLIS",
-             "{!term f=bday}1976-07-04T12:08:56.123Z/MINUTE+56SECONDS+235MILLIS",             
+             "{!term f=bday}1976-07-04T12:08:56.123Z/MINUTE+56SECONDS+235MILLIS",
              "{!term f=bday}"+july4,
              "{!term f=bday}"+july4Long,
              "bday:\"" + july4Long + "\""
@@ -930,7 +926,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     assertQ("check math on absolute date#4",
             req("q", "bday:["+july4+"/MINUTE+1MINUTE TO *]"),
             "*[count(//doc)=5]");
-    
+
     assertQ("check count for before now",
             req("q", "bday:[* TO NOW]"), "*[count(//doc)=4]");
 
@@ -945,13 +941,13 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
 
     assertQ("check count for near stuff",
             req("q", "bday:[NOW-1MONTH TO NOW+2HOURS]"), "*[count(//doc)=4]");
-    
+
     assertQ("check counts using fixed NOW",
             req("q", "bday:[NOW/DAY TO NOW/DAY+1DAY]",
                 "NOW", "205369736000" // 1976-07-04T23:08:56.235Z
                 ),
             "*[count(//doc)=1]");
-                
+
     assertQ("check counts using fixed NOW and TZ rounding",
             req("q", "bday:[NOW/DAY TO NOW/DAY+1DAY]",
                 "TZ", "GMT+01",
@@ -974,7 +970,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
 //            req("q", "id:99"),
 //            "//date[@name='bday'][.='1999-01-01T12:34:56.9Z']");
 //  }
-  
+
   @Test
   public void testPatternReplaceFilter() {
 
@@ -983,11 +979,11 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
     assertU(adoc("id", "2",
                  "patternreplacefilt", "  What's Up Doc?"));
     assertU(commit());
- 
+
     assertQ("don't find Up",
             req("q", "patternreplacefilt:Up"),
             "*[count(//doc)=0]");
-    
+
     assertQ("find doc",
             req("q", "patternreplacefilt:__What_s_Up_Doc_"),
             "*[count(//doc)=1]");
@@ -1025,7 +1021,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
                  e.getMessage().contains(f));
     }
   }
-  
+
 //   /** this doesn't work, but if it did, this is how we'd test it. */
 //   public void testOverwriteFalse() {
 
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 bd2cbbf..2d1fe5b 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
@@ -51,7 +51,6 @@ public class TestEmbeddedSolrServerSchemaAPI extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void initClass() throws Exception {
-    assertNull("no system props clash please", System.getProperty("managed.schema.mutable"));
     System.setProperty("managed.schema.mutable", ""+
       random().nextBoolean()
     );
diff --git a/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java b/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java
index 7000753..cf94936 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java
@@ -118,6 +118,7 @@ public class ClusterStateMockUtil {
     Map<String,Object> collectionProps = new HashMap<>();
     collectionProps.put(ZkStateReader.MAX_SHARDS_PER_NODE, Integer.toString(maxShardsPerNode));
     collectionProps.put(ZkStateReader.REPLICATION_FACTOR, Integer.toString(replicationFactor));
+    collectionProps.put("id",  -1l);
     Map<String,DocCollection> collectionStates = new HashMap<>();
     DocCollection docCollection = null;
     String collName = null;
@@ -137,7 +138,7 @@ public class ClusterStateMockUtil {
         case "s":
           replicas = new HashMap<>();
           if(collName == null) collName = "collection" + (collectionStates.size() + 1);
-          slice = new Slice(sliceName = "slice" + (slices.size() + 1), replicas, null,  collName, nodeName -> "http://" + nodeName);
+          slice = new Slice(sliceName = "slice" + (slices.size() + 1), replicas, null,  collName, -1l, nodeName -> "http://" + nodeName);
           slices.put(slice.getName(), slice);
 
           // hack alert: the DocCollection constructor copies over active slices to its active slice map in the constructor
@@ -172,11 +173,11 @@ public class ClusterStateMockUtil {
             replicaPropMap.put(Slice.LEADER, "true");
           }
           replicaPropMap.put("id", "1");
-          replica = new Replica(replicaName, replicaPropMap, collName, sliceName, nodeName -> "http://" + nodeName);
+          replica = new Replica(replicaName, replicaPropMap, collName, -1l, sliceName, nodeName -> "http://" + nodeName);
           replicas.put(replica.getName(), replica);
 
           // hack alert: re-create slice with existing data and new replicas map so that it updates its internal leader attribute
-          slice = new Slice(slice.getName(), replicas, null, collName, nodeName -> "http://" + nodeName);
+          slice = new Slice(slice.getName(), replicas, null, collName,-1l, nodeName -> "http://" + nodeName);
           slices.put(slice.getName(), slice);
           // we don't need to update doc collection again because we aren't adding a new slice or changing its state
           break;
diff --git a/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java b/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java
index 989be85..c9cf3d8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java
@@ -47,14 +47,17 @@ public class ClusterStateTest extends SolrTestCaseJ4 {
     props.put("id", "1");
     props.put("prop1", "value");
     props.put("prop2", "value2");
-    Replica replica = new Replica("node1", props, "collection1", "shard1", nodeName -> "http://" + nodeName);
+    Replica replica = new Replica("node1", props, "collection1", -1l,"shard1", nodeName -> "http://" + nodeName);
     sliceToProps.put("node1", replica);
-    Slice slice = new Slice("shard1", sliceToProps, null, "collection1", nodeName -> "http://" + nodeName);
+    Slice slice = new Slice("shard1", sliceToProps, null, "collection1",-1l, nodeName -> "http://" + nodeName);
     slices.put("shard1", slice);
-    Slice slice2 = new Slice("shard2", sliceToProps, null, "collection1", nodeName -> "http://" + nodeName);
+    Slice slice2 = new Slice("shard2", sliceToProps, null, "collection1",-1l, nodeName -> "http://" + nodeName);
     slices.put("shard2", slice2);
-    collectionStates.put("collection1", new DocCollection("collection1", slices, null, DocRouter.DEFAULT));
-    collectionStates.put("collection2", new DocCollection("collection2", slices, null, DocRouter.DEFAULT));
+
+    Map<String, Object> cprops = new HashMap<>();
+    cprops.put("id", -1l);
+    collectionStates.put("collection1", new DocCollection("collection1", slices, cprops, DocRouter.DEFAULT));
+    collectionStates.put("collection2", new DocCollection("collection2", slices, cprops, DocRouter.DEFAULT));
 
     ClusterState clusterState = ClusterState.getRefCS(collectionStates, -1);
     byte[] bytes = Utils.toJSON(clusterState);
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 2ccdab5..e74bb79 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -241,10 +241,10 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
   }
 
   @Test
-  //@Ignore // MRM TODO: - testing large numbers
+  @Ignore // MRM TODO: - testing large numbers
   public void testCreateAndDeleteCollection() throws Exception {
     String collectionName = "solrj_test";
-    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 4, 4) // 24 * 24 = 576
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 36, 36) // 24 * 24 = 576
             .process(cluster.getSolrClient());
 
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
index a9ca835..0b5d63c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
@@ -162,7 +162,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
       props.put(ZkStateReader.NODE_NAME_PROP,  Integer.toString(nodeNumber));
       props.put(ZkStateReader.CORE_NAME_PROP, "");
 
-      replica = new Replica("", props, "", shard, zkStateReader);
+      replica = new Replica("", props, "", -1l, shard, zkStateReader);
 
       this.es = es;
       if (this.es == null) {
@@ -278,7 +278,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
         ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(
             ZkNodeProps.load(data));
         // MRM TODO:
-        Replica replica = new Replica("", leaderProps.getNodeProps().getProperties(), collection, slice, zkStateReader);
+        Replica replica = new Replica("", leaderProps.getNodeProps().getProperties(), collection, -1l, slice, zkStateReader);
 
         return replica.getCoreUrl();
       } catch (NoNodeException | SessionExpiredException e) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index 4aed7da..1d29be0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -68,6 +68,7 @@ import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
 import static org.apache.solr.cloud.AbstractDistribZkTestBase.verifyReplicaStatus;
+import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.anyBoolean;
 import static org.mockito.Mockito.anyInt;
 import static org.mockito.Mockito.anyString;
@@ -229,7 +230,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
               ZkStateReader.SHARD_ID_PROP, shardId,
               ZkStateReader.COLLECTION_PROP, collection);
           LeaderElector elector = new LeaderElector(overseer.getZkController());
-          Replica replica = new Replica(coreName, props.getProperties(), collection, shardId, zkStateReader);
+          Replica replica = new Replica(coreName, props.getProperties(), collection, -1l, shardId, zkStateReader);
           ShardLeaderElectionContextBase ctx = new ShardLeaderElectionContextBase(
               nodeName + "_" + coreName, shardId, collection, replica, null,
               zkStateReader.getZkClient());
@@ -904,7 +905,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
 
           Replica leaderProps;
           try {
-            leaderProps = zkController.getLeaderProps(COLLECTION, "shard1", 1000, false);
+            leaderProps = zkController.getLeaderProps(COLLECTION, 1l, "shard1", 1000, false);
           } catch (SolrException e) {
             return false;
           } catch (InterruptedException e) {
@@ -1269,8 +1270,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
     when(zkController.getZkClient()).thenReturn(zkClient);
     when(zkController.getZkStateReader()).thenReturn(reader);
 
-    when(zkController.getLeaderProps(anyString(), anyString(), anyInt())).thenCallRealMethod();
-    when(zkController.getLeaderProps(anyString(), anyString(), anyInt(), anyBoolean())).thenCallRealMethod();
+    when(zkController.getLeaderProps(anyString(), anyLong(), anyString(), anyInt())).thenCallRealMethod();
+    when(zkController.getLeaderProps(anyString(),  anyLong(), anyString(), anyInt(), anyBoolean())).thenCallRealMethod();
     doReturn(getCloudDataProvider(zkAddress, zkClient, reader))
         .when(zkController).getSolrCloudManager();
     return zkController;
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java b/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java
index 3b08a26..5ddeb39 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java
@@ -273,11 +273,12 @@ public class TestHashPartitioner extends SolrTestCaseJ4 {
     Map<String,Slice> slices = new HashMap<>();
     for (int i=0; i<ranges.size(); i++) {
       Range range = ranges.get(i);
-      Slice slice = new Slice("shard"+(i+1), null, map("range",range), "collections1", nodeName -> "http://" + nodeName);
+      Slice slice = new Slice("shard"+(i+1), null, map("range",range), "collections1", -1l, nodeName -> "http://" + nodeName);
       slices.put(slice.getName(), slice);
     }
-
-    DocCollection coll = new DocCollection("collection1", slices, null, router);
+    Map<String, Object> collectionProps = new HashMap<>();
+    collectionProps.put("id", -1l);
+    DocCollection coll = new DocCollection("collection1", slices, collectionProps, router);
     return coll;
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
index ff544a9..c66a1b8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -68,7 +68,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 
-@Slow
+@LuceneTestCase.Nightly
 public class TestPullReplica extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
diff --git a/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java b/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
index 84eeee9..f9b5967 100644
--- a/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
+++ b/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
@@ -123,7 +123,7 @@ public class CoreSorterTest extends SolrTestCaseJ4 {
           addNewReplica(replicas, collection, slice, downNodes);
         }
         Map<String, Replica> replicaMap = replicas.stream().collect(Collectors.toMap(Replica::getName, Function.identity()));
-        sliceMap.put(slice, new Slice(slice, replicaMap, map(), collection, nodeName -> "http://" + nodeName));
+        sliceMap.put(slice, new Slice(slice, replicaMap, map(), collection, -1l, nodeName -> "http://" + nodeName));
       }
       DocCollection col = new DocCollection(collection, sliceMap, map(), DocRouter.DEFAULT);
       collToState.put(collection, col);
@@ -192,7 +192,7 @@ public class CoreSorterTest extends SolrTestCaseJ4 {
   protected Replica addNewReplica(List<Replica> replicaList, String collection, String slice, List<String> possibleNodes) {
     String replica = "r" + replicaList.size();
     String node = possibleNodes.get(random().nextInt(possibleNodes.size())); // place on a random node
-    Replica r = new Replica(replica, map("core", replica, "node_name", node), collection, slice, nodeName -> "http://" + nodeName);
+    Replica r = new Replica(replica, map("core", replica, "node_name", node), collection, -1l, slice, nodeName -> "http://" + nodeName);
     replicaList.add(r);
     return r;
   }
diff --git a/solr/core/src/test/org/apache/solr/handler/component/SearchHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/component/SearchHandlerTest.java
index 28562f1..8aaafe4 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/SearchHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/SearchHandlerTest.java
@@ -37,16 +37,23 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
+import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class SearchHandlerTest extends SolrTestCaseJ4 
 {
   @BeforeClass
-  public static void beforeTests() throws Exception {
+  public static void beforeSearchHandlerTest() throws Exception {
     initCore("solrconfig.xml","schema.xml");
   }
 
+  @AfterClass
+  public static void afterSearchHandlerTest() throws Exception {
+    deleteCore();
+  }
+
   @Test
   public void testInitialization()
   {
diff --git a/solr/core/src/test/org/apache/solr/request/TestIntervalFaceting.java b/solr/core/src/test/org/apache/solr/request/TestIntervalFaceting.java
index 46e65d8..8651921 100644
--- a/solr/core/src/test/org/apache/solr/request/TestIntervalFaceting.java
+++ b/solr/core/src/test/org/apache/solr/request/TestIntervalFaceting.java
@@ -57,6 +57,7 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
   private final static long DATE_START_TIME_RANDOM_TEST = 1499797224224L;
+  public static final long[] EMPTY_LONGS = {};
   private final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'", Locale.ROOT);
   
   @BeforeClass
@@ -508,22 +509,29 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
     assertInterval("test_l_dv", "[0,2]", new long[]{0, 1, 2}, new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{3, Integer.MAX_VALUE, Long.MAX_VALUE});
     assertInterval("test_l_dv", "[0,2)", new long[]{0, 1}, new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{2, 3, Integer.MAX_VALUE, Long.MAX_VALUE});
 
-    assertInterval("test_l_dv", "(0,*)", new long[]{1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{-1, 0, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{});
-    assertInterval("test_l_dv", "(*,2)", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 0, 1}, new long[]{}, new long[]{2, 3, Integer.MAX_VALUE, Long.MAX_VALUE});
-    assertInterval("test_l_dv", "(*,*)", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 0, 1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{}, new long[]{});
+    assertInterval("test_l_dv", "(0,*)", new long[]{1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{-1, 0, Integer.MIN_VALUE, Long.MIN_VALUE},
+        EMPTY_LONGS);
+    assertInterval("test_l_dv", "(*,2)", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 0, 1}, EMPTY_LONGS, new long[]{2, 3, Integer.MAX_VALUE, Long.MAX_VALUE});
+    assertInterval("test_l_dv", "(*,*)", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 0, 1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, EMPTY_LONGS,
+        EMPTY_LONGS);
 
-    assertInterval("test_l_dv", "[0,*]", new long[]{0, 1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{});
-    assertInterval("test_l_dv", "[*,2]", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 0, 1, 2}, new long[]{}, new long[]{3, Integer.MAX_VALUE, Long.MAX_VALUE});
-    assertInterval("test_l_dv", "[*,*]", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 0, 1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{}, new long[]{});
+    assertInterval("test_l_dv", "[0,*]", new long[]{0, 1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE},
+        EMPTY_LONGS);
+    assertInterval("test_l_dv", "[*,2]", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 0, 1, 2}, EMPTY_LONGS, new long[]{3, Integer.MAX_VALUE, Long.MAX_VALUE});
+    assertInterval("test_l_dv", "[*,*]", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 0, 1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, EMPTY_LONGS,
+        EMPTY_LONGS);
 
-    assertInterval("test_l_dv", "(2,2)", new long[]{}, new long[]{2, 1, 0, -1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{3, Integer.MAX_VALUE, Long.MAX_VALUE});
-    assertInterval("test_l_dv", "(0,0)", new long[]{}, new long[]{0, -1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE});
+    assertInterval("test_l_dv", "(2,2)", EMPTY_LONGS, new long[]{2, 1, 0, -1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{3, Integer.MAX_VALUE, Long.MAX_VALUE});
+    assertInterval("test_l_dv", "(0,0)", EMPTY_LONGS, new long[]{0, -1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE});
 
-    assertInterval("test_l_dv", "(0," + Long.MAX_VALUE + "]", new long[]{1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{0, -1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{});
+    assertInterval("test_l_dv", "(0," + Long.MAX_VALUE + "]", new long[]{1, 2, 3, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{0, -1, Integer.MIN_VALUE, Long.MIN_VALUE},
+        EMPTY_LONGS);
     assertInterval("test_l_dv", "(0," + Long.MAX_VALUE + ")", new long[]{1, 2, 3, Integer.MAX_VALUE}, new long[]{0, -1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{Long.MAX_VALUE});
     assertInterval("test_l_dv", "(" + Long.MIN_VALUE + ",0)", new long[]{-1, Integer.MIN_VALUE}, new long[]{Long.MIN_VALUE}, new long[]{1, 2, Integer.MAX_VALUE, Long.MAX_VALUE});
-    assertInterval("test_l_dv", "[" + Long.MIN_VALUE + ",0)", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{}, new long[]{1, 2, Integer.MAX_VALUE, Long.MAX_VALUE});
-    assertInterval("test_l_dv", "[" + Long.MIN_VALUE + "," + Long.MAX_VALUE + "]", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 1, 2, Integer.MAX_VALUE, Long.MAX_VALUE}, new long[]{}, new long[]{});
+    assertInterval("test_l_dv", "[" + Long.MIN_VALUE + ",0)", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE},
+        EMPTY_LONGS, new long[]{1, 2, Integer.MAX_VALUE, Long.MAX_VALUE});
+    assertInterval("test_l_dv", "[" + Long.MIN_VALUE + "," + Long.MAX_VALUE + "]", new long[]{-1, Integer.MIN_VALUE, Long.MIN_VALUE, 1, 2, Integer.MAX_VALUE, Long.MAX_VALUE},
+        EMPTY_LONGS, EMPTY_LONGS);
     assertInterval("test_l_dv", "(" + Long.MIN_VALUE + "," + Long.MAX_VALUE + ")", new long[]{-1, Integer.MIN_VALUE, 1, 2, Integer.MAX_VALUE}, new long[]{Long.MIN_VALUE}, new long[]{Long.MAX_VALUE});
 
     assertInterval("test_l_dv", "( 0,2)", new long[]{1}, new long[]{0, -1, Integer.MIN_VALUE, Long.MIN_VALUE}, new long[]{2, 3, Integer.MAX_VALUE, Long.MAX_VALUE});
diff --git a/solr/core/src/test/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactoryTest.java
index 0c7f4ce..667278a 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactoryTest.java
@@ -74,9 +74,10 @@ public class AddSchemaFieldsUpdateProcessorFactoryTest extends UpdateProcessorTe
     doc.addField(fieldName, null);
 
     SolrInputDocument finalDoc = doc;
-    SolrTestCaseUtil.expectThrows(AssertionError.class, () -> processAdd("add-fields-no-run-processor", finalDoc));
-
-    SolrTestCaseUtil.expectThrows(AssertionError.class, () -> processAdd("add-fields-no-run-processor", new SolrInputDocument(null, null)));
+    // MRM TODO: we are being real here, aserts don't throw in the real
+//    SolrTestCaseUtil.expectThrows(AssertionError.class, () -> processAdd("add-fields-no-run-processor", finalDoc));
+//
+//    SolrTestCaseUtil.expectThrows(AssertionError.class, () -> processAdd("add-fields-no-run-processor", new SolrInputDocument(null, null)));
   }
 
   public void testSingleField() throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateRemovalJavabinTest.java b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateRemovalJavabinTest.java
index 12fc4b4..eca17a3 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateRemovalJavabinTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateRemovalJavabinTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrInputDocument;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -52,7 +53,7 @@ public class AtomicUpdateRemovalJavabinTest extends SolrCloudTestCase {
   private static final Date DATE_2 = Date.from(Instant.ofEpochSecond(1554243909));
 
   @BeforeClass
-  public static void setupCluster() throws Exception {
+  public static void beforeAtomicUpdateRemovalJavabinTest() throws Exception {
     configureCluster(1)
         .addConfig("conf", SolrTestUtil.configset("cloud-dynamic"))
         .configure();
@@ -61,8 +62,6 @@ public class AtomicUpdateRemovalJavabinTest extends SolrCloudTestCase {
         .setMaxShardsPerNode(MAX_SHARDS_PER_NODE)
         .process(cluster.getSolrClient());
 
-    cluster.waitForActiveCollection(COLLECTION, 1, 1);
-
     final SolrInputDocument doc1 = SolrTestCaseJ4.sdoc(
         "id", "1",
         "title_s", "title_1", "title_s", "title_2",
@@ -75,6 +74,11 @@ public class AtomicUpdateRemovalJavabinTest extends SolrCloudTestCase {
     req.commit(cluster.getSolrClient(), COLLECTION);
   }
 
+  @AfterClass
+  public static void afterAtomicUpdateRemovalJavabinTest() throws Exception {
+    shutdownCluster();
+  }
+
   @Test
   public void testAtomicUpdateRemovalOfStrField() throws Exception {
     ensureFieldHasValues("1", "title_s", "title_1", "title_2");
diff --git a/solr/core/src/test/org/apache/solr/update/processor/PreAnalyzedUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/PreAnalyzedUpdateProcessorTest.java
index 20ce997..381fb46 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/PreAnalyzedUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/PreAnalyzedUpdateProcessorTest.java
@@ -18,6 +18,7 @@ package org.apache.solr.update.processor;
 
 import org.apache.lucene.document.Field;
 import org.apache.solr.common.SolrInputDocument;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -40,10 +41,16 @@ public class PreAnalyzedUpdateProcessorTest extends UpdateProcessorTestBase {
   };
   
   @BeforeClass
-  public static void beforeClass() throws Exception {
+  public static void beforePreAnalyzedUpdateProcessorTest() throws Exception {
+    useFactory(null);
     initCore("solrconfig-update-processor-chains.xml", "schema12.xml");
   }
 
+  @AfterClass
+  public static void afterPreAnalyzedUpdateProcessorTest() throws Exception {
+    deleteCore();
+  }
+
   @Test
   public void testSimple() throws Exception {
     test("pre-analyzed-simple", simpleTitle, simpleTeststop);
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
index c873503..e7e60ab 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
@@ -268,8 +268,7 @@ public class CloudHttp2SolrClient extends BaseCloudSolrClient {
     }
 
     public Builder(ZkStateReader zkStateReader) {
-      ZkClientClusterStateProvider stateProvider = new ZkClientClusterStateProvider(zkStateReader, false);
-      this.stateProvider = stateProvider;
+      this.stateProvider = new ZkClientClusterStateProvider(zkStateReader, false);
     }
 
     /**
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 6895a6a..7687742 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -126,6 +126,7 @@ public class CloudSolrClient extends BaseCloudSolrClient {
     } catch (IOException ignore) {
       // no-op: not much we can do here
     }
+    IOUtils.closeQuietly(stateProvider);
   }
 
   private void propagateLBClientConfigOptions(Builder builder) {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
index 2700347..743b891 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
@@ -18,7 +18,6 @@
 package org.apache.solr.client.solrj.impl;
 
 import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.ParWork;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -32,7 +31,6 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -41,7 +39,7 @@ import java.util.Set;
 public class ZkClientClusterStateProvider implements ClusterStateProvider, Replica.NodeNameToBaseUrl {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  ZkStateReader zkStateReader;
+  volatile ZkStateReader zkStateReader;
   private boolean closeZkStateReader = false;
   final String zkHost;
   int zkConnectTimeout = 15000;
@@ -154,16 +152,24 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider, Repli
   }
 
   @Override
-  public synchronized void connect() {
+  public void connect() {
     if (isClosed) {
       throw new AlreadyClosedException();
     }
     if (this.zkStateReader == null) {
-      this.zkStateReader = new ZkStateReader(zkHost, zkClientTimeout, zkConnectTimeout);
-      this.zkStateReader.createClusterStateWatchersAndUpdate();
+      boolean createWatchers = false;
+      synchronized (this) {
+        if (this.zkStateReader == null) {
+          this.zkStateReader = new ZkStateReader(zkHost, zkClientTimeout, zkConnectTimeout);
+          createWatchers = true;
+        }
+      }
+      if (createWatchers)  {
+        this.zkStateReader.createClusterStateWatchersAndUpdate();
+      }
     }
   }
-  
+
   public ZkStateReader getZkStateReader() {
     if (isClosed) {
       throw new AlreadyClosedException();
@@ -180,7 +186,7 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider, Repli
   }
   
   @Override
-  public synchronized void close() throws IOException {
+  public void close() throws IOException {
     if (isClosed) {
       return;
     }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
index a29e497..8256125 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
@@ -21,7 +21,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -235,10 +234,10 @@ public class ClusterState implements JSONWriter.Writable {
     Map<String, Object> sliceObjs = (Map<String, Object>) objs.get(DocCollection.SHARDS);
     if (sliceObjs == null) {
       // legacy format from 4.0... there was no separate "shards" level to contain the collection shards.
-      slices = Slice.loadAllFromMap(zkStateReader, name, objs);
+      slices = Slice.loadAllFromMap(zkStateReader, name, (Long) objs.get("id"), objs);
       props = Collections.emptyMap();
     } else {
-      slices = Slice.loadAllFromMap(zkStateReader, name, sliceObjs);
+      slices = Slice.loadAllFromMap(zkStateReader, name, (Long) objs.get("id"), sliceObjs);
       props = new HashMap<>(objs);
       objs.remove(DocCollection.SHARDS);
     }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index 97845c0..03ca547 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -54,8 +54,6 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
 
   private final String name;
   private final Map<String, Slice> slices;
-  private final Map<String, List<Replica>> nodeNameReplicas;
-  private final Map<String, List<Replica>> nodeNameLeaderReplicas;
   private final DocRouter router;
 
   private final Integer replicationFactor;
@@ -83,8 +81,6 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     this.name = name;
     this.withStateUpdates = withStateUpdates;
     this.slices = slices;
-    this.nodeNameLeaderReplicas = new HashMap<>();
-    this.nodeNameReplicas = new HashMap<>();
     this.replicationFactor = (Integer) verifyProp(props, REPLICATION_FACTOR);
     this.numNrtReplicas = (Integer) verifyProp(props, NRT_REPLICAS, 0);
     this.numTlogReplicas = (Integer) verifyProp(props, TLOG_REPLICAS, 0);
@@ -94,37 +90,13 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     this.readOnly = readOnly == null ? Boolean.FALSE : readOnly;
 
     this.id = (Long) props.get("id");
-    
-    Iterator<Map.Entry<String, Slice>> iter = slices.entrySet().iterator();
 
-    while (iter.hasNext()) {
-      Map.Entry<String, Slice> slice = iter.next();
-      for (Replica replica : slice.getValue()) {
-        addNodeNameReplica(replica);
-      }
-    }
+    Objects.requireNonNull(this.id, "'id' must not be null");
+
     this.router = router;
     assert name != null && slices != null;
   }
 
-  private void addNodeNameReplica(Replica replica) {
-    List<Replica> replicas = nodeNameReplicas.get(replica.getNodeName());
-    if (replicas == null) {
-      replicas = new ArrayList<>();
-      nodeNameReplicas.put(replica.getNodeName(), replicas);
-    }
-    replicas.add(replica);
-
-    if (replica.getStr(Slice.LEADER) != null) {
-      List<Replica> leaderReplicas = nodeNameLeaderReplicas.get(replica.getNodeName());
-      if (leaderReplicas == null) {
-        leaderReplicas = new ArrayList<>();
-        nodeNameLeaderReplicas.put(replica.getNodeName(), leaderReplicas);
-      }
-      leaderReplicas.add(replica);
-    }
-  }
-  
   public static Object verifyProp(Map<String, Object> props, String propName) {
     return verifyProp(props, propName, null);
   }
@@ -225,14 +197,34 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
    * Get the list of replicas hosted on the given node or <code>null</code> if none.
    */
   public List<Replica> getReplicas(String nodeName) {
-    return nodeNameReplicas.get(nodeName);
+    Iterator<Map.Entry<String, Slice>> iter = slices.entrySet().iterator();
+    List<Replica> replicas = new ArrayList<>(slices.size());
+    while (iter.hasNext()) {
+      Map.Entry<String, Slice> slice = iter.next();
+      for (Replica replica : slice.getValue()) {
+        if (replica.getNodeName().equals(nodeName)) {
+          replicas.add(replica);
+        }
+      }
+    }
+    return replicas;
   }
 
   /**
    * Get the list of all leaders hosted on the given node or <code>null</code> if none.
    */
   public List<Replica> getLeaderReplicas(String nodeName) {
-    return nodeNameLeaderReplicas.get(nodeName);
+    Iterator<Map.Entry<String, Slice>> iter = slices.entrySet().iterator();
+    List<Replica> leaders = new ArrayList<>(slices.size());
+    while (iter.hasNext()) {
+      Map.Entry<String, Slice> slice = iter.next();
+      Replica leader = slice.getValue().getLeader();
+      if (leader != null && leader.getNodeName().equals(nodeName)) {
+        leaders.add(leader);
+      }
+
+    }
+    return leaders;
   }
 
   public int getZNodeVersion(){
@@ -296,6 +288,14 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     return null;
   }
 
+  public Map<String,Replica> getReplicaByIds() {
+    Map<String,Replica> ids = new HashMap<>();
+    for (Slice slice : slices.values()) {
+      ids.putAll(slice.getReplicaByIds());
+    }
+    return ids;
+  }
+
   public Slice getSlice(Replica replica) {
     for (Slice slice : slices.values()) {
       Replica r = slice.getReplica(replica.getName());
@@ -318,7 +318,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     long[] highest = new long[1];
     List<Replica> replicas = getReplicas();
     replicas.forEach(replica -> highest[0] = Math.max(highest[0], replica.id));
-    return highest[0];
+    return highest[0] + 1;
   }
 
   /**
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
index 43aeb9f..0db8243 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
@@ -25,16 +25,6 @@ import org.apache.solr.common.util.Utils;
 
 public class Replica extends ZkNodeProps {
 
-  Long id;
-  final Long collId;
-
-  public String getId() {
-    if (collId == null) {
-      return null;
-    }
-    return collId + "-" + id.toString();
-  }
-
   /**
    * The replica's state. In general, if the node the replica is hosted on is
    * not under {@code /live_nodes} in ZK, the replica's state should be
@@ -156,7 +146,7 @@ public class Replica extends ZkNodeProps {
   public final String slice, collection;
   private final String baseUrl;
 
-  public Replica(String name, Map<String,Object> propMap, String collection, String slice, NodeNameToBaseUrl nodeNameToBaseUrl) {
+  public Replica(String name, Map<String,Object> propMap, String collection, Long collectionId, String slice, NodeNameToBaseUrl nodeNameToBaseUrl) {
     super(propMap);
     this.collection = collection;
     this.slice = slice;
@@ -165,11 +155,13 @@ public class Replica extends ZkNodeProps {
     this.nodeName = (String) propMap.get(ZkStateReader.NODE_NAME_PROP);
 
     String rawId = (String) propMap.get("id");
-    if (rawId != null && !rawId.contains(":")) {
+
+    if (!rawId.contains(":")) {
       this.id = Long.parseLong(rawId);
     }
 
-    this.collId = propMap.containsKey("collId") ? Long.parseLong((String) propMap.get("collId")) : null;
+    this.collectionId = collectionId;
+
     this.baseUrl = nodeNameToBaseUrl.getBaseUrlForNodeName(this.nodeName);
     type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
     Objects.requireNonNull(this.collection, "'collection' must not be null");
@@ -177,10 +169,10 @@ public class Replica extends ZkNodeProps {
     Objects.requireNonNull(this.name, "'name' must not be null");
     Objects.requireNonNull(this.nodeName, "'node_name' must not be null");
     Objects.requireNonNull(this.type, "'type' must not be null");
-    // Objects.requireNonNull(this.id, "'id' must not be null");
+    Objects.requireNonNull(this.collectionId, "'collectionId' must not be null");
 
     if (propMap.get(ZkStateReader.STATE_PROP) != null) {
-      if (propMap.get(ZkStateReader.STATE_PROP) instanceof  State) {
+      if (propMap.get(ZkStateReader.STATE_PROP) instanceof State) {
         this.state = (State) propMap.get(ZkStateReader.STATE_PROP);
       } else {
         this.state = State.getState((String) propMap.get(ZkStateReader.STATE_PROP));
@@ -191,14 +183,16 @@ public class Replica extends ZkNodeProps {
     }
   }
 
-  public Replica(String name, Map<String,Object> propMap, String collection, String slice, String baseUrl) {
+  public Replica(String name, Map<String,Object> propMap, String collection, Long collectionId, String slice, String baseUrl) {
     super(propMap);
     this.collection = collection;
     this.slice = slice;
     this.name = name;
     this.nodeName = (String) propMap.get(ZkStateReader.NODE_NAME_PROP);
     this.id = propMap.containsKey("id") ? Long.parseLong((String) propMap.get("id")) : null;
-    this.collId = propMap.containsKey("collId") ? Long.parseLong((String) propMap.get("collId")) : null;
+    this.collectionId = collectionId;
+
+    Objects.requireNonNull(this.collectionId, "'collectionId' must not be null");
     this.baseUrl =  baseUrl;
     type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
     if (propMap.get(ZkStateReader.STATE_PROP) != null) {
@@ -213,6 +207,18 @@ public class Replica extends ZkNodeProps {
     }
   }
 
+  Long id;
+  final Long collectionId;
+
+  public String getId() {
+    return collectionId + "-" + id.toString();
+  }
+
+  public Long getCollectionId() {
+    return collectionId;
+  }
+
+
   public String getCollection(){
     return collection;
   }
@@ -288,6 +294,6 @@ public class Replica extends ZkNodeProps {
 
   @Override
   public String toString() {
-    return name + ':' + Utils.toJSONString(propMap); // small enough, keep it on one line (i.e. no indent)
+    return name + "(" + getId() + ")" + ':' + Utils.toJSONString(propMap); // small enough, keep it on one line (i.e. no indent)
   }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
index 398a940..0408104 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
@@ -41,7 +41,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
   private final HashMap<String,Replica> idToReplica;
 
   /** Loads multiple slices into a Map from a generic Map that probably came from deserialized JSON. */
-  public static Map<String,Slice> loadAllFromMap(Replica.NodeNameToBaseUrl nodeNameToBaseUrl, String collection, Map<String, Object> genericSlices) {
+  public static Map<String,Slice> loadAllFromMap(Replica.NodeNameToBaseUrl nodeNameToBaseUrl, String collection, long id, Map<String, Object> genericSlices) {
     if (genericSlices == null) return Collections.emptyMap();
     Map<String, Slice> result = new LinkedHashMap<>(genericSlices.size());
     for (Map.Entry<String, Object> entry : genericSlices.entrySet()) {
@@ -50,7 +50,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
       if (val instanceof Slice) {
         result.put(name, (Slice) val);
       } else if (val instanceof Map) {
-        result.put(name, new Slice(name, null, (Map<String, Object>) val, collection, nodeNameToBaseUrl));
+        result.put(name, new Slice(name, null, (Map<String, Object>) val, collection, id, nodeNameToBaseUrl));
       }
     }
     return result;
@@ -65,6 +65,10 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
     return idToReplica.get(id);
   }
 
+  public Map<String,Replica> getReplicaByIds() {
+    return idToReplica;
+  }
+
   /** The slice's state. */
   public enum State {
 
@@ -134,7 +138,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
    * @param replicas The replicas of the slice.  This is used directly and a copy is not made.  If null, replicas will be constructed from props.
    * @param props  The properties of the slice - a shallow copy will always be made.
    */
-  public Slice(String name, Map<String,Replica> replicas, Map<String,Object> props, String collection, Replica.NodeNameToBaseUrl nodeNameToBaseUrl) {
+  public Slice(String name, Map<String,Replica> replicas, Map<String,Object> props, String collection, Long collectionId, Replica.NodeNameToBaseUrl nodeNameToBaseUrl) {
     super( props==null ? new LinkedHashMap<String,Object>(2) : new LinkedHashMap<>(props));
     this.name = name;
     this.collection = collection;
@@ -169,7 +173,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
     replicationFactor = null;  // future
 
     // add the replicas *after* the other properties (for aesthetics, so it's easy to find slice properties in the JSON output)
-    this.replicas = replicas != null ? replicas : makeReplicas(collection,name, (Map<String,Object>)propMap.get(REPLICAS));
+    this.replicas = replicas != null ? replicas : makeReplicas(collection, collectionId, name, (Map<String,Object>)propMap.get(REPLICAS));
 
     this.idToReplica = new HashMap<>(this.replicas.size());
 
@@ -203,7 +207,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
   }
 
 
-  private Map<String,Replica> makeReplicas(String collection, String slice,Map<String,Object> genericReplicas) {
+  private Map<String,Replica> makeReplicas(String collection, Long collectionId, String slice,Map<String,Object> genericReplicas) {
     if (genericReplicas == null) return new HashMap<>(1);
     Map<String,Replica> result = new LinkedHashMap<>(genericReplicas.size());
     for (Map.Entry<String,Object> entry : genericReplicas.entrySet()) {
@@ -213,7 +217,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
       if (val instanceof Replica) {
         r = (Replica)val;
       } else {
-        r = new Replica(name, (Map<String,Object>)val, collection, slice, nodeNameToBaseUrl);
+        r = new Replica(name, (Map<String,Object>)val, collection, collectionId, slice, nodeNameToBaseUrl);
       }
       result.put(name, r);
     }
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 6cdd270..cfcc550 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
@@ -282,6 +282,17 @@ public class SolrZkClient implements Closeable {
     }
   }
 
+  public void deleteAsync(final String path, final int version)
+      throws InterruptedException, KeeperException {
+    connManager.getKeeper().delete(path, version, (rc, path1, ctx) -> {
+      if (rc != 0) {
+        log.error("got zk error deleting path {} {}", path1, rc);
+        KeeperException e = KeeperException.create(KeeperException.Code.get(rc), path1);
+        log.error("Exception deleting znode path=" + path1, e);
+      }
+    }, "");
+  }
+
   /**
    * Wraps the watcher so that it doesn't fire off ZK's event queue. In order to guarantee that a watch object will
    * only be triggered once for a given notification, users need to wrap their watcher using this method before
@@ -400,14 +411,19 @@ public class SolrZkClient implements Closeable {
     }
   }
 
+  public Stat setData(final String path, final byte data[], final int version, boolean retryOnConnLoss)
+      throws KeeperException, InterruptedException {
+    return setData(path, data, version, retryOnConnLoss, true);
+  }
+
   /**
    * Returns node's state
    */
-  public Stat setData(final String path, final byte data[], final int version, boolean retryOnConnLoss)
+  public Stat setData(final String path, final byte data[], final int version, boolean retryOnConnLoss, boolean retryOnSessionExpiration)
       throws KeeperException, InterruptedException {
 
     if (retryOnConnLoss) {
-      return ZkCmdExecutor.retryOperation(zkCmdExecutor, new SetData(connManager.getKeeper(), path, data, version));
+      return ZkCmdExecutor.retryOperation(zkCmdExecutor, new SetData(connManager.getKeeper(), path, data, version), retryOnSessionExpiration);
     } else {
       return connManager.getKeeper().setData(path, data, version);
     }
@@ -1242,6 +1258,10 @@ public class SolrZkClient implements Closeable {
     this.higherLevelIsClosed = isClosed;
   }
 
+  public IsClosed getHigherLevelIsClosed() {
+    return this.higherLevelIsClosed;
+  }
+
   /**
    * Update all ACLs for a zk tree based on our configured {@link ZkACLProvider}.
    * @param root the root node to recursively update
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java
index a96c865..01fc704 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java
@@ -307,7 +307,7 @@ public class ZkMaintenanceUtils {
       public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
         String filename = file.getFileName().toString();
         if (filenameExclusions != null && filenameExclusions.matcher(filename).matches()) {
-          log.info("uploadToZK skipping '{}' due to filenameExclusions '{}'", filename, filenameExclusions);
+          log.debug("uploadToZK skipping '{}' due to filenameExclusions '{}'", filename, filenameExclusions);
           return FileVisitResult.CONTINUE;
         }
         String zkNode = createZkNodeName(zkPath, rootPath, file);
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
index 7394257..cf8bc86 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
@@ -129,6 +129,14 @@ public class ZkNodeProps implements JSONWriter.Writable {
   /**
    * Get a string property value.
    */
+  public Long getLong(String key, Long def) {
+    Object o = propMap.get(key);
+    return o == null ? def : Long.valueOf(o.toString());
+  }
+
+  /**
+   * Get a string property value.
+   */
   public String getStr(String key,String def) {
     Object o = propMap.get(key);
     return o == null ? def : o.toString();
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 fa38e9d..20a018d 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
@@ -497,7 +497,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         try {
           boolean success = latch.await(1000, TimeUnit.MILLISECONDS);
           if (!success) {
-            log.warn("Timed waiting to see {} node in zk \n{}", "/cluster/init", clusterState);
+            throw new SolrException(ErrorCode.SERVER_ERROR, "cluster not found/not ready");
           }
           if (log.isDebugEnabled()) log.debug("Done waiting on latch");
         } catch (InterruptedException e) {
@@ -881,6 +881,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     if (log.isDebugEnabled()) log.debug("Closing ZkStateReader");
     assert closeTracker != null ? closeTracker.close() : true;
 
+    closed = true;
     try {
       IOUtils.closeQuietly(clusterPropertiesWatcher);
       Future<?> cpc = collectionPropsCacheCleaner;
@@ -993,7 +994,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
           if (!mustBeLive) {
             if (zkLeader == null) {
-              zkLeader = getLeaderProps(collection, shard);
+              zkLeader = getLeaderProps(collection, coll.getId(), shard);
             }
             if (zkLeader != null && zkLeader.getName().equals(leader.getName())) {
               returnLeader.set(leader);
@@ -1010,7 +1011,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
             }
             if (!mustBeLive) {
               if (zkLeader == null) {
-                zkLeader = getLeaderProps(collection, shard);
+                zkLeader = getLeaderProps(collection, coll.getId(), shard);
               }
               if (zkLeader != null && zkLeader.getName().equals(replica.getName())) {
                 returnLeader.set(replica);
@@ -1025,7 +1026,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     } catch (TimeoutException e) {
       throw new TimeoutException("No registered leader was found after waiting for "
           + timeout + "ms " + ", collection: " + collection + " slice: " + shard + " saw state=" + clusterState.getCollectionOrNull(collection)
-          + " with live_nodes=" + liveNodes + " zkLeaderNode=" + getLeaderProps(collection, shard));
+          + " with live_nodes=" + liveNodes + " zkLeaderNode=" + getLeaderProps(collection, coll.getId(), shard));
     }
 
     Replica leader = returnLeader.get();
@@ -1033,21 +1034,22 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     if (leader == null) {
       throw new SolrException(ErrorCode.SERVER_ERROR, "No registered leader was found "
           + "collection: " + collection + " slice: " + shard + " saw state=" + clusterState.getCollectionOrNull(collection)
-          + " with live_nodes=" + liveNodes + " zkLeaderNode=" + getLeaderProps(collection, shard));
+          + " with live_nodes=" + liveNodes + " zkLeaderNode=" + getLeaderProps(collection, coll.getId(), shard));
     }
 
     return leader;
   }
 
-  public Replica getLeaderProps(final String collection, final String slice) {
+  public Replica getLeaderProps(final String collection, long collId, final String slice) {
 
     try {
       byte[] data = zkClient.getData(ZkStateReader.getShardLeadersPath(collection, slice), null, null);
       ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(ZkNodeProps.load(data));
       String name = leaderProps.getNodeProps().getStr(ZkStateReader.CORE_NAME_PROP);
       leaderProps.getNodeProps().getProperties().remove(ZkStateReader.CORE_NAME_PROP);
+
       // MRM TODO: - right key for leader name?
-      return new Replica(name, leaderProps.getNodeProps().getProperties(), collection, slice, this);
+      return new Replica(name, leaderProps.getNodeProps().getProperties(), collection, collId, slice, this);
 
     } catch (KeeperException.NoNodeException e) {
       return null;
@@ -1357,7 +1359,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     private final String coll;
     private volatile StateUpdateWatcher stateUpdateWatcher;
 
-    private final ReentrantLock collectionStateLock = new ReentrantLock(true);
+    private final ReentrantLock collectionStateLock = new ReentrantLock();
+    private volatile boolean closed;
 
     CollectionStateWatcher(String coll) {
       this.coll = coll;
@@ -1367,6 +1370,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
     @Override
     public void process(WatchedEvent event) {
+      if (zkClient.isClosed() || closed) return;
+
       // session events are not change events, and do not remove the watcher
       if (EventType.None.equals(event.getType())) {
         return;
@@ -1398,7 +1403,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
      * with the results of the refresh.
      */
     public void refresh() {
-      // MRM TODO: per collection lock, put it on the watcher object
       collectionStateLock.lock();
       try {
         DocCollection newState = fetchCollectionState(coll);
@@ -1431,7 +1435,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     }
 
     public void removeWatch() {
-      CountDownLatch latch = new CountDownLatch(2);
+
       String collectionCSNPath = getCollectionSCNPath(coll);
       try {
         zkClient.removeWatches(collectionCSNPath, this, WatcherType.Any, true);
@@ -1487,7 +1491,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
       m.remove("_cs_ver_");
 
-      // TODO per collection?
       collectionStateLock.lock();
       try {
         Set<Entry<String,Object>> entrySet = m.entrySet();
@@ -1510,7 +1513,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
             }
 
             Replica replica = docCollection.getReplicaById(id);
-            if (log.isDebugEnabled()) log.debug("Got additional state update {} {}", replica.getName(), state == null ? "leader" : state);
+            if (log.isDebugEnabled()) log.debug("Got additional state update replica={} id={} ids={} {} {}", replica, id, docCollection.getReplicaByIds(), state == null ? "leader" : state);
 
             if (replica != null) {
 
@@ -1531,7 +1534,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
                   if ("true".equals(r.getProperty(LEADER_PROP))) {
                     Map<String,Object> props = new HashMap<>(r.getProperties());
                     props.remove(LEADER_PROP);
-                    Replica newReplica = new Replica(r.getName(), props, coll, r.getSlice(), ZkStateReader.this);
+                    Replica newReplica = new Replica(r.getName(), props, coll, docCollection.getId(), r.getSlice(), ZkStateReader.this);
                     replicasMap.put(r.getName(), newReplica);
                   }
                 }
@@ -1543,13 +1546,13 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
                 }
               }
 
-              Replica newReplica = new Replica(replica.getName(), properties, coll, replica.getSlice(), ZkStateReader.this);
+              Replica newReplica = new Replica(replica.getName(), properties, coll, docCollection.getId(), replica.getSlice(), ZkStateReader.this);
 
               if (log.isDebugEnabled()) log.debug("add new replica {}", newReplica);
 
               replicasMap.put(replica.getName(), newReplica);
 
-              Slice newSlice = new Slice(slice.getName(), replicasMap, slice.getProperties(), coll, ZkStateReader.this);
+              Slice newSlice = new Slice(slice.getName(), replicasMap, slice.getProperties(), coll, replica.id, ZkStateReader.this);
 
               Map<String,Slice> newSlices = new HashMap<>(docCollection.getSlicesMap());
               newSlices.put(slice.getName(), newSlice);
@@ -1564,7 +1567,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
               //  }
             } else {
-              if (log.isDebugEnabled()) log.debug("Could not find core to update local state {} {}", replica.getName(), state);
+              if (log.isDebugEnabled()) log.debug("Could not find core to update local state {} {}", id, state);
             }
           }
           if (changedCollections.size() > 0) {
@@ -1618,6 +1621,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
     @Override
     public void close() throws IOException {
+      this.closed = true;
+      IOUtils.closeQuietly(stateUpdateWatcher);
 //      SolrZooKeeper zk = zkClient.getSolrZooKeeper();
 //      if (zk != null) {
 //        try {
@@ -1633,6 +1638,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
     private class StateUpdateWatcher implements Watcher, Closeable {
       private final String stateUpdatesPath;
+      private volatile boolean closed;
 
       public StateUpdateWatcher(String stateUpdatesPath) {
         this.stateUpdatesPath = stateUpdatesPath;
@@ -1640,6 +1646,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
       @Override
       public void close() throws IOException {
+        this.closed = true;
 //        SolrZooKeeper zk = zkClient.getSolrZooKeeper();
 //        if (zk != null) {
 //          if (stateUpdateWatcher != null) {
@@ -1656,7 +1663,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
       @Override
       public void process(WatchedEvent event) {
-        if (closed || zkClient.isClosed()) return;
+        if (zkClient.isClosed() || closed) return;
         if (log.isDebugEnabled()) log.debug("_statupdates event {}", event);
 
         try {
@@ -1784,7 +1791,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
    */
   // MRM TODO: persistent watch
   class CollectionsChildWatcher implements Watcher, Closeable {
-
+    volatile boolean watchRemoved = true;
     @Override
     public void process(WatchedEvent event) {
       if (ZkStateReader.this.closed) {
@@ -1824,6 +1831,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     }
 
     public void createWatch() {
+      watchRemoved = false;
       try {
         zkClient.addWatch(COLLECTIONS_ZKNODE, this, AddWatchMode.PERSISTENT);
       } catch (Exception e) {
@@ -1832,6 +1840,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     }
 
     public void removeWatch() {
+      if (watchRemoved) return;
+      watchRemoved = true;
       try {
         zkClient.removeWatches(COLLECTIONS_ZKNODE, this, WatcherType.Any, true);
       } catch (KeeperException.NoWatcherException e) {
@@ -2415,7 +2425,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     try {
       notifications.submit(new Notification(collection, collectionState, collectionWatches));
     } catch (RejectedExecutionException e) {
-      if (closed == false) {
+      if (!closed) {
         log.error("Couldn't run collection notifications for {}", collection, e);
       }
     }
@@ -2440,23 +2450,23 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (node != null) {
         MDCLoggingContext.setNode(node);
       }
-      List<DocCollectionWatcher> watchers = new ArrayList<>();
 
-      collectionWatches.compute(collection, (k, v) -> {
-        if (v == null) return null;
-        watchers.addAll(v.stateWatchers);
-        return v;
-      });
-
-      for (DocCollectionWatcher watcher : watchers) {
-        if (log.isDebugEnabled()) log.debug("Notify DocCollectionWatcher {} {}", watcher, collectionState);
-        try {
-          if (watcher.onStateChanged(collectionState)) {
-            removeDocCollectionWatcher(collection, watcher);
-          }
-        } catch (Exception exception) {
-          ParWork.propagateInterrupt(exception);
-          log.warn("Error on calling watcher", exception);
+      CollectionWatch<DocCollectionWatcher> watchers = collectionWatches.get(collection);
+      if (watchers != null) {
+        try (ParWork work = new ParWork(this)) {
+          watchers.stateWatchers.forEach(watcher -> {
+           // work.collect("", () -> {
+              if (log.isDebugEnabled()) log.debug("Notify DocCollectionWatcher {} {}", watcher, collectionState);
+              try {
+                if (watcher.onStateChanged(collectionState)) {
+                  removeDocCollectionWatcher(collection, watcher);
+                }
+              } catch (Exception exception) {
+                ParWork.propagateInterrupt(exception);
+                log.warn("Error on calling watcher", exception);
+              }
+            });
+         // });
         }
       }
     }
diff --git a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
index afcc1f7..e096ef5 100644
--- a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
@@ -44,6 +44,7 @@ import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.util.ExternalPaths;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -57,7 +58,7 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
   private static final String CONFIG_NAME = "techproducts_config";
 
   @BeforeClass
-  public static void setupCluster() throws Exception {
+  public static void beforeJsonRequestApiTest() throws Exception {
     configureCluster(1)
         .addConfig(CONFIG_NAME, new File(ExternalPaths.TECHPRODUCTS_CONFIGSET).toPath())
         .configure();
@@ -75,6 +76,11 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
     assertEquals(0, updateResponse.getStatus());
   }
 
+  @AfterClass
+  public static void afterJsonRequestApiTest() throws Exception {
+    shutdownCluster();
+  }
+
   @Test
   public void testSimpleJsonQuery() throws Exception {
     SolrClient solrClient = cluster.getSolrClient();
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BaseSolrClientWireMockTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BaseSolrClientWireMockTest.java
index a795841..8cb3028 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BaseSolrClientWireMockTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BaseSolrClientWireMockTest.java
@@ -42,7 +42,6 @@ import com.github.tomakehurst.wiremock.http.HttpServer;
 import com.github.tomakehurst.wiremock.http.HttpServerFactory;
 import com.github.tomakehurst.wiremock.http.StubRequestHandler;
 import com.github.tomakehurst.wiremock.jetty94.Jetty94HttpServer;
-import com.google.common.base.Optional;
 import org.apache.lucene.util.QuickPatchThreadsFilter;
 import org.apache.solr.SolrIgnoredThreadsFilter;
 import org.apache.solr.SolrTestCase;
@@ -188,6 +187,7 @@ public abstract class BaseSolrClientWireMockTest extends SolrTestCase {
       "        \"state\":\"active\",\n" +
       "        \"type\":\"NRT\",\n" +
       "        \"force_set_state\":\"false\",\n" +
+      "        \"id\":\"1\",\n" +
       "        \"leader\":\"true\"}}},\n" +
       "    \"s1\":{\n" +
       "      \"range\":\"80000000-ffffffff\",\n" +
@@ -199,6 +199,7 @@ public abstract class BaseSolrClientWireMockTest extends SolrTestCase {
       "        \"state\":\"active\",\n" +
       "        \"type\":\"NRT\",\n" +
       "        \"force_set_state\":\"false\",\n" +
+      "        \"id\":\"2\",\n" +
       "        \"leader\":\"true\"}}}}";
 
   protected static DocCollection buildMockDocCollection() throws IOException {
@@ -215,7 +216,7 @@ public abstract class BaseSolrClientWireMockTest extends SolrTestCase {
     props.put("nrtReplicas", "1");
     props.put("id", 1l);
 
-    return new DocCollection(BUILT_IN_MOCK_COLLECTION, Slice.loadAllFromMap(nodeName -> mockSolr.baseUrl() + "/solr", BUILT_IN_MOCK_COLLECTION, slices), props, DocRouter.DEFAULT);
+    return new DocCollection(BUILT_IN_MOCK_COLLECTION, Slice.loadAllFromMap(nodeName -> mockSolr.baseUrl() + "/solr", BUILT_IN_MOCK_COLLECTION, -1l, slices), props, DocRouter.DEFAULT);
   }
 
   protected static void updateReplicaBaseUrl(JsonNode json, String shard, String replica, String baseUrl) {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientRetryTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientRetryTest.java
index adb1ee6..6d8c78e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientRetryTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientRetryTest.java
@@ -32,21 +32,27 @@ 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.util.TestInjection;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
-@Ignore // MRM TODO: ~ still a bit flakey when run in the suite, usually passes in the IDE though
 public class CloudHttp2SolrClientRetryTest extends SolrCloudTestCase {
   private static final int NODE_COUNT = 1;
 
   @BeforeClass
-  public static void setupCluster() throws Exception {
+  public static void beforeCloudHttp2SolrClientRetryTest() throws Exception {
+    System.setProperty("solr.enableMetrics", "true");
     configureCluster(NODE_COUNT)
         .addConfig("conf", SolrTestUtil.getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
         .configure();
   }
 
+  @AfterClass
+  public static void afterCloudHttp2SolrClientRetryTest() throws Exception {
+    shutdownCluster();
+  }
+
+
   @Test
   public void testRetry() throws Exception {
     String collectionName = "testRetry";
@@ -58,7 +64,7 @@ public class CloudHttp2SolrClientRetryTest extends SolrCloudTestCase {
 
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.set(CommonParams.QT, "/admin/metrics");
-      String updateRequestCountKey = "solr.core.testRetry.shard1.replica_n1:UPDATE./update.requestTimes:count";
+      String updateRequestCountKey = "solr.core.testRetry.s1.testRetry_s1_r_n1:UPDATE./update.requestTimes:count";
       params.set("key", updateRequestCountKey);
       params.set("indent", "true");
 
@@ -66,7 +72,7 @@ public class CloudHttp2SolrClientRetryTest extends SolrCloudTestCase {
       NamedList<Object> namedList = response.getResponse();
       System.out.println(namedList);
       NamedList metrics = (NamedList) namedList.get("metrics");
-      assertEquals(1L, metrics.get(updateRequestCountKey));
+      assertEquals(metrics.toString(), 1L, metrics.get(updateRequestCountKey));
 
       TestInjection.failUpdateRequests = "true:100";
       try {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientRetryTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientRetryTest.java
index 84a4578..21e2782 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientRetryTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientRetryTest.java
@@ -29,20 +29,27 @@ 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.util.TestInjection;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-@LuceneTestCase.AwaitsFix(bugUrl = "flakey test on getting metrics")
 public class CloudSolrClientRetryTest extends SolrCloudTestCase {
   private static final int NODE_COUNT = 1;
 
   @BeforeClass
   public static void setupCluster() throws Exception {
+    System.setProperty("solr.enableMetrics", "true");
     configureCluster(NODE_COUNT)
         .addConfig("conf", SolrTestUtil.getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
         .configure();
   }
 
+  @AfterClass
+  public static void afterCloudSolrClientRetryTest() throws Exception {
+    shutdownCluster();
+  }
+
+
   @Test
   public void testRetry() throws Exception {
     String collectionName = "testRetry";
@@ -54,7 +61,7 @@ public class CloudSolrClientRetryTest extends SolrCloudTestCase {
 
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(CommonParams.QT, "/admin/metrics");
-    String updateRequestCountKey = "solr.core.testRetry.shard1.replica_n1:UPDATE./update.requestTimes:count";
+    String updateRequestCountKey = "solr.core.testRetry.s1.testRetry_s1_r_n1:UPDATE./update.requestTimes:count";
     params.set("key", updateRequestCountKey);
     params.set("indent", "true");
 
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/TestCloudSolrClientConnections.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/TestCloudSolrClientConnections.java
index 754bcd2..c613d4d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/TestCloudSolrClientConnections.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/TestCloudSolrClientConnections.java
@@ -16,76 +16,29 @@
  */
 package org.apache.solr.client.solrj.impl;
 
-import java.nio.file.Path;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.SolrTestUtil;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ZkConfigManager;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.junit.Ignore;
 import org.junit.Test;
 
-@Ignore // MRM TODO: debug
 public class TestCloudSolrClientConnections extends SolrTestCaseJ4 {
 
   @Test
-  public void testCloudClientCanConnectAfterClusterComesUp() throws Exception {
-
-    // Start by creating a cluster with no jetties
-    MiniSolrCloudCluster cluster = new MiniSolrCloudCluster(0, SolrTestUtil.createTempDir(), buildJettyConfig("/solr"));
-    try {
-
-      CloudHttp2SolrClient client = cluster.getSolrClient();
-      CollectionAdminRequest.List listReq = new CollectionAdminRequest.List();
-
-      SolrException e = LuceneTestCase.expectThrows(SolrException.class, () -> client.request(listReq));
-      assertTrue("Unexpected message: " + e.getMessage(), e.getMessage().contains("cluster not found/not ready"));
-
-      cluster.startJettySolrRunner();
-      cluster.waitForAllNodes(30);
-      client.connect(20, TimeUnit.SECONDS);
-
-      // should work now!
-      client.request(listReq);
-
-    }
-    finally {
-      cluster.shutdown();
-    }
-
-  }
-
-  @Test
   public void testCloudClientUploads() throws Exception {
 
-    Path configPath = SolrTestUtil.getFile("solrj").toPath().resolve("solr/configsets/configset-2/conf");
-
-    MiniSolrCloudCluster cluster = new MiniSolrCloudCluster(0, SolrTestUtil.createTempDir(), buildJettyConfig("/solr"));
-    try {
-      CloudHttp2SolrClient client = cluster.getSolrClient();
-      SolrException e = LuceneTestCase.expectThrows(SolrException.class, () -> {
-        ((ZkClientClusterStateProvider)client.getClusterStateProvider()).uploadConfig(configPath, "testconfig");
-      });
-      assertTrue("Unexpected message: " + e.getMessage(), e.getMessage().contains("cluster not found/not ready"));
-
-      cluster.startJettySolrRunner();
-      cluster.waitForAllNodes(30);
-      client.connect(20, TimeUnit.SECONDS);
-
-      ((ZkClientClusterStateProvider)client.getClusterStateProvider()).uploadConfig(configPath, "testconfig");
-
-      ZkConfigManager configManager = new ZkConfigManager(client.getZkStateReader().getZkClient());
-      assertTrue("List of uploaded configs does not contain 'testconfig'", configManager.listConfigs().contains("testconfig"));
-
-    } finally {
-      cluster.shutdown();
-    }
+    SolrException e = LuceneTestCase.expectThrows(SolrException.class, () -> {
+      MiniSolrCloudCluster cluster = null;
+      try  {
+         cluster = new MiniSolrCloudCluster(0, SolrTestUtil.createTempDir(), buildJettyConfig("/solr"));
+      } finally {
+        if (cluster != null) cluster.shutdown();
+      }
+    });
+    assertTrue("Unexpected message: " + e.getMessage(), e.getMessage().contains("cluster not found/not ready"));
   }
 
   @Test
@@ -98,7 +51,7 @@ public class TestCloudSolrClientConnections extends SolrTestCaseJ4 {
     try {
       final ZkClientClusterStateProvider zkHost_provider
         = new ZkClientClusterStateProvider(cluster.getSolrClient().getZkStateReader());
-      
+
       checkAndCloseProvider(zkHost_provider);
       
       final ZkStateReader reusedZkReader = new ZkStateReader(cluster.getZkClient());
@@ -137,7 +90,7 @@ public class TestCloudSolrClientConnections extends SolrTestCaseJ4 {
     LuceneTestCase.expectThrows(AlreadyClosedException.class, () -> {
         Object ignored = provider.getClusterState();
       });
-    
+
   }
 
 }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparatorTest.java
index 28d3b8b..5544a6e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparatorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparatorTest.java
@@ -73,8 +73,9 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node4",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "TLOG"
-            ),"collection1","shard1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "TLOG",
+                "id", "-1"
+            ),"collection1",-1l, "shard1", nodeName -> "http://" + nodeName
         )
     );
 
@@ -122,8 +123,9 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node1",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "NRT"
-            ),"collection1","shard1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "NRT",
+                "id", "-1"
+            ),"collection1",-1l, "shard1", nodeName -> "http://" + nodeName
         )
     );
     replicas.add(
@@ -132,8 +134,9 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node2",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "TLOG"
-            ),"collection1","shard1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "TLOG",
+                "id", "-1"
+            ),"collection1",-1l, "shard1", nodeName -> "http://" + nodeName
         )
     );
     replicas.add(
@@ -142,8 +145,9 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node3",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "PULL"
-            ),"collection1","shard1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "PULL",
+                "id", "-1"
+            ),"collection1",-1l, "shard1", nodeName -> "http://" + nodeName
         )
     );
     return replicas;
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/ReplicaListTransformerTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/ReplicaListTransformerTest.java
index a33e6a4..3351d8a 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/ReplicaListTransformerTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/ReplicaListTransformerTest.java
@@ -156,8 +156,9 @@ public class ReplicaListTransformerTest extends SolrTestCase {
         final Map<String, Object> propMap = new HashMap<String, Object>();
         propMap.put(ZkStateReader.NODE_NAME_PROP, url);
         propMap.put("type", "NRT");
+        propMap.put("id", String.valueOf(ii));
         // a skeleton replica, good enough for this test's purposes
-        final Replica replica = new Replica(name, propMap, "c1", "s1", new Replica.NodeNameToBaseUrl() {
+        final Replica replica = new Replica(name, propMap, "c1",-1l,"s1", new Replica.NodeNameToBaseUrl() {
           @Override
           public String getBaseUrlForNodeName(String nodeName) {
             return Utils.getBaseUrlForNodeName(name, "http");
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/RequestReplicaListTransformerGeneratorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/RequestReplicaListTransformerGeneratorTest.java
index 91d6b9d..0bcee65 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/RequestReplicaListTransformerGeneratorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/RequestReplicaListTransformerGeneratorTest.java
@@ -82,8 +82,9 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node4",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "TLOG"
-            ), "c1","s1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "TLOG",
+                "id", "-1"
+            ), "c1",-1l, "s1", nodeName -> "http://" + nodeName
         )
     );
 
@@ -94,8 +95,9 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node5",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "PULL"
-            ), "c1","s1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "PULL",
+                "id", "-1"
+            ), "c1",-1l, "s1", nodeName -> "http://" + nodeName
         )
     );
 
@@ -133,8 +135,9 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node1",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "NRT"
-            ),"c1","s1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "NRT",
+                "id", "-1"
+            ),"c1",-1l, "s1", nodeName -> "http://" + nodeName
         )
     );
     replicas.add(
@@ -143,8 +146,9 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node2",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "TLOG"
-            ),"c1","s1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "TLOG",
+                "id", "-1"
+            ),"c1",-1l, "s1", nodeName -> "http://" + nodeName
         )
     );
     replicas.add(
@@ -153,8 +157,9 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
             map(
                 ZkStateReader.NODE_NAME_PROP, "node3",
                 ZkStateReader.CORE_NAME_PROP, "collection1",
-                ZkStateReader.REPLICA_TYPE, "PULL"
-            ),"c1","s1", nodeName -> "http://" + nodeName
+                ZkStateReader.REPLICA_TYPE, "PULL",
+                "id", "-1"
+            ),"c1",-1l, "s1", nodeName -> "http://" + nodeName
         )
     );
     return replicas;
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/ShufflingReplicaListTransformerTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/ShufflingReplicaListTransformerTest.java
index 0d67d5f..bf75048 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/ShufflingReplicaListTransformerTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/ShufflingReplicaListTransformerTest.java
@@ -41,8 +41,9 @@ public class ShufflingReplicaListTransformerTest extends SolrTestCase {
       propMap.put("core", "core" + counter);
       propMap.put("type", "NRT");
       propMap.put("node_name", "node" + counter);
+      propMap.put("id", "-1");
       counter++;
-      replicas.add(new Replica(url, propMap, "c1", "s1", nodeName -> "http://" + nodeName));
+      replicas.add(new Replica(url, propMap, "c1", -1l, "s1", nodeName -> "http://" + nodeName));
     }
     implTestTransform(replicas);
   }
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCase.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCase.java
index 2e365c6..c172366 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCase.java
@@ -313,7 +313,7 @@ public class SolrTestCase extends Assert {
       Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
     }
     // based on randomized SSL config, set SocketFactoryRegistryProvider appropriately
-    if(isSSLMode()) {
+    if (isSSLMode()) {
       // SolrCloud tests should usually clear this
       System.setProperty("urlScheme", "https");
     } else {
@@ -323,7 +323,6 @@ public class SolrTestCase extends Assert {
     System.setProperty("useCompoundFile", "true");
     System.setProperty("solr.tests.maxBufferedDocs", "1000");
 
-
     System.setProperty("pkiHandlerPrivateKeyPath", SolrTestCaseJ4.class.getClassLoader().getResource("cryptokeys/priv_key512_pkcs8.pem").toExternalForm());
     System.setProperty("pkiHandlerPublicKeyPath", SolrTestCaseJ4.class.getClassLoader().getResource("cryptokeys/pub_key512.der").toExternalForm());
 
@@ -367,12 +366,14 @@ public class SolrTestCase extends Assert {
     System.setProperty("solr.tests.EnumFieldType", "org.apache.solr.schema.EnumFieldType");
     System.setProperty("solr.tests.numeric.dv", "true");
 
+    System.setProperty("managed.schema.mutable", "false");
+
     if (!LuceneTestCase.TEST_NIGHTLY) {
       //TestInjection.randomDelayMaxInCoreCreationInSec = 2;
       Lucene86Codec codec = new Lucene86Codec(Lucene50StoredFieldsFormat.Mode.BEST_SPEED);
       //Codec.setDefault(codec);
       disableReuseOfCryptoKeys();
-     // System.setProperty("solr.zkstatewriter.throttle", "30");
+      System.setProperty("solr.zkstatewriter.throttle", "0");
       System.setProperty("solr.stateworkqueue.throttle", "0");
 
       System.setProperty("zkReaderGetLeaderRetryTimeoutMs", "800");
@@ -723,13 +724,13 @@ public class SolrTestCase extends Assert {
       thread.interrupt();
       return true;
     }
-    if (thread.getName().contains(ParWork.ROOT_EXEC_NAME + "-") || thread.getName().contains("ParWork-") || thread.getName().contains("Core-")
-        || thread.getName().contains("ProcessThread(")) {
+    if ((thread.getName().contains(ParWork.ROOT_EXEC_NAME + "-") || thread.getName().contains("ParWork-") || thread.getName().contains("Core-")
+        || thread.getName().contains("ProcessThread(") && thread.getState() != Thread.State.TERMINATED)) {
       log.warn("interrupt on {}", thread.getName());
       thread.interrupt();
       return true;
     }
-    if (interruptThreadListContains(nameContains, thread.getName())) {
+    if (interruptThreadListContains(nameContains, thread.getName()) && thread.getState() != Thread.State.TERMINATED) {
       log.warn("interrupt on {}", thread.getName());
       thread.interrupt();
       return true;
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index ddd8109..6011adb 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -331,6 +331,7 @@ public class MiniSolrCloudCluster {
       solrZkClient = zkStateReader.getZkClient();
 
     } catch (Throwable t) {
+      shutdown();
       ParWork.propagateInterrupt(t);
       throw new SolrException(ErrorCode.SERVER_ERROR, t);
     }
diff --git a/solr/test-framework/src/java/org/apache/solr/update/processor/UpdateProcessorTestBase.java b/solr/test-framework/src/java/org/apache/solr/update/processor/UpdateProcessorTestBase.java
index 5f14a13..a47c600 100644
--- a/solr/test-framework/src/java/org/apache/solr/update/processor/UpdateProcessorTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/update/processor/UpdateProcessorTestBase.java
@@ -130,6 +130,7 @@ public class UpdateProcessorTestBase extends SolrTestCaseJ4 {
   protected void finish(final String chain) throws IOException {
     SolrCore core = h.getCore();
     UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+
     assertNotNull("No Chain named: " + chain, pc);
 
     SolrQueryResponse rsp = new SolrQueryResponse();
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index 30b3ec1..4734821 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -240,9 +240,7 @@ public class TestHarness extends BaseTestHarness {
   }
 
   public SolrCore getCore() {
-    SolrCore core = container.getCore(coreName);
-
-    return core;
+    return container.getCore(coreName);
   }
 
   /** Gets the core with its reference count incremented.
@@ -266,15 +264,14 @@ public class TestHarness extends BaseTestHarness {
   public String update(String xml) {
     try {
       try (DirectSolrConnection connection = new DirectSolrConnection(getCore())) {
-        try (SolrCore core = getCore()) {
-          SolrRequestHandler handler = core.getRequestHandler("/update");
-          // prefer the handler mapped to /update, but use our generic backup handler
-          // if that lookup fails
-          if (handler == null) {
-            handler = updater;
-          }
-          return connection.request(handler, null, xml);
+
+        SolrRequestHandler handler = connection.getCore().getRequestHandler("/update");
+        // prefer the handler mapped to /update, but use our generic backup handler
+        // if that lookup fails
+        if (handler == null) {
+          handler = updater;
         }
+        return connection.request(handler, null, xml);
       }
     } catch (SolrException e) {
       throw e;
diff --git a/solr/test-framework/src/resources/logconf/log4j2-std.xml b/solr/test-framework/src/resources/logconf/log4j2-std.xml
index 7e384fc..0dae120 100644
--- a/solr/test-framework/src/resources/logconf/log4j2-std.xml
+++ b/solr/test-framework/src/resources/logconf/log4j2-std.xml
@@ -23,10 +23,17 @@
             <PatternLayout pattern="%style{%-4r}{yellow} %highlight{%maxLen{%-5p}{6}} %style{(%maxLen{%t}{8})}{yellow,bold} [%style{%X{node_name} %X{core}}{cyan}] %style{%c{1.}}{cyan} %highlight{%m %notEmpty{%ex}}\n"/>
         </Console>
 
+        <Console name="STDERR" target="SYSTEM_ERR">
+            <PatternLayout pattern="%maxLen{%-4r %-5p (%t) [%X{node_name} %X{collection} %X{shard} %X{replica} %X{core} %X{trace_id}] %c{1.} %m%notEmpty{
+        =>%ex{short}}}{10240}%n"/>
+        </Console>
+
         <File name="FILE" fileName="${sys:user.home}/solr-test.log" immediateFlush="false" append="false">
             <PatternLayout pattern="%style{%-4r}{yellow} %highlight{%maxLen{%-5p}{6}} %style{(%maxLen{%t}{8})}{yellow,bold} [%style{%X{node_name} %X{core}}{cyan}] %style{%c{1.}}{cyan} %highlight{%m %notEmpty{%ex}}\n"/>
         </File>
 
+
+        <Null name="null" />
     </Appenders>
     <Loggers>
         <AsyncLogger name="org.apache.zookeeper" level="WARN"/>
@@ -63,15 +70,16 @@
 
         <AsyncLogger name="org.apache.solr.update.LoggingInfoStream" level="OFF"/>
 
-        <AsyncLogger name="org.apache.solr.core.CoreContainer.Deprecation" level="OFF"/>
+        <AsyncLogger name="org.apache.solr.core.CoreContainer.Deprecation" additivity="false" level="OFF">
+            <AppenderRef ref="null"/>
+        </AsyncLogger>
 
 
         <AsyncLogger name="com.google.inject.servlet" level="INFO"/>
         <AsyncLogger name="org.apache.solr.client.solrj.impl.Http2SolrClient" level="INFO"/>
 
         <AsyncRoot level="INFO">
-            <AppenderRef ref="STDERR_COLOR"/>
-            <AppenderRef ref="FILE"/>
+            <AppenderRef ref="STDERR"/>
          </AsyncRoot>
      </Loggers>
  </Configuration>