You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2019/10/02 09:22:31 UTC

[lucene-solr] 01/02: SOLR-13707, SOLR-13659, SOLR-13565, SOLR-13650, SOLR-13710, SOLR-13721, SOLR-13637: Reverted these commits for a fresh start

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

ishan pushed a commit to branch jira/SOLR-13661_2
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 956a503341a4516801248339a65b22f4d47d7184
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Wed Oct 2 14:51:12 2019 +0530

    SOLR-13707, SOLR-13659, SOLR-13565, SOLR-13650, SOLR-13710, SOLR-13721, SOLR-13637: Reverted these commits for a fresh start
---
 solr/CHANGES.txt                                   |  13 -
 solr/core/src/java/org/apache/solr/api/ApiBag.java |  26 +-
 .../org/apache/solr/cloud/ReplicateFromLeader.java |   2 +-
 .../solr/cloud/autoscaling/IndexSizeTrigger.java   |   2 +-
 .../java/org/apache/solr/core/BlobRepository.java  | 188 +------
 .../java/org/apache/solr/core/CoreContainer.java   |  15 +-
 .../java/org/apache/solr/core/MemClassLoader.java  |  25 +-
 .../java/org/apache/solr/core/PackageManager.java  | 370 -------------
 .../src/java/org/apache/solr/core/PluginBag.java   | 323 ++++++++----
 .../src/java/org/apache/solr/core/PluginInfo.java  |  79 +--
 .../src/java/org/apache/solr/core/RuntimeLib.java  | 227 --------
 .../src/java/org/apache/solr/core/SolrConfig.java  |  56 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |  42 +-
 .../org/apache/solr/core/SolrResourceLoader.java   |  27 +-
 .../apache/solr/handler/ReplicationHandler.java    |   2 +-
 .../apache/solr/handler/RequestHandlerBase.java    |  10 +-
 .../org/apache/solr/handler/SolrConfigHandler.java | 574 +++++++++------------
 .../solr/handler/admin/CollectionHandlerApi.java   | 313 ++---------
 .../solr/handler/admin/CollectionsHandler.java     |   2 +-
 .../handler/component/QueryElevationComponent.java |   2 +-
 .../solr/handler/component/SuggestComponent.java   |   2 +-
 .../org/apache/solr/metrics/SolrMetricManager.java |  17 +-
 .../apache/solr/metrics/SolrMetricProducer.java    |  12 +-
 .../solr/rest/schema/FieldTypeXmlAdapter.java      |   5 +-
 .../java/org/apache/solr/search/CacheConfig.java   | 192 ++++---
 .../java/org/apache/solr/search/FastLRUCache.java  |  37 +-
 .../src/java/org/apache/solr/search/LFUCache.java  |   6 +-
 .../src/java/org/apache/solr/search/LRUCache.java  |  11 +-
 .../src/java/org/apache/solr/search/SolrCache.java |  25 +-
 .../org/apache/solr/search/SolrCacheHolder.java    |  82 +--
 .../apache/solr/search/SolrDocumentFetcher.java    |   3 +-
 .../org/apache/solr/search/SolrIndexSearcher.java  |  28 +-
 .../apache/solr/security/AuthenticationPlugin.java |  17 +-
 .../solr/security/PermissionNameProvider.java      |   3 -
 .../apache/solr/servlet/SolrDispatchFilter.java    |  12 +-
 .../processor/UpdateRequestProcessorChain.java     |  16 +-
 .../src/java/org/apache/solr/util/CryptoKeys.java  |  22 +-
 .../src/test-files/cryptokeys/priv_key2048.pem     |  27 -
 .../core/src/test-files/cryptokeys/priv_key512.pem |   9 -
 .../core/src/test-files/cryptokeys/pub_key2048.der | Bin 294 -> 0 bytes
 solr/core/src/test-files/cryptokeys/pub_key512.der | Bin 94 -> 0 bytes
 .../src/test-files/runtimecode/MyDocCache.java     |  35 --
 solr/core/src/test-files/runtimecode/cache.jar.bin | Bin 820 -> 0 bytes
 .../src/test-files/runtimecode/cache_v2.jar.bin    | Bin 818 -> 0 bytes
 .../test-files/runtimecode/runtimelibs_v3.jar.bin  | Bin 7337 -> 0 bytes
 solr/core/src/test-files/runtimecode/sig.txt       |  97 ----
 .../apache/solr/cloud/TestClusterProperties.java   |   6 +-
 .../test/org/apache/solr/cloud/TestCryptoKeys.java |   2 +-
 .../solr/core/BlobRepositoryMockingTest.java       |  22 +-
 .../org/apache/solr/core/TestDynamicLoading.java   |  10 +-
 .../apache/solr/core/TestDynamicLoadingUrl.java    |   8 +-
 .../apache/solr/core/TestSolrConfigHandler.java    |   6 +-
 .../solr/handler/TestContainerReqHandler.java      | 483 +++--------------
 .../solr/handler/admin/MetricsHandlerTest.java     |  22 +-
 .../solr/handler/admin/TestApiFramework.java       |  57 +-
 .../test/org/apache/solr/search/TestLFUCache.java  |   6 +-
 .../solr/security/BasicAuthIntegrationTest.java    |   4 +-
 .../apache/solr/update/processor/RuntimeUrp.java   |   2 +-
 .../adding-custom-plugins-in-solrcloud-mode.adoc   |  16 +-
 .../org/apache/solr/client/solrj/SolrResponse.java |   9 +-
 .../client/solrj/request/CollectionApiMapping.java |  17 -
 .../solr/common/cloud/ClusterProperties.java       |  14 +-
 .../apache/solr/common/cloud/ZkStateReader.java    |  90 ++--
 .../apache/solr/common/params/CommonParams.java    |   2 -
 .../apache/solr/common/util/CommandOperation.java  |   9 +-
 .../org/apache/solr/common/util/ExecutorUtil.java  |   1 -
 .../java/org/apache/solr/common/util/StrUtils.java |  11 +-
 .../java/org/apache/solr/common/util/Utils.java    | 337 ++++++------
 .../src/resources/apispec/cluster.Commands.json    |  41 --
 .../cluster.Commands.runtimelib.properties.json    |  23 -
 ...nfig.Commands.addRequestHandler.properties.json |   2 +-
 .../apispec/core.config.Commands.generic.json      |   2 +-
 solr/solrj/src/resources/apispec/core.config.json  |   3 +-
 .../solrj/src/resources/apispec/node.blob.GET.json |  11 -
 solr/solrj/src/resources/apispec/node.ext.json     |  13 -
 .../solr/cloud/AbstractFullDistribZkTestBase.java  |   4 +-
 .../org/apache/solr/cloud/SolrCloudTestCase.java   |  29 +-
 77 files changed, 1188 insertions(+), 3030 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index bc27b0a..51436a6 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -113,8 +113,6 @@ New Features
   when using compositeIds.  Document distribution is calculated using the "id_prefix" field (if it exists) containing
   just the compositeId prefixes, or directly from the indexed "id" field otherwise. (yonik, Megan Carey)
 
-* SOLR-13565: Node level runtime libs loaded from remote urls  (noble)
-
 * SOLR-13553: Node level custom RequestHandlers (noble)
 
 * SOLR-13622: Add cat() stream source to create tuples from lines in local files (Jason Gerlowski and Joel Bernstein)
@@ -124,16 +122,9 @@ New Features
 
 * SOLR-13682: command line option to export documents to a file (noble)
 
-* SOLR-13650: Solr now can define and add "packages" with plugins. Each plugin can choose to
-  load from one of those packages & updating packages can reload those plugins independently (noble)
-
 * SOLR-13257: Support deterministic replica routing preferences for better cache usage (Michael Gibney
   via Christine Poerschke, Tomás Fernández Löbbe)
 
-* SOLR-13707: API to expose the currently used package name, details for each plugin (noble)
-
-* SOLR-13710: Persist package jars locally & expose them over http at /api/node/blob  (noble)
-
 * SOLR-13122: Ability to query aliases in Solr Admin UI (janhoy)
 
 * SOLR-13713: JWTAuthPlugin to support multiple JWKS endpoints (janhoy)
@@ -160,8 +151,6 @@ Improvements
 
 * SOLR-6305: Ability to set the replication factor for index files created by HDFSDirectoryFactory (Boris Pasko via Kevin Risden)
 
-* SOLR-13677: All Metrics Gauges should be unregistered by the objects that registered them (noble)
-
 * SOLR-13702: Some components register twice their metric names (janhoy)
 
 * SOLR-11601: Improved error message when geodist(llpsf) is used with arguments referring to a LatLonPointSpatialField.
@@ -277,8 +266,6 @@ Other Changes
 
 * SOLR-13643: Add Getters/Setters in ResponseBuilder for analytics response handling (Neal Sidhwaney via Munendra S N)
 
-* SOLR-13659: Refactor CacheConfig to lazily load the the implementation class (noble)
-
 * SOLR-13680: Use try-with-resource to close the closeable resource (Furkan KAMACI, Munendra S N)
 
 * SOLR-13573: Add SolrRangeQuery getters for upper, lower bound (Brian Rhees via Jason Gerlowski)
diff --git a/solr/core/src/java/org/apache/solr/api/ApiBag.java b/solr/core/src/java/org/apache/solr/api/ApiBag.java
index bfeb0ef..8a3f972 100644
--- a/solr/core/src/java/org/apache/solr/api/ApiBag.java
+++ b/solr/core/src/java/org/apache/solr/api/ApiBag.java
@@ -230,28 +230,22 @@ public class ApiBag {
   }
 
   public static class ReqHandlerToApi extends Api implements PermissionNameProvider {
-     PluginBag.PluginHolder<SolrRequestHandler> rh;
+    SolrRequestHandler rh;
 
     public ReqHandlerToApi(SolrRequestHandler rh, SpecProvider spec) {
       super(spec);
-      this.rh = new PluginBag.PluginHolder(new PluginInfo(SolrRequestHandler.TYPE, Collections.emptyMap()),rh );
-    }
-
-    public ReqHandlerToApi(PluginBag.PluginHolder<SolrRequestHandler> rh, SpecProvider spec) {
-      super(spec);
       this.rh = rh;
     }
 
     @Override
     public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
-      rh.get().handleRequest(req, rsp);
+      rh.handleRequest(req, rsp);
     }
 
     @Override
     public Name getPermissionName(AuthorizationContext ctx) {
-      SolrRequestHandler handler = rh.get();
-      if (handler instanceof PermissionNameProvider) {
-        return ((PermissionNameProvider) handler).getPermissionName(ctx);
+      if (rh instanceof PermissionNameProvider) {
+        return ((PermissionNameProvider) rh).getPermissionName(ctx);
       }
       return null;
     }
@@ -345,22 +339,22 @@ public class ApiBag {
   }
 
   public static class LazyLoadedApi extends Api {
+
+    private final PluginBag.PluginHolder<SolrRequestHandler> holder;
     private Api delegate;
 
     protected LazyLoadedApi(SpecProvider specProvider, PluginBag.PluginHolder<SolrRequestHandler> lazyPluginHolder) {
       super(specProvider);
-      delegate =  new ReqHandlerToApi(lazyPluginHolder, spec);
+      this.holder = lazyPluginHolder;
     }
 
     @Override
     public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+      if (!holder.isLoaded()) {
+        delegate = new ReqHandlerToApi(holder.get(), ApiBag.EMPTY_SPEC);
+      }
       delegate.call(req, rsp);
     }
-
-    @Override
-    public ValidatingJsonMap getSpec() {
-      return super.getSpec();
-    }
   }
 
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java b/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
index 17a6ec3..957b321 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
@@ -133,7 +133,7 @@ public class ReplicateFromLeader {
 
   public void stopReplication() {
     if (replicationProcess != null) {
-      replicationProcess.shutdown();
+      replicationProcess.close();
     }
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
index 6023f43..f32669c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
@@ -24,13 +24,13 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.Locale;
 
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
diff --git a/solr/core/src/java/org/apache/solr/core/BlobRepository.java b/solr/core/src/java/org/apache/solr/core/BlobRepository.java
index ea2f6d7..24bb88e 100644
--- a/solr/core/src/java/org/apache/solr/core/BlobRepository.java
+++ b/solr/core/src/java/org/apache/solr/core/BlobRepository.java
@@ -16,20 +16,17 @@
  */
 package org.apache.solr.core;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
-import java.nio.file.Path;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
@@ -37,14 +34,10 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.regex.Pattern;
 
-import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
-import org.apache.lucene.util.IOUtils;
-import org.apache.solr.api.Api;
-import org.apache.solr.api.V2HttpCall;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -52,32 +45,22 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.handler.RequestHandlerBase;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.security.AuthorizationContext;
-import org.apache.solr.security.PermissionNameProvider;
 import org.apache.solr.util.SimplePostTool;
 import org.apache.zookeeper.server.ByteBufferInputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
 import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
 import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
 import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
-import static org.apache.solr.handler.ReplicationHandler.FILE_STREAM;
 
 /**
  * The purpose of this class is to store the Jars loaded in memory and to keep only one copy of the Jar in a single node.
  */
 public class BlobRepository {
-  private static final long MAX_JAR_SIZE = Long.parseLong(System.getProperty("runtime.lib.size", String.valueOf(5 * 1024 * 1024)));
+  private static final long MAX_JAR_SIZE = Long.parseLong(System.getProperty("runtme.lib.size", String.valueOf(5 * 1024 * 1024)));
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   static final Random RANDOM;
   static final Pattern BLOB_KEY_PATTERN_CHECKER = Pattern.compile(".*/\\d+");
@@ -105,14 +88,6 @@ public class BlobRepository {
     this.coreContainer = coreContainer;
   }
 
-  public Collection<String> getFiles() {
-    return Arrays.asList(getBlobsPath().toFile().list());
-  }
-
-  public Path getBlobsPath() {
-    return SolrResourceLoader.getBlobsDirPath(this.coreContainer.getResourceLoader().getInstancePath());
-  }
-
   // I wanted to {@link SolrCore#loadDecodeAndCacheBlob(String, Decoder)} below but precommit complains
 
   /**
@@ -141,12 +116,12 @@ public class BlobRepository {
     return getBlobIncRef(key.concat(decoder.getName()), () -> addBlob(key, decoder));
   }
 
-  BlobContentRef getBlobIncRef(String key, Decoder decoder, String url, String sha256) {
+  BlobContentRef getBlobIncRef(String key, Decoder decoder, String url, String sha512) {
     StringBuffer keyBuilder = new StringBuffer(key);
     if (decoder != null) keyBuilder.append(decoder.getName());
-    keyBuilder.append("/").append(sha256);
+    keyBuilder.append("/").append(sha512);
 
-    return getBlobIncRef(keyBuilder.toString(), () -> new BlobContent<>(key, fetchBlobAndVerify(key, url, sha256), decoder));
+    return getBlobIncRef(keyBuilder.toString(), () -> new BlobContent<>(key, fetchBlobAndVerify(key, url, sha512), decoder));
   }
 
   // do the actual work returning the appropriate type...
@@ -191,80 +166,34 @@ public class BlobRepository {
     return aBlob;
   }
 
-  static String INVALID_JAR_MSG = "Invalid jar from {0} , expected sha256 hash : {1} , actual : {2}";
-
-  private ByteBuffer fetchBlobAndVerify(String key, String url, String sha256) throws IOException {
-    ByteBuffer byteBuffer = null;
-    if (sha256 != null) {
-      byteBuffer = getFromLocalFs(sha256);
-    }
-    if (byteBuffer == null) byteBuffer = getAndValidate(key, url, sha256);
-    return byteBuffer;
-  }
+  static String INVALID_JAR_MSG = "Invalid jar from {0} , expected sha512 hash : {1} , actual : {2}";
 
-  private ByteBuffer getAndValidate(String key, String url, String sha256) throws IOException {
+  private ByteBuffer fetchBlobAndVerify(String key, String url, String sha512) {
     ByteBuffer byteBuffer = fetchFromUrl(key, url);
-    String computedDigest = sha256Digest(byteBuffer);
-    if (!computedDigest.equals(sha256)) {
-      throw new SolrException(SERVER_ERROR, StrUtils.formatString(INVALID_JAR_MSG, url, sha256, computedDigest));
-    }
-    File file = new File(getBlobsPath().toFile(), sha256);
-    try (FileOutputStream fos = new FileOutputStream(file)) {
-      fos.write(byteBuffer.array(), byteBuffer.arrayOffset(), byteBuffer.limit());
-      IOUtils.fsync(file.toPath(), false);
-    }
-    return byteBuffer;
-  }
+    String computedDigest = sha512Digest(byteBuffer);
+    if (!computedDigest.equals(sha512)) {
+      throw new SolrException(SERVER_ERROR, StrUtils.formatString(INVALID_JAR_MSG, url, sha512, computedDigest));
 
-  public String putBlob(InputStream is) throws SolrException {
-    byte[] b = new byte[(int) MAX_JAR_SIZE + 1];
-    String sha256 = null;
-    try {
-      int sz = is.read(b);
-
-      if (sz > MAX_JAR_SIZE)
-        throw new SolrException(BAD_REQUEST, "size is more than permitted , use system property runtime.lib.size to change it");
-      sha256 = sha256Digest(ByteBuffer.wrap(b, 0, sz));
-      File file = new File(getBlobsPath().toFile(), sha256);
-      try (FileOutputStream fos = new FileOutputStream(file)) {
-        fos.write(b, 0, sz);
-      }
-      IOUtils.fsync(file.toPath(), false);
-    } catch (IOException e) {
-      throw new SolrException(BAD_REQUEST, e);
-    }
-    return sha256;
-
-  }
-
-  private ByteBuffer getFromLocalFs(String sha256) throws IOException {
-    Path p = getBlobsPath();
-    File f = new File(p.toFile(), sha256);
-    if (!f.exists()) return null;
-    byte[] b = new byte[(int) f.length()];
-    try (FileInputStream fis = new FileInputStream(f)) {
-      fis.read(b);
-      ByteBuffer byteBuffer = ByteBuffer.wrap(b);
-      if (sha256.equals(sha256Digest(byteBuffer))) {
-        return byteBuffer;
-      } else {
-        return null;
-
-      }
     }
+    return byteBuffer;
   }
 
-  public static String sha256Digest(ByteBuffer buf) {
+  public static String sha512Digest(ByteBuffer byteBuffer) {
+    MessageDigest digest = null;
     try {
-      return DigestUtils.sha256Hex(new ByteBufferInputStream(ByteBuffer.wrap( buf.array(), buf.arrayOffset(), buf.limit())));
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to compute sha256", e);
+      digest = MessageDigest.getInstance("SHA-512");
+    } catch (NoSuchAlgorithmException e) {
+      //unlikely
+      throw new SolrException(SERVER_ERROR, e);
     }
+    digest.update(byteBuffer);
+    return String.format(
+        Locale.ROOT,
+        "%0128x",
+        new BigInteger(1, digest.digest()));
   }
 
 
-
-
   /**
    * Package local for unit tests only please do not use elsewhere
    */
@@ -285,14 +214,13 @@ public class BlobRepository {
       entity = response.getEntity();
       int statusCode = response.getStatusLine().getStatusCode();
       if (statusCode != 200) {
-        throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no such resource available: " + key + ", url : " + url);
+        throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no such blob or version available: " + key);
       }
 
       try (InputStream is = entity.getContent()) {
         b = SimplePostTool.inputStreamToByteArray(is, MAX_JAR_SIZE);
       }
     } catch (Exception e) {
-      log.error("Error loading resource " + url, e);
       if (e instanceof SolrException) {
         throw (SolrException) e;
       } else {
@@ -353,68 +281,6 @@ public class BlobRepository {
     }
   }
 
-  BlobRead blobRead = new BlobRead();
-
-
-  class BlobRead extends RequestHandlerBase implements PermissionNameProvider {
-
-
-    @Override
-    public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) {
-
-    }
-
-    @Override
-    public String getDescription() {
-      return "List fetch blobs";
-    }
-
-    @Override
-    public Name getPermissionName(AuthorizationContext request) {
-      return null;
-    }
-
-    @Override
-    public Collection<Api> getApis() {
-      return Collections.singleton(new Api(Utils.getSpec("node.blob.GET")) {
-        @Override
-        public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
-          String sha256 = ((V2HttpCall) req.getHttpSolrCall()).getUrlParts().get("sha256");
-          if (sha256 == null) {
-            rsp.add("blob", getFiles());
-          } else {
-            try {
-              ByteBuffer buf = getFromLocalFs(sha256);
-              if(buf == null){
-                throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No such blob");
-              } else {
-                ModifiableSolrParams solrParams = new ModifiableSolrParams();
-                solrParams.add(CommonParams.WT, FILE_STREAM);
-                req.setParams( SolrParams.wrapDefaults(solrParams, req.getParams()));
-                rsp.add(FILE_STREAM, (SolrCore.RawWriter) os -> os.write(buf.array(), buf.arrayOffset(), buf.limit()));
-              }
-
-            } catch (IOException e) {
-              throw new SolrException(SERVER_ERROR,e);
-            }
-          }
-
-        }
-      });
-    }
-
-    @Override
-    public Boolean registerV1() {
-      return Boolean.FALSE;
-    }
-
-    @Override
-    public Boolean registerV2() {
-      return Boolean.TRUE;
-    }
-  }
-
-
   public static class BlobContent<T> {
     public final String key;
     private final T content; // holds byte buffer or cached object, holding both is a waste of memory
@@ -468,7 +334,7 @@ public class BlobRepository {
   public static class BlobContentRef<T> {
     public final BlobContent<T> blob;
 
-    public BlobContentRef(BlobContent<T> blob) {
+    private BlobContentRef(BlobContent<T> blob) {
       this.blob = blob;
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index ac85f6d..b50ed6d 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -218,8 +218,6 @@ public class CoreContainer {
 
   protected volatile AutoscalingHistoryHandler autoscalingHistoryHandler;
 
-  private final PackageManager clusterPropertiesListener = new PackageManager(this);
-
 
   // Bits for the state variable.
   public final static long LOAD_COMPLETE = 0x1L;
@@ -627,7 +625,6 @@ public class CoreContainer {
 
     zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
     if (isZooKeeperAware()) {
-      getZkController().getZkStateReader().registerClusterPropertiesListener(clusterPropertiesListener);
       pkiAuthenticationPlugin = new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName(),
           (PublicKeyHandler) containerHandlers.get(PublicKeyHandler.PATH));
       pkiAuthenticationPlugin.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), metricTag, "/authentication/pki");
@@ -640,8 +637,6 @@ public class CoreContainer {
     reloadSecurityProperties();
     this.backupRepoFactory = new BackupRepositoryFactory(cfg.getBackupRepositoryPlugins());
 
-    containerHandlers.put("/ext", clusterPropertiesListener.extHandler);
-    containerHandlers.put("/blob-get", blobRepository.blobRead);
     createHandler(ZK_PATH, ZookeeperInfoHandler.class.getName(), ZookeeperInfoHandler.class);
     createHandler(ZK_STATUS_PATH, ZookeeperStatusHandler.class.getName(), ZookeeperStatusHandler.class);
     collectionsHandler = createHandler(COLLECTIONS_HANDLER_PATH, cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
@@ -1542,7 +1537,7 @@ public class CoreContainer {
       } catch (SolrCoreState.CoreIsClosedException e) {
         throw e;
       } catch (Exception e) {
-        coreInitFailures.put(cd.getName(), new CoreLoadFailure(cd, e));
+        coreInitFailures.put(cd.getName(), new CoreLoadFailure(cd, (Exception) e));
         throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to reload core [" + cd.getName() + "]", e);
       } finally {
         if (!success && newCore != null && newCore.getOpenCount() > 0) {
@@ -1787,14 +1782,6 @@ public class CoreContainer {
     return handler;
   }
 
-  public PluginBag<SolrRequestHandler> getContainerHandlers() {
-    return containerHandlers;
-  }
-
-  public PackageManager getPackageManager(){
-    return clusterPropertiesListener;
-  }
-
   public CoreAdminHandler getMultiCoreHandler() {
     return coreAdminHandler;
   }
diff --git a/solr/core/src/java/org/apache/solr/core/MemClassLoader.java b/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
index 997e0b4..cf6bb4d 100644
--- a/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
@@ -26,7 +26,6 @@ import java.security.CodeSource;
 import java.security.ProtectionDomain;
 import java.security.cert.Certificate;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -44,28 +43,20 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private boolean allJarsLoaded = false;
   private final SolrResourceLoader parentLoader;
-  private List<RuntimeLib> libs = new ArrayList<>();
+  private List<PluginBag.RuntimeLib> libs = new ArrayList<>();
   private Map<String, Class> classCache = new HashMap<>();
   private List<String> errors = new ArrayList<>();
 
 
-  public MemClassLoader(List<RuntimeLib> libs, SolrResourceLoader resourceLoader) {
+  public MemClassLoader(List<PluginBag.RuntimeLib> libs, SolrResourceLoader resourceLoader) {
     this.parentLoader = resourceLoader;
     this.libs = libs;
   }
 
-  public int getZnodeVersion(){
-    int result = -1;
-    for (RuntimeLib lib : libs) {
-      if(lib.znodeVersion > result) result = lib.znodeVersion;
-    }
-    return result;
-  }
-
   synchronized void loadRemoteJars() {
     if (allJarsLoaded) return;
     int count = 0;
-    for (RuntimeLib lib : libs) {
+    for (PluginBag.RuntimeLib lib : libs) {
       if (lib.getUrl() != null) {
         try {
           lib.loadJar();
@@ -79,13 +70,10 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
     if (count == libs.size()) allJarsLoaded = true;
   }
 
-  public Collection<String> getErrors(){
-    return errors;
-  }
   public synchronized void loadJars() {
     if (allJarsLoaded) return;
 
-    for (RuntimeLib lib : libs) {
+    for (PluginBag.RuntimeLib lib : libs) {
       try {
         lib.loadJar();
         lib.verify();
@@ -145,7 +133,7 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
 
     String path = name.replace('.', '/').concat(".class");
     ByteBuffer buf = null;
-    for (RuntimeLib lib : libs) {
+    for (PluginBag.RuntimeLib lib : libs) {
       try {
         buf = lib.getFileContent(path);
         if (buf != null) {
@@ -162,7 +150,7 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
 
   @Override
   public void close() throws Exception {
-    for (RuntimeLib lib : libs) {
+    for (PluginBag.RuntimeLib lib : libs) {
       try {
         lib.close();
       } catch (Exception e) {
@@ -188,7 +176,6 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
     try {
       return findClass(cname).asSubclass(expectedType);
     } catch (Exception e) {
-      log.error("Error loading class from runtime libs ", e);
       if (e instanceof SolrException) {
         throw (SolrException) e;
       } else {
diff --git a/solr/core/src/java/org/apache/solr/core/PackageManager.java b/solr/core/src/java/org/apache/solr/core/PackageManager.java
deleted file mode 100644
index 7eb00a5..0000000
--- a/solr/core/src/java/org/apache/solr/core/PackageManager.java
+++ /dev/null
@@ -1,370 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.core;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-import org.apache.lucene.analysis.util.ResourceLoader;
-import org.apache.solr.api.Api;
-import org.apache.solr.api.V2HttpCall;
-import org.apache.solr.common.MapWriter;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ClusterPropertiesListener;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CoreAdminParams;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.handler.RequestHandlerBase;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.security.AuthorizationContext;
-import org.apache.solr.security.PermissionNameProvider;
-import org.apache.solr.util.plugin.PluginInfoInitialized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.common.params.CommonParams.PACKAGE;
-import static org.apache.solr.common.params.CommonParams.VERSION;
-import static org.apache.solr.core.RuntimeLib.SHA256;
-
-public class PackageManager implements ClusterPropertiesListener {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  private final CoreContainer coreContainer;
-
-  private Map<String, Package> pkgs = new HashMap<>();
-
-  final ExtHandler extHandler;
-  private int myversion = -1;
-
-  public int getZNodeVersion(String pkg) {
-    Package p = pkgs.get(pkg);
-    return p == null ? -1 : p.lib.getZnodeVersion();
-  }
-  public RuntimeLib getLib(String name){
-    Package p = pkgs.get(name);
-    return p == null? null: p.lib;
-  }
-
-  static class Package implements MapWriter {
-    final RuntimeLib lib;
-    final MemClassLoader loader;
-    final String name;
-
-    @Override
-    public void writeMap(EntryWriter ew) throws IOException {
-      lib.writeMap(ew);
-    }
-
-    Package(RuntimeLib lib, MemClassLoader loader, int zkVersion, String name) {
-      this.lib = lib;
-      this.loader = loader;
-      this.name = name;
-    }
-
-    public String getName() {
-      return name;
-    }
-
-
-    public boolean isModified(Map map) {
-      return (!Objects.equals(lib.getSha256(), (map).get(SHA256)) ||
-          !Objects.equals(lib.getSig(), (map).get(SHA256)));
-    }
-  }
-
-  PackageManager(CoreContainer coreContainer) {
-    this.coreContainer = coreContainer;
-    extHandler = new ExtHandler(this);
-  }
-
-
-  public <T> T newInstance(String cName, Class<T> expectedType, String pkg) {
-    try {
-      return coreContainer.getResourceLoader().newInstance(cName, expectedType,
-          null, new Class[]{CoreContainer.class}, new Object[]{coreContainer});
-    } catch (SolrException e) {
-      Package p = pkgs.get(pkg);
-
-      if (p != null) {
-        try {
-          Class<? extends T> klas = p.loader.findClass(cName, expectedType);
-          try {
-            return klas.getConstructor(CoreContainer.class).newInstance(coreContainer);
-          } catch (NoSuchMethodException ex) {
-            return klas.getConstructor().newInstance();
-          }
-        } catch (Exception ex) {
-          if (!p.loader.getErrors().isEmpty()) {
-            //some libraries were no loaded due to some errors. May the class was there in those libraries
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-                "There were errors loading some libraries: " + StrUtils.join(p.loader.getErrors(), ','), ex);
-          }
-          //there were no errors in loading any libraries. The class was probably not suppoed to be there in those libraries
-          // so throw the original exception
-          throw e;
-        }
-      } else {
-        throw e;
-      }
-    }
-  }
-
-  @Override
-  public boolean onChange(Map<String, Object> properties) {
-    log.info("clusterprops.json changed , version {}", coreContainer.getZkController().getZkStateReader().getClusterPropsVersion());
-    int v = coreContainer.getZkController().getZkStateReader().getClusterPropsVersion();
-    boolean modified = updatePackages(properties, v);
-    extHandler.updateReqHandlers(properties, modified);
-    for (SolrCore core : coreContainer.solrCores.getCores()) {
-      pkgs.forEach((s, pkg) -> core.packageUpdated(pkg.lib));
-    }
-    myversion = v;
-    return false;
-  }
-
-
-  private boolean updatePackages(Map<String, Object> properties, int ver) {
-    Map m = (Map) properties.getOrDefault(PACKAGE, Collections.emptyMap());
-    if (pkgs.isEmpty() && m.isEmpty()) return false;
-    boolean[] needsReload = new boolean[1];
-    if (m.size() == pkgs.size()) {
-      m.forEach((k, v) -> {
-        if (v instanceof Map) {
-          Package pkg = pkgs.get(k);
-          if (pkg == null || pkg.isModified((Map) v)) {
-            needsReload[0] = true;
-          }
-        }
-      });
-    } else {
-      needsReload[0] = true;
-    }
-    if (needsReload[0]) {
-      createNewClassLoaders(m, ver);
-    }
-    return needsReload[0];
-  }
-
-  public ResourceLoader getResourceLoader(String pkg) {
-    Package p = pkgs.get(pkg);
-    return p == null ? coreContainer.getResourceLoader() : p.loader;
-  }
-
-  void createNewClassLoaders(Map m, int ver) {
-    boolean[] loadedAll = new boolean[1];
-    loadedAll[0] = true;
-    Map<String, Package> newPkgs = new LinkedHashMap<>();
-    m.forEach((k, v) -> {
-      if (v instanceof Map) {
-        Map map = new HashMap((Map) v);
-        map.put(CoreAdminParams.NAME, String.valueOf(k));
-        String name = (String) k;
-        Package existing = pkgs.get(name);
-        if (existing != null && !existing.isModified(map)) {
-          //this package has not changed
-          newPkgs.put(name, existing);
-        }
-
-        RuntimeLib lib = new RuntimeLib(coreContainer);
-        lib.znodeVersion = ver;
-        try {
-          lib.init(new PluginInfo(RuntimeLib.TYPE, map));
-          if (lib.getUrl() == null) {
-            log.error("Unable to initialize runtimeLib : " + Utils.toJSONString(v));
-            loadedAll[0] = false;
-          }
-          lib.loadJar();
-
-          newPkgs.put(name, new Package(lib,
-              new MemClassLoader(Collections.singletonList(lib), coreContainer.getResourceLoader()),
-              ver, name));
-        } catch (Exception e) {
-          log.error("error loading a runtimeLib " + Utils.toJSONString(v), e);
-          loadedAll[0] = false;
-
-        }
-      }
-    });
-
-    if (loadedAll[0]) {
-      log.info("Libraries changed. New memclassloader created with jars {}",
-          newPkgs.values().stream().map(it -> it.lib.getUrl()).collect(Collectors.toList()));
-      this.pkgs = newPkgs;
-
-    }
-  }
-
-  static class ExtHandler extends RequestHandlerBase implements PermissionNameProvider {
-    final PackageManager packageManager;
-
-    private Map<String, Handler> customHandlers = new HashMap<>();
-
-    ExtHandler(PackageManager packageManager) {
-      this.packageManager = packageManager;
-    }
-
-
-    @Override
-    public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) {
-      int v = req.getParams().getInt(ConfigOverlay.ZNODEVER, -1);
-      if (v >= 0) {
-        log.debug("expected version : {} , my version {}", v, packageManager.myversion);
-        ZkStateReader zkStateReader = packageManager.coreContainer.getZkController().getZkStateReader();
-        try {
-          zkStateReader.forceRefreshClusterProps(v);
-        } catch (SolrException e) {
-          log.error("Error refreshing state ", e);
-          throw e;
-        }
-      }
-      rsp.add("metadata", (MapWriter) ew -> ew.putIfNotNull(VERSION,
-          packageManager.coreContainer.getZkController().zkStateReader.getClusterPropsVersion()));
-      rsp.add(RuntimeLib.TYPE, packageManager.pkgs.values());
-      rsp.add(SolrRequestHandler.TYPE, customHandlers.values());
-
-    }
-
-    @Override
-    public Collection<Api> getApis() {
-      return Collections.singleton(new Api(Utils.getSpec("node.ext")) {
-        @Override
-        public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
-          String name = ((V2HttpCall) req.getHttpSolrCall()).getUrlParts().get("handlerName");
-          if (name == null) {
-            handleRequestBody(req, rsp);
-            return;
-          }
-          Handler handler = customHandlers.get(name);
-          if (handler == null) {
-            String err = StrUtils.formatString(" No such handler: {0}, available handlers : {1}", name, customHandlers.keySet());
-            log.error(err);
-            throw new SolrException(SolrException.ErrorCode.NOT_FOUND, err);
-          }
-          handler.handler.handleRequest(req, rsp);
-        }
-      });
-    }
-
-    private void updateReqHandlers(Map<String, Object> properties, boolean forceReload) {
-      Map m = (Map) properties.getOrDefault(SolrRequestHandler.TYPE, Collections.emptyMap());
-      if (m.isEmpty() && customHandlers.isEmpty()) return;
-      boolean hasChanged = true;
-      if (customHandlers.size() == m.size() && customHandlers.keySet().containsAll(m.keySet())) hasChanged = false;
-      if (forceReload || hasChanged) {
-        log.debug("RequestHandlers being reloaded : {}", m.keySet());
-        Map<String, Handler> newCustomHandlers = new HashMap<>();
-        m.forEach((k, v) -> {
-          if (v instanceof Map) {
-            Map metaData = (Map) v;
-            Handler existing = customHandlers.get(k);
-            String name = (String) k;
-            if (existing == null || existing.shouldReload(metaData, packageManager.pkgs)) {
-              String klas = (String) metaData.get(FieldType.CLASS_NAME);
-              if (klas != null) {
-                String pkg = (String) metaData.get(PACKAGE);
-                SolrRequestHandler inst = packageManager.newInstance(klas, SolrRequestHandler.class, pkg);
-                if (inst instanceof PluginInfoInitialized) {
-                  ((PluginInfoInitialized) inst).init(new PluginInfo(SolrRequestHandler.TYPE, metaData));
-                }
-                Package p = packageManager.pkgs.get(pkg);
-                newCustomHandlers.put(name, new Handler(inst, pkg, p == null ? -1 : p.lib.getZnodeVersion(), metaData, name));
-              } else {
-                log.error("Invalid requestHandler {}", Utils.toJSONString(v));
-              }
-
-            } else {
-              newCustomHandlers.put(name, existing);
-            }
-
-          } else {
-            log.error("Invalid data for requestHandler : {} , {}", k, v);
-          }
-        });
-
-        log.debug("Registering request handlers {} ", newCustomHandlers.keySet());
-        Map<String, Handler> old = customHandlers;
-        customHandlers = newCustomHandlers;
-        old.forEach((s, h) -> PluginBag.closeQuietly(h));
-      }
-    }
-
-    @Override
-    public String getDescription() {
-      return "Custom Handlers";
-    }
-
-
-    @Override
-    public Boolean registerV1() {
-      return Boolean.FALSE;
-    }
-
-    @Override
-    public Boolean registerV2() {
-      return Boolean.TRUE;
-    }
-
-    @Override
-    public Name getPermissionName(AuthorizationContext request) {
-      if (request.getResource().endsWith("/node/ext")) return Name.COLL_READ_PERM;
-      return Name.CUSTOM_PERM;
-    }
-
-    static class Handler implements MapWriter {
-      final SolrRequestHandler handler;
-      final String pkg;
-      final int zkversion;
-      final Map meta;
-      final String name;
-
-      @Override
-      public void writeMap(EntryWriter ew) throws IOException {
-        ew.put(NAME, name);
-        ew.put(ConfigOverlay.ZNODEVER, zkversion);
-        meta.forEach(ew.getBiConsumer());
-      }
-
-      Handler(SolrRequestHandler handler, String pkg, int version, Map meta, String name) {
-        this.handler = handler;
-        this.pkg = pkg;
-        this.zkversion = version;
-        this.meta = Utils.getDeepCopy(meta, 3);
-        this.name = name;
-      }
-
-      public boolean shouldReload(Map metaData, Map<String, Package> pkgs) {
-        Package p = pkgs.get(pkg);
-        //the metadata is same and the package has not changed since we last loaded
-        return !meta.equals(metaData) || p == null || p.lib.getZnodeVersion() > zkversion;
-      }
-    }
-  }
-
-}
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 a9ea65b..6088f52 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -16,8 +16,11 @@
  */
 package org.apache.solr.core;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -27,21 +30,24 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
 
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.lucene.analysis.util.ResourceLoaderAware;
 import org.apache.solr.api.Api;
 import org.apache.solr.api.ApiBag;
 import org.apache.solr.api.ApiSupport;
-import org.apache.solr.common.MapWriter;
+import org.apache.solr.cloud.CloudUtil;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
+import org.apache.solr.util.CryptoKeys;
+import org.apache.solr.util.SimplePostTool;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
@@ -50,6 +56,7 @@ import org.slf4j.LoggerFactory;
 
 import static java.util.Collections.singletonMap;
 import static org.apache.solr.api.ApiBag.HANDLER_NAME;
+import static org.apache.solr.common.params.CommonParams.NAME;
 
 /**
  * This manages the lifecycle of a set of plugin of the same type .
@@ -117,36 +124,24 @@ public class PluginBag<T> implements AutoCloseable {
     return result;
   }
 
-  private static <T> T createInitInstance(PluginInfo pluginInfo, SolrConfig.SolrPluginInfo pluginMeta,
-                                          SolrCore core, ResourceLoader resourceLoader,
-                                          boolean isRuntimeLib) {
-    T localInst = null;
-    try {
-      localInst = (T) SolrCore.createInstance(pluginInfo.className, pluginMeta.clazz, pluginMeta.getCleanTag(), core, resourceLoader);
-    } catch (SolrException e) {
-      if (isRuntimeLib && !(resourceLoader instanceof MemClassLoader)) {
-        throw new SolrException(SolrException.ErrorCode.getErrorCode(e.code()),
-            e.getMessage() + ". runtime library loading is not enabled, start Solr with -Denable.runtime.lib=true",
-            e.getCause());
-      }
-      throw e;
-
+  public PluginHolder<T> createPlugin(PluginInfo info) {
+    if ("true".equals(String.valueOf(info.attributes.get("runtimeLib")))) {
+      log.debug(" {} : '{}'  created with runtimeLib=true ", meta.getCleanTag(), info.name);
+      LazyPluginHolder<T> holder = new LazyPluginHolder<>(meta, info, core, RuntimeLib.isEnabled() ?
+          core.getMemClassLoader() :
+          core.getResourceLoader(), true);
 
+      return meta.clazz == UpdateRequestProcessorFactory.class ?
+          (PluginHolder<T>) new UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder(holder) :
+          holder;
+    } else if ("lazy".equals(info.attributes.get("startup")) && meta.options.contains(SolrConfig.PluginOpts.LAZY)) {
+      log.debug("{} : '{}' created with startup=lazy ", meta.getCleanTag(), info.name);
+      return new LazyPluginHolder<T>(meta, info, core, core.getResourceLoader(), false);
+    } else {
+      T inst = core.createInstance(info.className, (Class<T>) meta.clazz, meta.getCleanTag(), null, core.getResourceLoader());
+      initInstance(inst, info);
+      return new PluginHolder<>(info, inst);
     }
-    initInstance(localInst, pluginInfo);
-    if (localInst instanceof SolrCoreAware) {
-      SolrResourceLoader.assertAwareCompatibility(SolrCoreAware.class, localInst);
-      ((SolrCoreAware) localInst).inform(core);
-    }
-    if (localInst instanceof ResourceLoaderAware) {
-      SolrResourceLoader.assertAwareCompatibility(ResourceLoaderAware.class, localInst);
-      try {
-        ((ResourceLoaderAware) localInst).inform(core.getResourceLoader());
-      } catch (IOException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "error initializing component", e);
-      }
-    }
-    return localInst;
   }
 
   /** make a plugin available in an alternate name. This is an internal API and not for public use
@@ -200,7 +195,7 @@ public class PluginBag<T> implements AutoCloseable {
     return old == null ? null : old.get();
   }
 
-  public PluginHolder<T> put(String name, PluginHolder<T> plugin) {
+  PluginHolder<T> put(String name, PluginHolder<T> plugin) {
     Boolean registerApi = null;
     Boolean disableHandler = null;
     if (plugin.pluginInfo != null) {
@@ -329,60 +324,13 @@ public class PluginBag<T> implements AutoCloseable {
     }
   }
 
-  public static void closeQuietly(Object inst)  {
-    try {
-      if (inst != null && inst instanceof AutoCloseable) ((AutoCloseable) inst).close();
-    } catch (Exception e) {
-      log.error("Error closing "+ inst , e);
-    }
-  }
-
-  public PluginHolder<T> createPlugin(PluginInfo info) {
-    String pkg = info.attributes.get(CommonParams.PACKAGE);
-    if (pkg != null) {
-      log.debug(" {} : '{}'  created with package={} ", meta.getCleanTag(), info.name, pkg);
-      PluginHolder<T> holder = new PackagePluginHolder<T>(info, core, meta);
-      return meta.clazz == UpdateRequestProcessorFactory.class ?
-          (PluginHolder<T>) new UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder((PluginHolder<UpdateRequestProcessorFactory>) holder) :
-          holder;
-
-    } else if (info.isRuntimePlugin()) {
-      log.debug(" {} : '{}'  created with runtimeLib=true ", meta.getCleanTag(), info.name);
-      LazyPluginHolder<T> holder = new LazyPluginHolder<>(meta, info, core, RuntimeLib.isEnabled() ?
-          core.getMemClassLoader() :
-          core.getResourceLoader(), true);
-
-      return meta.clazz == UpdateRequestProcessorFactory.class ?
-          (PluginHolder<T>) new UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder((PluginHolder<UpdateRequestProcessorFactory>) holder) :
-          holder;
-    } else if ("lazy".equals(info.attributes.get("startup")) && meta.options.contains(SolrConfig.PluginOpts.LAZY)) {
-      log.debug("{} : '{}' created with startup=lazy ", meta.getCleanTag(), info.name);
-      return new LazyPluginHolder<T>(meta, info, core, core.getResourceLoader(), false);
-    } else {
-      T inst = SolrCore.createInstance(info.className, (Class<T>) meta.clazz, meta.getCleanTag(), null, core.getResourceLoader());
-      initInstance(inst, info);
-      return new PluginHolder<>(info, inst);
-    }
-  }
-
-  public Api v2lookup(String path, String method, Map<String, String> parts) {
-    if (apiBag == null) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "this should not happen, looking up for v2 API at the wrong place");
-    }
-    return apiBag.lookup(path, method, parts);
-  }
-
-  public ApiBag getApiBag() {
-    return apiBag;
-  }
-
   /**
    * An indirect reference to a plugin. It just wraps a plugin instance.
    * subclasses may choose to lazily load the plugin
    */
   public static class PluginHolder<T> implements AutoCloseable {
+    private T inst;
     protected final PluginInfo pluginInfo;
-    T inst;
     boolean registerAPI = false;
 
     public PluginHolder(PluginInfo info) {
@@ -410,7 +358,7 @@ public class PluginBag<T> implements AutoCloseable {
       // can close() be called concurrently with other methods?
       if (isLoaded()) {
         T myInst = get();
-        closeQuietly(myInst);
+        if (myInst != null && myInst instanceof AutoCloseable) ((AutoCloseable) myInst).close();
       }
     }
 
@@ -478,62 +426,209 @@ public class PluginBag<T> implements AutoCloseable {
         MemClassLoader loader = (MemClassLoader) resourceLoader;
         loader.loadJars();
       }
-      lazyInst = createInitInstance(pluginInfo,pluginMeta,core,resourceLoader, isRuntimeLib);
+      Class<T> clazz = (Class<T>) pluginMeta.clazz;
+      T localInst = null;
+      try {
+        localInst = core.createInstance(pluginInfo.className, clazz, pluginMeta.getCleanTag(), null, resourceLoader);
+      } catch (SolrException e) {
+        if (isRuntimeLib && !(resourceLoader instanceof MemClassLoader)) {
+          throw new SolrException(SolrException.ErrorCode.getErrorCode(e.code()),
+              e.getMessage() + ". runtime library loading is not enabled, start Solr with -Denable.runtime.lib=true",
+              e.getCause());
+        }
+        throw e;
+
+
+      }
+      initInstance(localInst, pluginInfo);
+      if (localInst instanceof SolrCoreAware) {
+        SolrResourceLoader.assertAwareCompatibility(SolrCoreAware.class, localInst);
+        ((SolrCoreAware) localInst).inform(core);
+      }
+      if (localInst instanceof ResourceLoaderAware) {
+        SolrResourceLoader.assertAwareCompatibility(ResourceLoaderAware.class, localInst);
+        try {
+          ((ResourceLoaderAware) localInst).inform(core.getResourceLoader());
+        } catch (IOException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "error initializing component", e);
+        }
+      }
+      lazyInst = localInst;  // only assign the volatile until after the plugin is completely ready to use
       return true;
     }
 
 
   }
 
-  public class PackagePluginHolder<T> extends PluginHolder<T> {
-    private final SolrCore core;
-    private final SolrConfig.SolrPluginInfo pluginMeta;
-    private final PackageManager packageManager;
-    private final String pkg;
-    private RuntimeLib runtimeLib;
+  /**
+   * This represents a Runtime Jar. A jar requires two details , name and version
+   */
+  public static class RuntimeLib implements PluginInfoInitialized, AutoCloseable {
+    private String name, version, sig, sha512, url;
+    private BlobRepository.BlobContentRef<ByteBuffer> jarContent;
+    private final CoreContainer coreContainer;
+    private boolean verified = false;
 
-    public PackagePluginHolder(PluginInfo info, SolrCore core, SolrConfig.SolrPluginInfo pluginMeta) {
-      super(info);
-      this.core = core;
-      this.pluginMeta = pluginMeta;
-      this.pkg = info.attributes.get(CommonParams.PACKAGE);
-      this.core.addPackageListener(new SolrCore.PkgListener() {
-        @Override
-        public String packageName() {
-          return pkg;
+    @Override
+    public void init(PluginInfo info) {
+      name = info.attributes.get(NAME);
+      url = info.attributes.get("url");
+      sig = info.attributes.get("sig");
+      if(url == null) {
+        Object v = info.attributes.get("version");
+        if (name == null || v == null) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "runtimeLib must have name and version");
         }
+        version = String.valueOf(v);
+      } else {
+        sha512 = info.attributes.get("sha512");
+        if(sha512 == null){
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "runtimeLib with url must have a 'sha512' attribute");
+        }
+        ByteBuffer buf = null;
+        buf = coreContainer.getBlobRepository().fetchFromUrl(name, url);
 
-        @Override
-        public PluginInfo pluginInfo() {
-          return info;
+        String digest = BlobRepository.sha512Digest(buf);
+        if(!sha512.equals(digest))  {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, StrUtils.formatString(BlobRepository.INVALID_JAR_MSG, url, sha512, digest)  );
         }
+        log.info("dynamic library verified {}, sha512: {}", url, sha512);
+
+      }
+
+    }
+
+    public RuntimeLib(SolrCore core) {
+      coreContainer = core.getCoreContainer();
+    }
+
+    public String getUrl(){
+      return url;
+    }
+
+    void loadJar() {
+      if (jarContent != null) return;
+      synchronized (this) {
+        if (jarContent != null) return;
+
+        jarContent = url == null?
+            coreContainer.getBlobRepository().getBlobIncRef(name + "/" + version):
+            coreContainer.getBlobRepository().getBlobIncRef(name, null,url,sha512);
+
+      }
+    }
+
+    public static boolean isEnabled() {
+      return Boolean.getBoolean("enable.runtime.lib");
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    public String getVersion() {
+      return version;
+    }
+
+    public String getSig() {
+      return sig;
+
+    }
+
+    public ByteBuffer getFileContent(String entryName) throws IOException {
+      if (jarContent == null)
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "jar not available: " + name  );
+      return getFileContent(jarContent.blob, entryName);
+
+    }
 
-        @Override
-        public MapWriter lib() {
-          return runtimeLib;
+    public ByteBuffer getFileContent(BlobRepository.BlobContent<ByteBuffer> blobContent,  String entryName) throws IOException {
+      ByteBuffer buff = blobContent.get();
+      ByteArrayInputStream zipContents = new ByteArrayInputStream(buff.array(), buff.arrayOffset(), buff.limit());
+      ZipInputStream zis = new ZipInputStream(zipContents);
+      try {
+        ZipEntry entry;
+        while ((entry = zis.getNextEntry()) != null) {
+          if (entryName == null || entryName.equals(entry.getName())) {
+            SimplePostTool.BAOS out = new SimplePostTool.BAOS();
+            byte[] buffer = new byte[2048];
+            int size;
+            while ((size = zis.read(buffer, 0, buffer.length)) != -1) {
+              out.write(buffer, 0, size);
+            }
+            out.close();
+            return out.getByteBuffer();
+          }
         }
+      } finally {
+        zis.closeEntry();
+      }
+      return null;
+    }
+
+
+    @Override
+    public void close() throws Exception {
+      if (jarContent != null) coreContainer.getBlobRepository().decrementBlobRefCount(jarContent);
+    }
 
-        @Override
-        public void changed(RuntimeLib lib) {
-          int myVersion = runtimeLib == null? -1 : runtimeLib.znodeVersion;
-          if(lib.getZnodeVersion() > myVersion) reload();
+    public static List<RuntimeLib> getLibObjects(SolrCore core, List<PluginInfo> libs) {
+      List<RuntimeLib> l = new ArrayList<>(libs.size());
+      for (PluginInfo lib : libs) {
+        RuntimeLib rtl = new RuntimeLib(core);
+        try {
+          rtl.init(lib);
+        } catch (Exception e) {
+          log.error("error loading runtime library", e);
         }
-      });
-      this.packageManager = core.getCoreContainer().getPackageManager();
-      reload();
+        l.add(rtl);
+      }
+      return l;
     }
 
+    public void verify() throws Exception {
+      if (verified) return;
+      if (jarContent == null) {
+        log.error("Calling verify before loading the jar");
+        return;
+      }
 
-    private void reload() {
-      if(inst == null) log.info("reloading plugin {} ", pluginInfo.name);
-      inst = createInitInstance(pluginInfo, pluginMeta,
-          core, packageManager.getResourceLoader(this.pkg), true);
-      this.runtimeLib = packageManager.getLib(pkg);
+      if (!coreContainer.isZooKeeperAware())
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Signing jar is possible only in cloud");
+      Map<String, byte[]> keys = CloudUtil.getTrustedKeys(coreContainer.getZkController().getZkClient(), "exe");
+      if (keys.isEmpty()) {
+        if (sig == null) {
+          verified = true;
+          return;
+        } else {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No public keys are available in ZK to verify signature for runtime lib  " + name);
+        }
+      } else if (sig == null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, StrUtils.formatString("runtimelib {0} should be signed with one of the keys in ZK /keys/exe ", name));
+      }
 
+      try {
+        String matchedKey = new CryptoKeys(keys).verify(sig, jarContent.blob.get());
+        if (matchedKey == null)
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No key matched signature for jar : " + name + " version: " + version);
+        log.info("Jar {} signed with {} successfully verified", name, matchedKey);
+      } catch (Exception e) {
+        if (e instanceof SolrException) throw e;
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error verifying key ", e);
+      }
     }
+  }
 
 
+  public Api v2lookup(String path, String method, Map<String, String> parts) {
+    if (apiBag == null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "this should not happen, looking up for v2 API at the wrong place");
+    }
+    return apiBag.lookup(path, method, parts);
   }
 
+  public ApiBag getApiBag() {
+    return apiBag;
+  }
 
 }
diff --git a/solr/core/src/java/org/apache/solr/core/PluginInfo.java b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
index e25bd92..1bc85ae 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginInfo.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
@@ -16,23 +16,14 @@
  */
 package org.apache.solr.core;
 
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.solr.common.MapSerializable;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.DOMUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
+import java.util.*;
+
 import static java.util.Arrays.asList;
 import static java.util.Collections.unmodifiableList;
 import static java.util.Collections.unmodifiableMap;
@@ -40,26 +31,23 @@ import static org.apache.solr.common.params.CoreAdminParams.NAME;
 import static org.apache.solr.schema.FieldType.CLASS_NAME;
 
 /**
- * An Object which represents a Plugin of any type
+ * An Object which represents a Plugin of any type 
+ *
  */
 public class PluginInfo implements MapSerializable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   public final String name, className, type;
   public final NamedList initArgs;
   public final Map<String, String> attributes;
   public final List<PluginInfo> children;
   private boolean isFromSolrConfig;
 
-  public List<String> pathInConfig;
-
   public PluginInfo(String type, Map<String, String> attrs, NamedList initArgs, List<PluginInfo> children) {
     this.type = type;
     this.name = attrs.get(NAME);
     this.className = attrs.get(CLASS_NAME);
     this.initArgs = initArgs;
     attributes = unmodifiableMap(attrs);
-    this.children = children == null ? Collections.emptyList() : unmodifiableList(children);
+    this.children = children == null ? Collections.<PluginInfo>emptyList(): unmodifiableList(children);
     isFromSolrConfig = false;
   }
 
@@ -74,7 +62,7 @@ public class PluginInfo implements MapSerializable {
     isFromSolrConfig = true;
   }
 
-  public PluginInfo(String type, Map<String, Object> map) {
+  public PluginInfo(String type, Map<String,Object> map) {
     LinkedHashMap m = new LinkedHashMap<>(map);
     initArgs = new NamedList();
     for (Map.Entry<String, Object> entry : map.entrySet()) {
@@ -99,7 +87,7 @@ public class PluginInfo implements MapSerializable {
     this.name = (String) m.get(NAME);
     this.className = (String) m.get(CLASS_NAME);
     attributes = unmodifiableMap(m);
-    this.children = Collections.emptyList();
+    this.children =  Collections.<PluginInfo>emptyList();
     isFromSolrConfig = true;
   }
 
@@ -114,7 +102,7 @@ public class PluginInfo implements MapSerializable {
       PluginInfo pluginInfo = new PluginInfo(nd, null, false, false);
       if (pluginInfo.isEnabled()) children.add(pluginInfo);
     }
-    return children.isEmpty() ? Collections.emptyList() : unmodifiableList(children);
+    return children.isEmpty() ? Collections.<PluginInfo>emptyList() : unmodifiableList(children);
   }
 
   @Override
@@ -129,37 +117,37 @@ public class PluginInfo implements MapSerializable {
     return sb.toString();
   }
 
-  public boolean isEnabled() {
+  public boolean isEnabled(){
     String enable = attributes.get("enable");
-    return enable == null || Boolean.parseBoolean(enable);
+    return enable == null || Boolean.parseBoolean(enable); 
   }
 
   public boolean isDefault() {
     return Boolean.parseBoolean(attributes.get("default"));
   }
 
-  public PluginInfo getChild(String type) {
+  public PluginInfo getChild(String type){
     List<PluginInfo> l = getChildren(type);
-    return l.isEmpty() ? null : l.get(0);
+    return  l.isEmpty() ? null:l.get(0);
   }
 
   public Map<String, Object> toMap(Map<String, Object> map) {
     map.putAll(attributes);
     Map m = map;
-    if (initArgs != null) m.putAll(initArgs.asMap(3));
-    if (children != null) {
+    if(initArgs!=null ) m.putAll(initArgs.asMap(3));
+    if(children != null){
       for (PluginInfo child : children) {
         Object old = m.get(child.name);
-        if (old == null) {
+        if(old == null){
           m.put(child.name, child.toMap(new LinkedHashMap<>()));
         } else if (old instanceof List) {
           List list = (List) old;
           list.add(child.toMap(new LinkedHashMap<>()));
-        } else {
+        }  else {
           ArrayList l = new ArrayList();
           l.add(old);
           l.add(child.toMap(new LinkedHashMap<>()));
-          m.put(child.name, l);
+          m.put(child.name,l);
         }
       }
 
@@ -167,47 +155,36 @@ public class PluginInfo implements MapSerializable {
     return m;
   }
 
-  /**
-   * Filter children by type
-   *
+  /**Filter children by type
    * @param type The type name. must not be null
    * @return The mathcing children
    */
-  public List<PluginInfo> getChildren(String type) {
-    if (children.isEmpty()) return children;
+  public List<PluginInfo> getChildren(String type){
+    if(children.isEmpty()) return children;
     List<PluginInfo> result = new ArrayList<>();
-    for (PluginInfo child : children) if (type.equals(child.type)) result.add(child);
+    for (PluginInfo child : children) if(type.equals(child.type)) result.add(child);
     return result;
   }
-
-  public static final PluginInfo EMPTY_INFO = new PluginInfo("", Collections.emptyMap(), new NamedList(), Collections.emptyList());
+  public static final PluginInfo EMPTY_INFO = new PluginInfo("",Collections.<String,String>emptyMap(), new NamedList(),Collections.<PluginInfo>emptyList());
 
   private static final HashSet<String> NL_TAGS = new HashSet<>
-      (asList("lst", "arr",
-          "bool",
-          "str",
-          "int", "long",
-          "float", "double"));
+    (asList("lst", "arr",
+        "bool",
+        "str",
+        "int", "long",
+        "float", "double"));
   public static final String DEFAULTS = "defaults";
   public static final String APPENDS = "appends";
   public static final String INVARIANTS = "invariants";
 
-  public boolean isFromSolrConfig() {
+  public boolean isFromSolrConfig(){
     return isFromSolrConfig;
 
   }
-
   public PluginInfo copy() {
     PluginInfo result = new PluginInfo(type, attributes,
         initArgs != null ? initArgs.clone() : null, children);
     result.isFromSolrConfig = isFromSolrConfig;
-    result.pathInConfig = pathInConfig;
     return result;
   }
-
-  public boolean isRuntimePlugin() {
-    return "true".equals(String.valueOf(attributes.get(RuntimeLib.TYPE)))
-        || (attributes.get(CommonParams.PACKAGE) != null);
-  }
-
 }
diff --git a/solr/core/src/java/org/apache/solr/core/RuntimeLib.java b/solr/core/src/java/org/apache/solr/core/RuntimeLib.java
deleted file mode 100644
index 1e1f5f7..0000000
--- a/solr/core/src/java/org/apache/solr/core/RuntimeLib.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.core;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-
-import org.apache.solr.cloud.CloudUtil;
-import org.apache.solr.common.MapWriter;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.util.CryptoKeys;
-import org.apache.solr.util.SimplePostTool;
-import org.apache.solr.util.plugin.PluginInfoInitialized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.params.CommonParams.NAME;
-
-/**
- * This represents a Runtime Jar. A jar requires two details , name and version
- */
-public class RuntimeLib implements PluginInfoInitialized, AutoCloseable, MapWriter {
-  public static final String TYPE = "runtimeLib";
-  public static final String SHA256 = "sha256";
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private final CoreContainer coreContainer;
-  private String name, version, sig, sha256, url;
-  private BlobRepository.BlobContentRef<ByteBuffer> jarContent;
-  private boolean verified = false;
-  int znodeVersion = -1;
-
-  @Override
-  public void writeMap(EntryWriter ew) throws IOException {
-    ew.putIfNotNull(NAME, name);
-    ew.putIfNotNull("url", url);
-    ew.putIfNotNull(version, version);
-    ew.putIfNotNull("sha256", sha256);
-    ew.putIfNotNull("sig", sig);
-    if (znodeVersion > -1) {
-      ew.put(ConfigOverlay.ZNODEVER, znodeVersion);
-    }
-  }
-  public int getZnodeVersion(){
-    return znodeVersion;
-  }
-
-  public RuntimeLib(CoreContainer coreContainer) {
-    this.coreContainer = coreContainer;
-  }
-
-  public static boolean isEnabled() {
-    return "true".equals(System.getProperty("enable.runtime.lib"));
-  }
-
-  public static List<RuntimeLib> getLibObjects(SolrCore core, List<PluginInfo> libs) {
-    List<RuntimeLib> l = new ArrayList<>(libs.size());
-    for (PluginInfo lib : libs) {
-      RuntimeLib rtl = new RuntimeLib(core.getCoreContainer());
-      try {
-        rtl.init(lib);
-      } catch (Exception e) {
-        log.error("error loading runtime library", e);
-      }
-      l.add(rtl);
-    }
-    return l;
-  }
-
-  @Override
-  public void init(PluginInfo info) {
-    name = info.attributes.get(NAME);
-    url = info.attributes.get("url");
-    sig = info.attributes.get("sig");
-    if (url == null) {
-      Object v = info.attributes.get("version");
-      if (name == null || v == null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "runtimeLib must have name and version");
-      }
-      version = String.valueOf(v);
-    } else {
-      sha256 = info.attributes.get(SHA256);
-      if (sha256 == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "runtimeLib with url must have a 'sha256' attribute");
-      }
-      ByteBuffer buf = coreContainer.getBlobRepository().fetchFromUrl(name, url);
-
-      String digest = BlobRepository.sha256Digest(buf);
-      if (!sha256.equals(digest)) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, StrUtils.formatString(BlobRepository.INVALID_JAR_MSG, url, sha256, digest));
-      }
-      verifyJarSignature(buf);
-
-      log.debug("dynamic library verified {}, sha256: {}", url, sha256);
-
-    }
-
-  }
-
-  public String getUrl() {
-    return url;
-  }
-
-  void loadJar() {
-    if (jarContent != null) return;
-    synchronized (this) {
-      if (jarContent != null) return;
-
-      jarContent = url == null ?
-          coreContainer.getBlobRepository().getBlobIncRef(name + "/" + version) :
-          coreContainer.getBlobRepository().getBlobIncRef(name, null, url, sha256);
-
-    }
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public String getVersion() {
-    return version;
-  }
-
-  public String getSig() {
-    return sig;
-
-  }
-
-  public String getSha256() {
-    return sha256;
-  }
-
-  public ByteBuffer getFileContent(String entryName) throws IOException {
-    if (jarContent == null)
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "jar not available: " + name);
-    return getFileContent(jarContent.blob, entryName);
-
-  }
-
-  public ByteBuffer getFileContent(BlobRepository.BlobContent<ByteBuffer> blobContent, String entryName) throws IOException {
-    ByteBuffer buff = blobContent.get();
-    ByteArrayInputStream zipContents = new ByteArrayInputStream(buff.array(), buff.arrayOffset(), buff.limit());
-    ZipInputStream zis = new ZipInputStream(zipContents);
-    try {
-      ZipEntry entry;
-      while ((entry = zis.getNextEntry()) != null) {
-        if (entryName == null || entryName.equals(entry.getName())) {
-          SimplePostTool.BAOS out = new SimplePostTool.BAOS();
-          byte[] buffer = new byte[2048];
-          int size;
-          while ((size = zis.read(buffer, 0, buffer.length)) != -1) {
-            out.write(buffer, 0, size);
-          }
-          out.close();
-          return out.getByteBuffer();
-        }
-      }
-    } finally {
-      zis.closeEntry();
-    }
-    return null;
-  }
-
-  @Override
-  public void close() throws Exception {
-    if (jarContent != null) coreContainer.getBlobRepository().decrementBlobRefCount(jarContent);
-  }
-
-  public void verify() throws Exception {
-    if (verified) return;
-    if (jarContent == null) {
-      log.error("Calling verify before loading the jar");
-      return;
-    }
-
-    if (!coreContainer.isZooKeeperAware())
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Signing jar is possible only in cloud");
-    verifyJarSignature(jarContent.blob.get());
-  }
-
-  void verifyJarSignature(ByteBuffer buf) {
-    Map<String, byte[]> keys = CloudUtil.getTrustedKeys(coreContainer.getZkController().getZkClient(), "exe");
-    if (keys.isEmpty()) {
-      if (sig == null) {
-        verified = true;
-        return;
-      } else {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No public keys are available in ZK to verify signature for runtime lib  " + name);
-      }
-    } else if (sig == null) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, StrUtils.formatString("runtimelib {0} should be signed with one of the keys in ZK /keys/exe ", name));
-    }
-
-    try {
-      String matchedKey = new CryptoKeys(keys).verify(sig, buf);
-      if (matchedKey == null)
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No key matched signature for jar : " + name + " version: " + version);
-      log.info("Jar {} signed with {} successfully verified", name, matchedKey);
-    } catch (Exception e) {
-      log.error("Signature verifying error ", e);
-      if (e instanceof SolrException) throw (SolrException) e;
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error verifying key ", e);
-    }
-  }
-}
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 2ab7851..0c494a5 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -29,7 +29,6 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.text.ParseException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -56,7 +55,6 @@ import org.apache.solr.common.MapSerializable;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.QueryResponseWriter;
@@ -273,8 +271,7 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
       args.put("size", "10000");
       args.put("initialSize", "10");
       args.put("showItems", "-1");
-      args.put("class", FastLRUCache.class.getName());
-      conf = new CacheConfig(args,"query/fieldValueCache");
+      conf = new CacheConfig(FastLRUCache.class, args, null);
     }
     fieldValueCacheConfig = conf;
     useColdSearcher = getBool("query/useColdSearcher", false);
@@ -297,11 +294,11 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     slowQueryThresholdMillis = getInt("query/slowQueryThresholdMillis", -1);
     for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
 
-    Map<String, CacheConfig> userCacheConfigs = CacheConfig.getConfigs(this, "query/cache");
+    Map<String, CacheConfig> userCacheConfigs = CacheConfig.getMultipleConfigs(this, "query/cache");
     List<PluginInfo> caches = getPluginInfos(SolrCache.class.getName());
     if (!caches.isEmpty()) {
       for (PluginInfo c : caches) {
-        userCacheConfigs.put(c.name, new CacheConfig(c.attributes, StrUtils.join(c.pathInConfig, '/')));
+        userCacheConfigs.put(c.name, CacheConfig.getConfig(this, "cache", c.attributes, null));
       }
     }
     this.userCacheConfigs = Collections.unmodifiableMap(userCacheConfigs);
@@ -375,17 +372,17 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
       .add(new SolrPluginInfo(TransformerFactory.class, "transformer", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
       .add(new SolrPluginInfo(SearchComponent.class, "searchComponent", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
       .add(new SolrPluginInfo(UpdateRequestProcessorFactory.class, "updateProcessor", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
-      .add(new SolrPluginInfo(SolrCache.class, SolrCache.TYPE, REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
-      // TODO: WTF is up with queryConverter???
-      // it apparently *only* works as a singleton? - SOLR-4304
-      // and even then -- only if there is a single SpellCheckComponent
-      // because of queryConverter.setIndexAnalyzer
+      .add(new SolrPluginInfo(SolrCache.class, "cache", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+          // TODO: WTF is up with queryConverter???
+          // it apparently *only* works as a singleton? - SOLR-4304
+          // and even then -- only if there is a single SpellCheckComponent
+          // because of queryConverter.setIndexAnalyzer
       .add(new SolrPluginInfo(QueryConverter.class, "queryConverter", REQUIRE_NAME, REQUIRE_CLASS))
-      .add(new SolrPluginInfo(RuntimeLib.class, RuntimeLib.TYPE, REQUIRE_NAME, MULTI_OK))
-      // this is hackish, since it picks up all SolrEventListeners,
-      // regardless of when/how/why they are used (or even if they are
-      // declared outside of the appropriate context) but there's no nice
-      // way around that in the PluginInfo framework
+      .add(new SolrPluginInfo(PluginBag.RuntimeLib.class, "runtimeLib", REQUIRE_NAME, MULTI_OK))
+          // this is hackish, since it picks up all SolrEventListeners,
+          // regardless of when/how/why they are used (or even if they are
+          // declared outside of the appropriate context) but there's no nice
+          // way around that in the PluginInfo framework
       .add(new SolrPluginInfo(InitParams.class, InitParams.TYPE, MULTI_OK, REQUIRE_NAME_IN_OVERLAY))
       .add(new SolrPluginInfo(SolrEventListener.class, "//listener", REQUIRE_CLASS, MULTI_OK, REQUIRE_NAME_IN_OVERLAY))
 
@@ -535,9 +532,6 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
     NodeList nodes = (NodeList) evaluate(tag, XPathConstants.NODESET);
     for (int i = 0; i < nodes.getLength(); i++) {
       PluginInfo pluginInfo = new PluginInfo(nodes.item(i), "[solrconfig.xml] " + tag, requireName, requireClass);
-      if (requireName) {
-        pluginInfo.pathInConfig = Arrays.asList(tag, pluginInfo.name);
-      }
       if (pluginInfo.isEnabled()) result.add(pluginInfo);
     }
     return result;
@@ -611,7 +605,7 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
           "cacheControl", cacheControlHeader);
     }
 
-    public enum LastModFrom {
+    public static enum LastModFrom {
       OPENTIME, DIRLASTMOD, BOGUS;
 
       /**
@@ -763,24 +757,20 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
       Map<String, Map> infos = overlay.getNamedPlugins(info.getCleanTag());
       if (!infos.isEmpty()) {
         LinkedHashMap<String, PluginInfo> map = new LinkedHashMap<>();
-        if (result != null) {
-          for (PluginInfo pluginInfo : result) {
-            //just create a UUID for the time being so that map key is not null
-            String name = pluginInfo.name == null ?
-                UUID.randomUUID().toString().toLowerCase(Locale.ROOT) :
-                pluginInfo.name;
-            map.put(name, pluginInfo);
-          }
+        if (result != null) for (PluginInfo pluginInfo : result) {
+          //just create a UUID for the time being so that map key is not null
+          String name = pluginInfo.name == null ?
+              UUID.randomUUID().toString().toLowerCase(Locale.ROOT) :
+              pluginInfo.name;
+          map.put(name, pluginInfo);
         }
         for (Map.Entry<String, Map> e : infos.entrySet()) {
-          PluginInfo value = new PluginInfo(info.getCleanTag(), e.getValue());
-          value.pathInConfig = Arrays.asList(info.getCleanTag(),e.getKey());
-          map.put(e.getKey(), value);
+          map.put(e.getKey(), new PluginInfo(info.getCleanTag(), e.getValue()));
         }
         result = new ArrayList<>(map.values());
       }
     }
-    return result == null ? Collections.emptyList() : result;
+    return result == null ? Collections.<PluginInfo>emptyList() : result;
   }
 
   public PluginInfo getPluginInfo(String type) {
@@ -955,7 +945,7 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
 
   private void addCacheConfig(Map queryMap, CacheConfig... cache) {
     if (cache == null) return;
-    for (CacheConfig config : cache) if (config != null) queryMap.put(config.getName(), config);
+    for (CacheConfig config : cache) if (config != null) queryMap.put(config.getNodeName(), config);
 
   }
 
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 ef5fbb7..b730f3e 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -79,7 +79,6 @@ import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.RecoveryStrategy;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
-import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterState;
@@ -239,17 +238,11 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
   public volatile boolean searchEnabled = true;
   public volatile boolean indexEnabled = true;
   public volatile boolean readOnly = false;
-  private List<PkgListener> packageListeners = new ArrayList<>();
-
 
   public Set<String> getMetricNames() {
     return metricNames;
   }
 
-  public List<PkgListener> getPackageListeners(){
-    return Collections.unmodifiableList(packageListeners);
-  }
-
   public Date getStartTimeStamp() {
     return startTime;
   }
@@ -360,26 +353,6 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
     }
   }
 
-  void packageUpdated(RuntimeLib lib) {
-    for (PkgListener listener : packageListeners) {
-      if(lib.getName().equals(listener.packageName())) listener.changed(lib);
-    }
-  }
-  public void addPackageListener(PkgListener listener){
-    packageListeners.add(listener);
-  }
-
-  public interface PkgListener {
-
-    String packageName();
-
-    PluginInfo pluginInfo();
-
-    void changed(RuntimeLib lib);
-
-    MapWriter lib();
-  }
-
 
   /**
    * Returns the indexdir as given in index.properties. If index.properties exists in dataDir and
@@ -865,7 +838,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       for (Constructor<?> con : cons) {
         Class<?>[] types = con.getParameterTypes();
         if (types.length == 2 && types[0] == SolrCore.class && types[1] == UpdateHandler.class) {
-          return (UpdateHandler) con.newInstance(this, updateHandler);
+          return UpdateHandler.class.cast(con.newInstance(this, updateHandler));
         }
       }
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error Instantiating " + msg + ", " + className + " could not find proper constructor for " + UpdateHandler.class.getName());
@@ -885,12 +858,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
 
   public <T extends Object> T createInitInstance(PluginInfo info, Class<T> cast, String msg, String defClassName) {
     if (info == null) return null;
-    String pkg = info.attributes.get(CommonParams.PACKAGE);
-    ResourceLoader resourceLoader = pkg != null?
-        coreContainer.getPackageManager().getResourceLoader(pkg):
-        getResourceLoader();
-
-    T o = createInstance(info.className == null ? defClassName : info.className, cast, msg, this, resourceLoader);
+    T o = createInstance(info.className == null ? defClassName : info.className, cast, msg, this, getResourceLoader());
     if (o instanceof PluginInfoInitialized) {
       ((PluginInfoInitialized) o).init(info);
     } else if (o instanceof NamedListInitializedPlugin) {
@@ -998,7 +966,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       this.codec = initCodec(solrConfig, this.schema);
 
       memClassLoader = new MemClassLoader(
-          RuntimeLib.getLibObjects(this, solrConfig.getPluginInfos(RuntimeLib.class.getName())),
+          PluginBag.RuntimeLib.getLibObjects(this, solrConfig.getPluginInfos(PluginBag.RuntimeLib.class.getName())),
           getResourceLoader());
       initIndex(prev != null, reload);
 
@@ -2437,6 +2405,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
 
       if (!success) {
         newSearcherOtherErrorsCounter.inc();
+        ;
         synchronized (searcherLock) {
           onDeckSearchers--;
 
@@ -3139,7 +3108,8 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
     try {
       Stat stat = zkClient.exists(zkPath, null, true);
       if (stat == null) {
-        return currentVersion > -1;
+        if (currentVersion > -1) return true;
+        return false;
       }
       if (stat.getVersion() > currentVersion) {
         log.debug("{} is stale will need an update from {} to {}", zkPath, currentVersion, stat.getVersion());
diff --git a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
index b3dc5e4..f27edbc 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
@@ -576,8 +576,8 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
       }
     }
   }
-
-  static final String[] empty = new String[0];
+  
+  static final String empty[] = new String[0];
   
   @Override
   public <T> T newInstance(String name, Class<T> expectedType) {
@@ -808,7 +808,6 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
    * manipulated using select Solr features (e.g. streaming expressions).
    */
   public static final String USER_FILES_DIRECTORY = "userfiles";
-  public static final String BLOBS_DIRECTORY = "blobs";
   public static void ensureUserFilesDataDir(Path solrHome) {
     final Path userFilesPath = getUserFilesPath(solrHome);
     final File userFilesDirectory = new File(userFilesPath.toString());
@@ -824,28 +823,10 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
     }
   }
 
-  public static void ensureBlobsDir(Path solrHome) {
-    final Path blobsDir = getBlobsDirPath(solrHome);
-    final File blobsFilesDirectory = new File(blobsDir.toString());
-    if (! blobsFilesDirectory.exists()) {
-      try {
-        final boolean created = blobsFilesDirectory.mkdir();
-        if (! created) {
-          log.warn("Unable to create [{}] directory in SOLR_HOME [{}].  Features requiring this directory may fail.", BLOBS_DIRECTORY, solrHome);
-        }
-      } catch (Exception e) {
-          log.warn("Unable to create [" + BLOBS_DIRECTORY + "] directory in SOLR_HOME [" + solrHome + "].  Features requiring this directory may fail.", e);
-      }
-    }
-  }
-
-  public static Path getBlobsDirPath(Path solrHome) {
-    return Paths.get(solrHome.toAbsolutePath().toString(), BLOBS_DIRECTORY).toAbsolutePath();
-  }
-
   public static Path getUserFilesPath(Path solrHome) {
     return Paths.get(solrHome.toAbsolutePath().toString(), USER_FILES_DIRECTORY).toAbsolutePath();
   }
+
   // Logs a message only once per startup
   private static void logOnceInfo(String key, String msg) {
     if (!loggedOnce.contains(key)) {
@@ -942,7 +923,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
         }
       }
-      try (OutputStream out = new FileOutputStream(confFile)) {
+      try (OutputStream out = new FileOutputStream(confFile);) {
         out.write(content);
       }
       log.info("Written confile " + resourceName);
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index 5098a0d..dc1d1b1 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -1387,7 +1387,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     });
   }
 
-  public void shutdown() {
+  public void close() {
     if (executorService != null) executorService.shutdown();
     if (pollingIndexFetcher != null) {
       pollingIndexFetcher.destroy();
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index 212c30c..eca391b 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -22,14 +22,11 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.collect.ImmutableList;
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
-import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
-import com.google.common.collect.ImmutableList;
-import org.apache.solr.api.Api;
-import org.apache.solr.api.ApiBag;
-import org.apache.solr.api.ApiSupport;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
@@ -46,6 +43,9 @@ import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.SolrPluginUtils;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.ApiSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
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 789526e..11c6404 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -36,7 +36,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
-import java.util.function.Consumer;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -48,7 +47,6 @@ import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
-import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -64,9 +62,9 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.ConfigOverlay;
+import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.RequestParams;
-import org.apache.solr.core.RuntimeLib;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -152,262 +150,11 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
 
   public static boolean getImmutable(SolrCore core) {
     NamedList configSetProperties = core.getConfigSetProperties();
-    if (configSetProperties == null) return false;
+    if(configSetProperties == null) return false;
     Object immutable = configSetProperties.get(IMMUTABLE_CONFIGSET_ARG);
-    return immutable != null && Boolean.parseBoolean(immutable.toString());
+    return immutable != null ? Boolean.parseBoolean(immutable.toString()) : false;
   }
 
-  public static String validateName(String s) {
-    for (int i = 0; i < s.length(); i++) {
-      char c = s.charAt(i);
-      if ((c >= 'A' && c <= 'Z') ||
-          (c >= 'a' && c <= 'z') ||
-          (c >= '0' && c <= '9') ||
-          c == '_' ||
-          c == '-' ||
-          c == '.'
-      ) continue;
-      else {
-        return formatString("''{0}'' name should only have chars [a-zA-Z_-.0-9] ", s);
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Block up to a specified maximum time until we see agreement on the schema
-   * version in ZooKeeper across all replicas for a collection.
-   */
-  public static void waitForAllReplicasState(String collection,
-                                             ZkController zkController,
-                                             String prop,
-                                             int expectedVersion,
-                                             int maxWaitSecs) {
-    final RTimer timer = new RTimer();
-    // get a list of active replica cores to query for the schema zk version (skipping this core of course)
-    List<PerReplicaCallable> concurrentTasks = new ArrayList<>();
-
-    for (String coreUrl : getActiveReplicaCoreUrls(zkController, collection)) {
-      PerReplicaCallable e = new PerReplicaCallable(coreUrl, prop, expectedVersion, maxWaitSecs);
-      concurrentTasks.add(e);
-    }
-    if (concurrentTasks.isEmpty()) return; // nothing to wait for ...
-
-    log.info(formatString("Waiting up to {0} secs for {1} replicas to set the property {2} to be of version {3} for collection {4}",
-        maxWaitSecs, concurrentTasks.size(), prop, expectedVersion, collection));
-
-    // use an executor service to invoke schema zk version requests in parallel with a max wait time
-    execInparallel(concurrentTasks, parallelExecutor -> {
-      try {
-        List<String> failedList = executeAll(expectedVersion, maxWaitSecs, concurrentTasks, parallelExecutor);
-        // if any tasks haven't completed within the specified timeout, it's an error
-        if (failedList != null)
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-              formatString("{0} out of {1} the property {2} to be of version {3} within {4} seconds! Failed cores: {5}",
-                  failedList.size(), concurrentTasks.size() + 1, prop, expectedVersion, maxWaitSecs, failedList));
-      } catch (InterruptedException e) {
-        log.warn(formatString(
-            "Core  was interrupted . trying to set the property {0} to version {1} to propagate to {2} replicas for collection {3}",
-            prop, expectedVersion, concurrentTasks.size(), collection));
-        Thread.currentThread().interrupt();
-      }
-    });
-
-    log.info("Took {}ms to set the property {} to be of version {} for collection {}",
-        timer.getTime(), prop, expectedVersion, collection);
-  }
-
-  public static void execInparallel(List<? extends PerReplicaCallable> concurrentTasks, Consumer<ExecutorService> fun) {
-    int poolSize = Math.min(concurrentTasks.size(), 10);
-    ExecutorService parallelExecutor =
-        ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new DefaultSolrThreadFactory("solrHandlerExecutor"));
-    try {
-
-      fun.accept(parallelExecutor);
-
-    } finally {
-      ExecutorUtil.shutdownAndAwaitTermination(parallelExecutor);
-    }
-  }
-
-  @Override
-  public SolrRequestHandler getSubHandler(String path) {
-    if (subPaths.contains(path)) return this;
-    if (path.startsWith("/params/")) return this;
-    List<String> p = StrUtils.splitSmart(path, '/', true);
-    if (p.size() > 1) {
-      if (subPaths.contains("/" + p.get(0))) return this;
-    }
-    return null;
-  }
-
-
-  private static Set<String> subPaths = new HashSet<>(Arrays.asList("/overlay", "/params", "/updateHandler",
-      "/query", "/jmx", "/requestDispatcher", "/znodeVersion"));
-
-  static {
-    for (SolrConfig.SolrPluginInfo solrPluginInfo : SolrConfig.plugins)
-      subPaths.add("/" + solrPluginInfo.getCleanTag());
-
-  }
-
-  //////////////////////// SolrInfoMBeans methods //////////////////////
-
-
-  @Override
-  public String getDescription() {
-    return "Edit solrconfig.xml";
-  }
-
-  @Override
-  public Category getCategory() {
-    return Category.ADMIN;
-  }
-
-
-  public static final String SET_PROPERTY = "set-property";
-  public static final String UNSET_PROPERTY = "unset-property";
-  public static final String SET_USER_PROPERTY = "set-user-property";
-  public static final String UNSET_USER_PROPERTY = "unset-user-property";
-  public static final String SET = "set";
-  public static final String UPDATE = "update";
-  public static final String CREATE = "create";
-  private static Set<String> cmdPrefixes = ImmutableSet.of(CREATE, UPDATE, "delete", "add");
-
-  public static List<String> executeAll(int expectedVersion, int maxWaitSecs, List<? extends PerReplicaCallable> concurrentTasks, ExecutorService parallelExecutor) throws InterruptedException {
-    List<Future<Boolean>> results =
-        parallelExecutor.invokeAll(concurrentTasks, maxWaitSecs, TimeUnit.SECONDS);
-
-    // determine whether all replicas have the update
-    List<String> failedList = null; // lazily init'd
-    for (int f = 0; f < results.size(); f++) {
-      Boolean success = false;
-      Future<Boolean> next = results.get(f);
-      if (next.isDone() && !next.isCancelled()) {
-        // looks to have finished, but need to check if it succeeded
-        try {
-          success = next.get();
-        } catch (ExecutionException e) {
-          // shouldn't happen since we checked isCancelled
-        }
-      }
-
-      if (!success) {
-        String coreUrl = concurrentTasks.get(f).coreUrl;
-        log.warn("Core " + coreUrl + "could not get the expected version " + expectedVersion);
-        if (failedList == null) failedList = new ArrayList<>();
-        failedList.add(coreUrl);
-      }
-    }
-    return failedList;
-  }
-
-  public static class PerReplicaCallable extends SolrRequest implements Callable<Boolean> {
-    protected String coreUrl;
-    String prop;
-    protected int expectedZkVersion;
-    protected Number remoteVersion = null;
-    int maxWait;
-
-    public PerReplicaCallable(String coreUrl, String prop, int expectedZkVersion, int maxWait) {
-      super(METHOD.GET, "/config/" + ZNODEVER);
-      this.coreUrl = coreUrl;
-      this.expectedZkVersion = expectedZkVersion;
-      this.prop = prop;
-      this.maxWait = maxWait;
-    }
-
-    @Override
-    public SolrParams getParams() {
-      return new ModifiableSolrParams()
-          .set(prop, expectedZkVersion)
-          .set(CommonParams.WT, CommonParams.JAVABIN);
-    }
-
-    @Override
-    public Boolean call() throws Exception {
-      final RTimer timer = new RTimer();
-      int attempts = 0;
-      try (HttpSolrClient solr = new HttpSolrClient.Builder(coreUrl).build()) {
-        // eventually, this loop will get killed by the ExecutorService's timeout
-        while (true) {
-          try {
-            long timeElapsed = (long) timer.getTime() / 1000;
-            if (timeElapsed >= maxWait) {
-              return false;
-            }
-            log.info("Time elapsed : {} secs, maxWait {}", timeElapsed, maxWait);
-            Thread.sleep(100);
-            MapWriter resp = solr.httpUriRequest(this).future.get();
-            if (verifyResponse(resp, attempts)) break;
-            attempts++;
-          } catch (Exception e) {
-            if (e instanceof InterruptedException) {
-              break; // stop looping
-            } else {
-              log.warn("Failed to get /schema/zkversion from " + coreUrl + " due to: " + e);
-            }
-          }
-        }
-      }
-      return true;
-    }
-
-    protected boolean verifyResponse(MapWriter mw, int attempts) {
-      NamedList resp = (NamedList) mw;
-      if (resp != null) {
-        Map m = (Map) resp.get(ZNODEVER);
-        if (m != null) {
-          remoteVersion = (Number) m.get(prop);
-          if (remoteVersion != null && remoteVersion.intValue() >= expectedZkVersion) return true;
-          log.info(formatString("Could not get expectedVersion {0} from {1} for prop {2}   after {3} attempts", expectedZkVersion, coreUrl, prop, attempts));
-
-        }
-      }
-      return false;
-    }
-
-
-    @Override
-    protected SolrResponse createResponse(SolrClient client) {
-      return null;
-    }
-  }
-
-  public static List<String> getActiveReplicaCoreUrls(ZkController zkController,
-                                                      String collection) {
-    List<String> activeReplicaCoreUrls = new ArrayList<>();
-    ClusterState clusterState = zkController.getZkStateReader().getClusterState();
-    Set<String> liveNodes = clusterState.getLiveNodes();
-    final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
-    if (docCollection != null && docCollection.getActiveSlices() != null && docCollection.getActiveSlices().size() > 0) {
-      final Collection<Slice> activeSlices = docCollection.getActiveSlices();
-      for (Slice next : activeSlices) {
-        Map<String, Replica> replicasMap = next.getReplicasMap();
-        if (replicasMap != null) {
-          for (Map.Entry<String, Replica> entry : replicasMap.entrySet()) {
-            Replica replica = entry.getValue();
-            if (replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
-              activeReplicaCoreUrls.add(replica.getCoreUrl());
-            }
-          }
-        }
-      }
-    }
-    return activeReplicaCoreUrls;
-  }
-
-  @Override
-  public Name getPermissionName(AuthorizationContext ctx) {
-    switch (ctx.getHttpMethod()) {
-      case "GET":
-        return Name.CONFIG_READ_PERM;
-      case "POST":
-        return Name.CONFIG_EDIT_PERM;
-      default:
-        return null;
-    }
-  }
 
   private class Command {
     private final SolrQueryRequest req;
@@ -510,54 +257,25 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
 
     private Map<String, Object> getConfigDetails(String componentType, SolrQueryRequest req) {
       String componentName = componentType == null ? null : req.getParams().get("componentName");
-      if(componentName == null && parts.size() > 2){
-        componentName = parts.get(2);
-        if(SolrRequestHandler.TYPE.equals(componentType)){
-          componentName = "/"+componentName;
-        }
-      }
-
       boolean showParams = req.getParams().getBool("expandParams", false);
       Map<String, Object> map = this.req.getCore().getSolrConfig().toMap(new LinkedHashMap<>());
-      if (SolrRequestHandler.TYPE.equals(componentType) || componentType == null) {
-        Map reqHandlers = (Map) map.get(SolrRequestHandler.TYPE);
-        if (reqHandlers == null) map.put(SolrRequestHandler.TYPE, reqHandlers = new LinkedHashMap<>());
-        List<PluginInfo> plugins = this.req.getCore().getImplicitHandlers();
-        for (PluginInfo plugin : plugins) {
-          if (SolrRequestHandler.TYPE.equals(plugin.type)) {
-            if (!reqHandlers.containsKey(plugin.name)) {
-              reqHandlers.put(plugin.name, plugin);
-            }
+      if (componentType != null && !SolrRequestHandler.TYPE.equals(componentType)) return map;
+      Map reqHandlers = (Map) map.get(SolrRequestHandler.TYPE);
+      if (reqHandlers == null) map.put(SolrRequestHandler.TYPE, reqHandlers = new LinkedHashMap<>());
+      List<PluginInfo> plugins = this.req.getCore().getImplicitHandlers();
+      for (PluginInfo plugin : plugins) {
+        if (SolrRequestHandler.TYPE.equals(plugin.type)) {
+          if (!reqHandlers.containsKey(plugin.name)) {
+            reqHandlers.put(plugin.name, plugin);
           }
         }
-        if (showParams) {
-          for (Object o : reqHandlers.entrySet()) {
-            Map.Entry e = (Map.Entry) o;
-            if (componentName == null || e.getKey().equals(componentName)) {
-              Map<String, Object> m = expandUseParams(req, e.getValue());
-              e.setValue(m);
-            }
-          }
-        }
-
       }
-
-      if (req.getParams().getBool("meta", false)) {
-        for (SolrCore.PkgListener pkgListener : req.getCore().getPackageListeners()) {
-          PluginInfo meta = pkgListener.pluginInfo();
-          if (meta.pathInConfig != null) {
-            Object obj = Utils.getObjectByPath(map, false, meta.pathInConfig);
-            if (obj instanceof Map) {
-              Map m = (Map) obj;
-              m.put("_packageinfo_", pkgListener.lib());
-            } else if(obj instanceof MapWriter){
-              MapWriter mw = (MapWriter) obj;
-              Utils.setObjectByPath(map, meta.pathInConfig, (MapWriter) ew -> {
-                mw.writeMap(ew);
-                ew.put("_packageinfo_", pkgListener.lib());
-              }, false);
-            }
-          }
+      if (!showParams) return map;
+      for (Object o : reqHandlers.entrySet()) {
+        Map.Entry e = (Map.Entry) o;
+        if (componentName == null || e.getKey().equals(componentName)) {
+          Map<String, Object> m = expandUseParams(req, e.getValue());
+          e.setValue(m);
         }
       }
 
@@ -633,8 +351,6 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
           }
         }
       } catch (Exception e) {
-
-        log.error("error executing commands " + Utils.toJSONString(ops), e);
         resp.setException(e);
         resp.add(CommandOperation.ERR_MSGS, singletonList(SchemaManager.getErrorStr(e)));
       }
@@ -709,7 +425,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
 
       List errs = CommandOperation.captureErrors(ops);
       if (!errs.isEmpty()) {
-        throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "error processing params", errs);
+        throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST,"error processing params", errs);
       }
 
       SolrResourceLoader loader = req.getCore().getResourceLoader();
@@ -772,7 +488,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
       }
       List errs = CommandOperation.captureErrors(ops);
       if (!errs.isEmpty()) {
-        throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "error processing commands", errs);
+        throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST,"error processing commands", errs);
       }
 
       SolrResourceLoader loader = req.getCore().getResourceLoader();
@@ -810,20 +526,20 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
       op.getMap(PluginInfo.INVARIANTS, null);
       op.getMap(PluginInfo.APPENDS, null);
       if (op.hasError()) return overlay;
-      if (info.clazz == RuntimeLib.class) {
-        if (!RuntimeLib.isEnabled()) {
+      if(info.clazz == PluginBag.RuntimeLib.class) {
+        if(!PluginBag.RuntimeLib.isEnabled()){
           op.addError("Solr not started with -Denable.runtime.lib=true");
           return overlay;
         }
         try {
-          new RuntimeLib(req.getCore().getCoreContainer()).init(new PluginInfo(info.tag, op.getDataMap()));
+          new PluginBag.RuntimeLib(req.getCore()).init(new PluginInfo(info.tag, op.getDataMap()));
         } catch (Exception e) {
           op.addError(e.getMessage());
           log.error("can't load this plugin ", e);
           return overlay;
         }
       }
-      if (!verifyClass(op, clz, info)) return overlay;
+      if (!verifyClass(op, clz, info.clazz)) return overlay;
       if (pluginExists(info, overlay, name)) {
         if (isCeate) {
           op.addError(formatString(" ''{0}'' already exists . Do an ''{1}'' , if you want to change it ", name, "update-" + info.getTagCleanLower()));
@@ -843,23 +559,16 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
 
     private boolean pluginExists(SolrConfig.SolrPluginInfo info, ConfigOverlay overlay, String name) {
       List<PluginInfo> l = req.getCore().getSolrConfig().getPluginInfos(info.clazz.getName());
-      for (PluginInfo pluginInfo : l) if (name.equals(pluginInfo.name)) return true;
+      for (PluginInfo pluginInfo : l) if(name.equals( pluginInfo.name)) return true;
       return overlay.getNamedPlugins(info.getCleanTag()).containsKey(name);
     }
 
-    private boolean verifyClass(CommandOperation op, String clz, SolrConfig.SolrPluginInfo pluginMeta) {
+    private boolean verifyClass(CommandOperation op, String clz, Class expected) {
       if (clz == null) return true;
-      PluginInfo info = new PluginInfo(pluginMeta.getCleanTag(), op.getDataMap());
-
-      if (info.isRuntimePlugin() && !RuntimeLib.isEnabled()) {
-        op.addError("node not started with enable.runtime.lib=true");
-        return false;
-      }
-
-      if (!"true".equals(String.valueOf(op.getStr(RuntimeLib.TYPE, null)))) {
+      if (!"true".equals(String.valueOf(op.getStr("runtimeLib", null)))) {
         //this is not dynamically loaded so we can verify the class right away
         try {
-          req.getCore().createInitInstance(new PluginInfo(SolrRequestHandler.TYPE, op.getDataMap()), pluginMeta.clazz, clz, "");
+          req.getCore().createInitInstance(new PluginInfo(SolrRequestHandler.TYPE, op.getDataMap()), expected, clz, "");
         } catch (Exception e) {
           op.addError(e.getMessage());
           return false;
@@ -957,6 +666,235 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
 
   }
 
+  public static String validateName(String s) {
+    for (int i = 0; i < s.length(); i++) {
+      char c = s.charAt(i);
+      if ((c >= 'A' && c <= 'Z') ||
+          (c >= 'a' && c <= 'z') ||
+          (c >= '0' && c <= '9') ||
+          c == '_' ||
+          c == '-' ||
+          c == '.'
+          ) continue;
+      else {
+        return formatString("''{0}'' name should only have chars [a-zA-Z_-.0-9] ", s);
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public SolrRequestHandler getSubHandler(String path) {
+    if (subPaths.contains(path)) return this;
+    if (path.startsWith("/params/")) return this;
+    return null;
+  }
+
+
+  private static Set<String> subPaths = new HashSet<>(Arrays.asList("/overlay", "/params", "/updateHandler",
+      "/query", "/jmx", "/requestDispatcher", "/znodeVersion"));
+
+  static {
+    for (SolrConfig.SolrPluginInfo solrPluginInfo : SolrConfig.plugins)
+      subPaths.add("/" + solrPluginInfo.getCleanTag());
+
+  }
+
+  //////////////////////// SolrInfoMBeans methods //////////////////////
+
+
+  @Override
+  public String getDescription() {
+    return "Edit solrconfig.xml";
+  }
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
+
+  public static final String SET_PROPERTY = "set-property";
+  public static final String UNSET_PROPERTY = "unset-property";
+  public static final String SET_USER_PROPERTY = "set-user-property";
+  public static final String UNSET_USER_PROPERTY = "unset-user-property";
+  public static final String SET = "set";
+  public static final String UPDATE = "update";
+  public static final String CREATE = "create";
+  private static Set<String> cmdPrefixes = ImmutableSet.of(CREATE, UPDATE, "delete", "add");
+
+  /**
+   * Block up to a specified maximum time until we see agreement on the schema
+   * version in ZooKeeper across all replicas for a collection.
+   */
+  private static void waitForAllReplicasState(String collection,
+                                              ZkController zkController,
+                                              String prop,
+                                              int expectedVersion,
+                                              int maxWaitSecs) {
+    final RTimer timer = new RTimer();
+    // get a list of active replica cores to query for the schema zk version (skipping this core of course)
+    List<PerReplicaCallable> concurrentTasks = new ArrayList<>();
+
+    for (String coreUrl : getActiveReplicaCoreUrls(zkController, collection)) {
+      PerReplicaCallable e = new PerReplicaCallable(coreUrl, prop, expectedVersion, maxWaitSecs);
+      concurrentTasks.add(e);
+    }
+    if (concurrentTasks.isEmpty()) return; // nothing to wait for ...
+
+    log.info(formatString("Waiting up to {0} secs for {1} replicas to set the property {2} to be of version {3} for collection {4}",
+        maxWaitSecs, concurrentTasks.size(), prop, expectedVersion, collection));
+
+    // use an executor service to invoke schema zk version requests in parallel with a max wait time
+    int poolSize = Math.min(concurrentTasks.size(), 10);
+    ExecutorService parallelExecutor =
+        ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new DefaultSolrThreadFactory("solrHandlerExecutor"));
+    try {
+      List<Future<Boolean>> results =
+          parallelExecutor.invokeAll(concurrentTasks, maxWaitSecs, TimeUnit.SECONDS);
+
+      // determine whether all replicas have the update
+      List<String> failedList = null; // lazily init'd
+      for (int f = 0; f < results.size(); f++) {
+        Boolean success = false;
+        Future<Boolean> next = results.get(f);
+        if (next.isDone() && !next.isCancelled()) {
+          // looks to have finished, but need to check if it succeeded
+          try {
+            success = next.get();
+          } catch (ExecutionException e) {
+            // shouldn't happen since we checked isCancelled
+          }
+        }
+
+        if (!success) {
+          String coreUrl = concurrentTasks.get(f).coreUrl;
+          log.warn("Core " + coreUrl + "could not get the expected version " + expectedVersion);
+          if (failedList == null) failedList = new ArrayList<>();
+          failedList.add(coreUrl);
+        }
+      }
+
+      // if any tasks haven't completed within the specified timeout, it's an error
+      if (failedList != null)
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            formatString("{0} out of {1} the property {2} to be of version {3} within {4} seconds! Failed cores: {5}",
+                failedList.size(), concurrentTasks.size() + 1, prop, expectedVersion, maxWaitSecs, failedList));
+
+    } catch (InterruptedException ie) {
+      log.warn(formatString(
+          "Core  was interrupted . trying to set the property {1} to version {2} to propagate to {3} replicas for collection {4}",
+          prop, expectedVersion, concurrentTasks.size(), collection));
+      Thread.currentThread().interrupt();
+    } finally {
+      ExecutorUtil.shutdownAndAwaitTermination(parallelExecutor);
+    }
+
+    log.info("Took {}ms to set the property {} to be of version {} for collection {}",
+        timer.getTime(), prop, expectedVersion, collection);
+  }
+
+  public static List<String> getActiveReplicaCoreUrls(ZkController zkController,
+                                                      String collection) {
+    List<String> activeReplicaCoreUrls = new ArrayList<>();
+    ClusterState clusterState = zkController.getZkStateReader().getClusterState();
+    Set<String> liveNodes = clusterState.getLiveNodes();
+    final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
+    if (docCollection != null && docCollection.getActiveSlices() != null && docCollection.getActiveSlices().size() > 0) {
+      final Collection<Slice> activeSlices = docCollection.getActiveSlices();
+      for (Slice next : activeSlices) {
+        Map<String, Replica> replicasMap = next.getReplicasMap();
+        if (replicasMap != null) {
+          for (Map.Entry<String, Replica> entry : replicasMap.entrySet()) {
+            Replica replica = entry.getValue();
+            if (replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
+              activeReplicaCoreUrls.add(replica.getCoreUrl());
+            }
+          }
+        }
+      }
+    }
+    return activeReplicaCoreUrls;
+  }
+
+  @Override
+  public Name getPermissionName(AuthorizationContext ctx) {
+    switch (ctx.getHttpMethod()) {
+      case "GET":
+        return Name.CONFIG_READ_PERM;
+      case "POST":
+        return Name.CONFIG_EDIT_PERM;
+      default:
+        return null;
+    }
+  }
+
+  private static class PerReplicaCallable extends SolrRequest implements Callable<Boolean> {
+    String coreUrl;
+    String prop;
+    int expectedZkVersion;
+    Number remoteVersion = null;
+    int maxWait;
+
+    PerReplicaCallable(String coreUrl, String prop, int expectedZkVersion, int maxWait) {
+      super(METHOD.GET, "/config/" + ZNODEVER);
+      this.coreUrl = coreUrl;
+      this.expectedZkVersion = expectedZkVersion;
+      this.prop = prop;
+      this.maxWait = maxWait;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      return new ModifiableSolrParams()
+          .set(prop, expectedZkVersion)
+          .set(CommonParams.WT, CommonParams.JAVABIN);
+    }
+
+    @Override
+    public Boolean call() throws Exception {
+      final RTimer timer = new RTimer();
+      int attempts = 0;
+      try (HttpSolrClient solr = new HttpSolrClient.Builder(coreUrl).build()) {
+        // eventually, this loop will get killed by the ExecutorService's timeout
+        while (true) {
+          try {
+            long timeElapsed = (long) timer.getTime() / 1000;
+            if (timeElapsed >= maxWait) {
+              return false;
+            }
+            log.info("Time elapsed : {} secs, maxWait {}", timeElapsed, maxWait);
+            Thread.sleep(100);
+            NamedList<Object> resp = solr.httpUriRequest(this).future.get();
+            if (resp != null) {
+              Map m = (Map) resp.get(ZNODEVER);
+              if (m != null) {
+                remoteVersion = (Number) m.get(prop);
+                if (remoteVersion != null && remoteVersion.intValue() >= expectedZkVersion) break;
+              }
+            }
+
+            attempts++;
+            log.info(formatString("Could not get expectedVersion {0} from {1} for prop {2}   after {3} attempts", expectedZkVersion, coreUrl, prop, attempts));
+          } catch (Exception e) {
+            if (e instanceof InterruptedException) {
+              break; // stop looping
+            } else {
+              log.warn("Failed to get /schema/zkversion from " + coreUrl + " due to: " + e);
+            }
+          }
+        }
+      }
+      return true;
+    }
+
+
+    @Override
+    protected SolrResponse createResponse(SolrClient client) {
+      return null;
+    }
+  }
+
   @Override
   public Collection<Api> getApis() {
     return ApiBag.wrapRequestHandlers(this,
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
index 2259a0e..d7d179a 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
@@ -17,71 +17,40 @@
 
 package org.apache.solr.handler.admin;
 
-import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumMap;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 
-import org.apache.solr.api.ApiBag;
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionApiMapping;
 import org.apache.solr.client.solrj.request.CollectionApiMapping.CommandMeta;
 import org.apache.solr.client.solrj.request.CollectionApiMapping.Meta;
 import org.apache.solr.client.solrj.request.CollectionApiMapping.V2EndPoint;
-import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.Callable;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterProperties;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.CommandOperation;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.ConfigOverlay;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.PluginInfo;
-import org.apache.solr.core.RuntimeLib;
-import org.apache.solr.handler.SolrConfigHandler;
 import org.apache.solr.handler.admin.CollectionsHandler.CollectionOperation;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.RTimer;
-import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.util.Arrays.asList;
-import static java.util.Collections.singletonList;
-import static org.apache.solr.common.util.CommandOperation.captureErrors;
-import static org.apache.solr.common.util.StrUtils.formatString;
-import static org.apache.solr.core.RuntimeLib.SHA256;
-
 public class CollectionHandlerApi extends BaseHandlerApiSupport {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   final CollectionsHandler handler;
   static Collection<ApiCommand> apiCommands = createCollMapping();
 
-  public CollectionHandlerApi(CollectionsHandler handler) {
-    this.handler = handler;
-  }
-
   private static Collection<ApiCommand> createCollMapping() {
-    Map<Meta, ApiCommand> apiMapping = new EnumMap<>(Meta.class);
+    Map<Meta, ApiCommand> result = new EnumMap<>(Meta.class);
 
     for (Meta meta : Meta.values()) {
       for (CollectionOperation op : CollectionOperation.values()) {
         if (op.action == meta.action) {
-          apiMapping.put(meta, new ApiCommand() {
+          result.put(meta, new ApiCommand() {
             @Override
             public CommandMeta meta() {
               return meta;
@@ -96,209 +65,30 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
       }
     }
     //The following APIs have only V2 implementations
-    addApi(apiMapping, Meta.GET_NODES, CollectionHandlerApi::getNodes);
-    addApi(apiMapping, Meta.SET_CLUSTER_PROPERTY_OBJ, CollectionHandlerApi::setClusterObj);
-    addApi(apiMapping, Meta.ADD_PACKAGE, wrap(CollectionHandlerApi::addUpdatePackage));
-    addApi(apiMapping, Meta.UPDATE_PACKAGE, wrap(CollectionHandlerApi::addUpdatePackage));
-    addApi(apiMapping, Meta.DELETE_RUNTIME_LIB, wrap(CollectionHandlerApi::deletePackage));
-    addApi(apiMapping, Meta.ADD_REQ_HANDLER, wrap(CollectionHandlerApi::addRequestHandler));
-    addApi(apiMapping, Meta.DELETE_REQ_HANDLER, wrap(CollectionHandlerApi::deleteReqHandler));
-
-    for (Meta meta : Meta.values()) {
-      if (apiMapping.get(meta) == null) {
-        log.error("ERROR_INIT. No corresponding API implementation for : " + meta.commandName);
-      }
-    }
-
-    return apiMapping.values();
-  }
-
-  static Command wrap(Command cmd) {
-    return info -> {
-      CoreContainer cc = ((CollectionHandlerApi) info.apiHandler).handler.coreContainer;
-      boolean modified = cmd.call(info);
-      if (modified) {
-        Stat stat = new Stat();
-        Map<String, Object> clusterProperties = new ClusterProperties(cc.getZkController().getZkClient()).getClusterProperties(stat);
-        try {
-          cc.getPackageManager().onChange(clusterProperties);
-        } catch (SolrException e) {
-          log.error("error executing command : " + info.op.jsonStr(), e);
-          throw e;
-        } catch (Exception e) {
-          log.error("error executing command : " + info.op.jsonStr(), e);
-          throw new SolrException(ErrorCode.SERVER_ERROR, "error executing command : ", e);
-        }
-        log.info("current version of clusterprops.json is {} , trying to get every node to update ", stat.getVersion());
-        log.debug("The current clusterprops.json:  {}", clusterProperties);
-        ((CollectionHandlerApi) info.apiHandler).waitForStateSync(stat.getVersion(), cc);
-
-      }
-      if (info.op != null && info.op.hasError()) {
-        log.error("Error in running command {} , current clusterprops.json : {}", Utils.toJSONString(info.op), Utils.toJSONString(new ClusterProperties(cc.getZkController().getZkClient()).getClusterProperties()));
-      }
-      return modified;
-
-    };
-  }
-
-  private static boolean getNodes(ApiInfo params) {
-    params.rsp.add("nodes", ((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getClusterState().getLiveNodes());
-    return false;
-  }
-
-  private static boolean deleteReqHandler(ApiInfo params) throws Exception {
-    String name = params.op.getStr("");
-    ClusterProperties clusterProperties = new ClusterProperties(((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getZkClient());
-    Map<String, Object> map = clusterProperties.getClusterProperties();
-    if (Utils.getObjectByPath(map, false, asList(SolrRequestHandler.TYPE, name)) == null) {
-      params.op.addError("NO such requestHandler with name :");
-      return false;
-    }
-    Map m = new LinkedHashMap();
-    Utils.setObjectByPath(m, asList(SolrRequestHandler.TYPE, name), null, true);
-    clusterProperties.setClusterProperties(m);
-    return true;
-  }
+    addApi(result, Meta.GET_NODES, params -> params.rsp.add("nodes", ((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getClusterState().getLiveNodes()));
+    addApi(result, Meta.SET_CLUSTER_PROPERTY_OBJ, params -> {
+      List<CommandOperation> commands = params.req.getCommands(true);
+      if (commands == null || commands.isEmpty()) throw new RuntimeException("Empty commands");
+      ClusterProperties clusterProperties = new ClusterProperties(((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getZkClient());
 
-  private static boolean addRequestHandler(ApiInfo params) throws Exception {
-    Map data = params.op.getDataMap();
-    String name = (String) data.get("name");
-    CoreContainer coreContainer = ((CollectionHandlerApi) params.apiHandler).handler.coreContainer;
-    ClusterProperties clusterProperties = new ClusterProperties(coreContainer.getZkController().getZkClient());
-    Map<String, Object> map = clusterProperties.getClusterProperties();
-    if (Utils.getObjectByPath(map, false, asList(SolrRequestHandler.TYPE, name)) != null) {
-      params.op.addError("A requestHandler already exists with the said name");
-      return false;
-    }
-    Map m = new LinkedHashMap();
-    Utils.setObjectByPath(m, asList(SolrRequestHandler.TYPE, name), data, true);
-    clusterProperties.setClusterProperties(m);
-    return true;
-  }
-
-  private static boolean deletePackage(ApiInfo params) throws Exception {
-    if (!RuntimeLib.isEnabled()) {
-      params.op.addError("node not started with enable.runtime.lib=true");
-      return false;
-    }
-    String name = params.op.getStr(CommandOperation.ROOT_OBJ);
-    ClusterProperties clusterProperties = new ClusterProperties(((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getZkClient());
-    Map<String, Object> props = clusterProperties.getClusterProperties();
-    List<String> pathToLib = asList(CommonParams.PACKAGE, name);
-    Map existing = (Map) Utils.getObjectByPath(props, false, pathToLib);
-    if (existing == null) {
-      params.op.addError("No such runtimeLib : " + name);
-      return false;
-    }
-    Map delta = new LinkedHashMap();
-    Utils.setObjectByPath(delta, pathToLib, null, true);
-    clusterProperties.setClusterProperties(delta);
-    return true;
-  }
-
-  private static boolean addUpdatePackage(ApiInfo params) throws Exception {
-    if (!RuntimeLib.isEnabled()) {
-      params.op.addError("node not started with enable.runtime.lib=true");
-      return false;
-    }
-
-    CollectionHandlerApi handler = (CollectionHandlerApi) params.apiHandler;
-    RuntimeLib lib = new RuntimeLib(handler.handler.coreContainer);
-    CommandOperation op = params.op;
-    String name = op.getStr("name");
-    ClusterProperties clusterProperties = new ClusterProperties(((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getZkClient());
-    Map<String, Object> props = clusterProperties.getClusterProperties();
-    List<String> pathToLib = asList(CommonParams.PACKAGE, name);
-    Map existing = (Map) Utils.getObjectByPath(props, false, pathToLib);
-    if (Meta.ADD_PACKAGE.commandName.equals(op.name)) {
-      if (existing != null) {
-        op.addError(StrUtils.formatString("The jar with a name ''{0}'' already exists ", name));
-        return false;
-      }
-    } else {
-      if (existing == null) {
-        op.addError(StrUtils.formatString("The jar with a name ''{0}'' does not exist", name));
-        return false;
-      }
-      if (Objects.equals(existing.get(SHA256), op.getDataMap().get(SHA256))) {
-        op.addError("Trying to update a jar with the same sha256");
-        return false;
-      }
-    }
-    try {
-      lib.init(new PluginInfo(SolrRequestHandler.TYPE, op.getDataMap()));
-    } catch (SolrException e) {
-      log.error("Error loading runtimelib ", e);
-      op.addError(e.getMessage());
-      return false;
-    }
-
-    Map delta = new LinkedHashMap();
-    Utils.setObjectByPath(delta, pathToLib, op.getDataMap(), true);
-    clusterProperties.setClusterProperties(delta);
-    return true;
-
-  }
-
-  private static boolean setClusterObj(ApiInfo params) {
-    ClusterProperties clusterProperties = new ClusterProperties(((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getZkClient());
-    try {
-      clusterProperties.setClusterProperties(params.op.getDataMap());
-    } catch (Exception e) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Error in API", e);
-    }
-    return false;
-  }
-
-  private void waitForStateSync(int expectedVersion, CoreContainer coreContainer) {
-    final RTimer timer = new RTimer();
-    int waitTimeSecs = 30;
-    // get a list of active replica cores to query for the schema zk version (skipping this core of course)
-    List<PerNodeCallable> concurrentTasks = new ArrayList<>();
-
-    ZkStateReader zkStateReader = coreContainer.getZkController().getZkStateReader();
-    for (String nodeName : zkStateReader.getClusterState().getLiveNodes()) {
-      PerNodeCallable e = new PerNodeCallable(zkStateReader.getBaseUrlForNodeName(nodeName), expectedVersion, waitTimeSecs);
-      concurrentTasks.add(e);
-    }
-    if (concurrentTasks.isEmpty()) return; // nothing to wait for ...
-
-    log.info("Waiting up to {} secs for {} nodes to update clusterprops to be of version {} ",
-        waitTimeSecs, concurrentTasks.size(), expectedVersion);
-    SolrConfigHandler.execInparallel(concurrentTasks, parallelExecutor -> {
       try {
-        List<String> failedList = SolrConfigHandler.executeAll(expectedVersion, waitTimeSecs, concurrentTasks, parallelExecutor);
-
-        // if any tasks haven't completed within the specified timeout, it's an error
-        if (failedList != null)
-          throw new SolrException(ErrorCode.SERVER_ERROR,
-              formatString("{0} out of {1} the property {2} to be of version {3} within {4} seconds! Failed cores: {5}",
-                  failedList.size(), concurrentTasks.size() + 1, expectedVersion, 30, failedList));
-      } catch (InterruptedException e) {
-        log.warn(formatString(
-            "Request was interrupted . trying to set the clusterprops to version {0} to propagate to {1} nodes ",
-            expectedVersion, concurrentTasks.size()));
-        Thread.currentThread().interrupt();
-
+        clusterProperties.setClusterProperties(commands.get(0).getDataMap());
+      } catch (Exception e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error in API", e);
       }
     });
 
-    log.info("Took {}ms to update the clusterprops to be of version {}  on {} nodes",
-        timer.getTime(), expectedVersion, concurrentTasks.size());
-
-  }
-
-  interface Command {
-
-
-    boolean call(ApiInfo info) throws Exception;
+    for (Meta meta : Meta.values()) {
+      if (result.get(meta) == null) {
+        log.error("ERROR_INIT. No corresponding API implementation for : " + meta.commandName);
+      }
+    }
 
+    return result.values();
   }
 
-  private static void addApi(Map<Meta, ApiCommand> mapping, Meta metaInfo, Command fun) {
-    mapping.put(metaInfo, new ApiCommand() {
-
+  private static void addApi(Map<Meta, ApiCommand> result, Meta metaInfo, Callable<ApiParams> fun) {
+    result.put(metaInfo, new ApiCommand() {
       @Override
       public CommandMeta meta() {
         return metaInfo;
@@ -306,72 +96,35 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
 
       @Override
       public void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) throws Exception {
-        CommandOperation op = null;
-        if (metaInfo.method == SolrRequest.METHOD.POST) {
-          List<CommandOperation> commands = req.getCommands(true);
-          if (commands == null || commands.size() != 1)
-            throw new SolrException(ErrorCode.BAD_REQUEST, "should have exactly one command");
-          op = commands.get(0);
-        }
-
-        fun.call(new ApiInfo(req, rsp, apiHandler, op));
-        if (op != null && op.hasError()) {
-          throw new ApiBag.ExceptionWithErrObject(ErrorCode.BAD_REQUEST, "error processing commands", captureErrors(singletonList(op)));
-        }
+        fun.call(new ApiParams(req, rsp, apiHandler));
       }
     });
   }
 
-  @Override
-  protected List<V2EndPoint> getEndPoints() {
-    return asList(CollectionApiMapping.EndPoint.values());
-  }
-
-  @Override
-  protected Collection<ApiCommand> getCommands() {
-    return apiCommands;
-  }
-
-  public static class PerNodeCallable extends SolrConfigHandler.PerReplicaCallable {
-
-    static final List<String> path = Arrays.asList("metadata", CommonParams.VERSION);
-
-    PerNodeCallable(String baseUrl, int expectedversion, int waitTime) {
-      super(baseUrl, ConfigOverlay.ZNODEVER, expectedversion, waitTime);
-    }
-
-    @Override
-    protected boolean verifyResponse(MapWriter mw, int attempts) {
-      remoteVersion = (Number) mw._get(path, -1);
-      if (remoteVersion.intValue() >= expectedZkVersion) return true;
-      log.info(formatString("Could not get expectedVersion {0} from {1} , remote val= {2}   after {3} attempts", expectedZkVersion, coreUrl, remoteVersion, attempts));
-
-      return false;
-    }
-
-    public String getPath() {
-      return "/____v2/node/ext";
-    }
-  }
-
-  static class ApiInfo {
+  static class ApiParams {
     final SolrQueryRequest req;
     final SolrQueryResponse rsp;
     final BaseHandlerApiSupport apiHandler;
-    final CommandOperation op;
 
-    ApiInfo(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler, CommandOperation op) {
+    ApiParams(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) {
       this.req = req;
       this.rsp = rsp;
       this.apiHandler = apiHandler;
-      this.op = op;
     }
   }
 
-  public static void postBlob(String baseUrl, ByteBuffer buf) throws IOException {
-    try(HttpSolrClient client = new HttpSolrClient.Builder(baseUrl+"/____v2/node/blob" ).build()){
+  public CollectionHandlerApi(CollectionsHandler handler) {
+    this.handler = handler;
+  }
 
-    }
+  @Override
+  protected Collection<ApiCommand> getCommands() {
+    return apiCommands;
+  }
+
+  @Override
+  protected List<V2EndPoint> getEndPoints() {
+    return Arrays.asList(CollectionApiMapping.EndPoint.values());
   }
 
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 445c0c5..5843a94 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -149,10 +149,10 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonAdminParams.IN_PLACE_MOVE;
 import static org.apache.solr.common.params.CommonAdminParams.NUM_SUB_SHARDS;
-import static org.apache.solr.common.params.CommonAdminParams.SPLIT_BY_PREFIX;
 import static org.apache.solr.common.params.CommonAdminParams.SPLIT_FUZZ;
 import static org.apache.solr.common.params.CommonAdminParams.SPLIT_METHOD;
 import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
+import static org.apache.solr.common.params.CommonAdminParams.SPLIT_BY_PREFIX;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.params.CommonParams.TIMING;
 import static org.apache.solr.common.params.CommonParams.VALUE_LONG;
diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
index 203d8f7..4f4f232 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
@@ -79,8 +79,8 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.XmlConfigFile;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.transform.ElevatedMarkerFactory;
 import org.apache.solr.response.transform.ExcludedMarkerFactory;
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java b/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
index 3ede10d..2d6fdb1 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
@@ -90,7 +90,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
 
   protected SolrMetricManager metricManager;
   protected String registryName;
-
+  
   /**
    * Key is the dictionary name used in SolrConfig, value is the corresponding {@link SolrSuggester}
    */
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index f029e60..7d2877d 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -690,13 +690,13 @@ public class SolrMetricManager {
     }
   }
 
-    /**
-     * This is a wrapper for {@link Gauge} metrics, which are usually implemented as
-     * lambdas that often keep a reference to their parent instance. In order to make sure that
-     * all such metrics are removed when their parent instance is removed / closed the
-     * metric is associated with an instance tag, which can be used then to remove
-     * wrappers with the matching tag using {@link #unregisterGauges(String, String)}.
-     */
+  /**
+   * This is a wrapper for {@link Gauge} metrics, which are usually implemented as
+   * lambdas that often keep a reference to their parent instance. In order to make sure that
+   * all such metrics are removed when their parent instance is removed / closed the
+   * metric is associated with an instance tag, which can be used then to remove
+   * wrappers with the matching tag using {@link #unregisterGauges(String, String)}.
+   */
   public static class GaugeWrapper<T> implements Gauge<T> {
     private final Gauge<T> gauge;
     private final String tag;
@@ -736,7 +736,7 @@ public class SolrMetricManager {
         removed.incrementAndGet();
         return true;
       } else {
-      return false;
+        return false;
       }
     });
     return removed.get();
@@ -774,7 +774,6 @@ public class SolrMetricManager {
       sb.append(name);
       return sb.toString();
     }
-
   }
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
index deb2b18..d5c23b5 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
@@ -23,13 +23,13 @@ public interface SolrMetricProducer {
 
   /**
    * Initializes metrics specific to this producer
-   * @param manager  an instance of {@link SolrMetricManager}
+   * @param manager an instance of {@link SolrMetricManager}
    * @param registry registry name where metrics are registered
-   * @param tag      a symbolic tag that represents this instance of the producer,
-   *                 or a group of related instances that have the same life-cycle. This tag is
-   *                 used when managing life-cycle of some metrics and is set when
-   *                 {@link #initializeMetrics(SolrMetricManager, String, String, String)} is called.
-   * @param scope    scope of the metrics (eg. handler name) to separate metrics of
+   * @param tag a symbolic tag that represents this instance of the producer,
+   * or a group of related instances that have the same life-cycle. This tag is
+   * used when managing life-cycle of some metrics and is set when
+   * {@link #initializeMetrics(SolrMetricManager, String, String, String)} is called.
+   * @param scope scope of the metrics (eg. handler name) to separate metrics of
    */
   void initializeMetrics(SolrMetricManager manager, String registry, String tag, String scope);
 }
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
index f437296..4abeedd 100644
--- a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
+++ b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
@@ -16,11 +16,12 @@
  */
 package org.apache.solr.rest.schema;
 
+import java.util.List;
+import java.util.Map;
+
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
-import java.util.List;
-import java.util.Map;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
diff --git a/solr/core/src/java/org/apache/solr/search/CacheConfig.java b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
index 753762a..16a9d57 100644
--- a/solr/core/src/java/org/apache/solr/search/CacheConfig.java
+++ b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
@@ -14,150 +14,148 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.solr.search;
 
 import javax.xml.xpath.XPathConstants;
-import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.analysis.util.ResourceLoader;
-import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.ConfigOverlay;
-import org.apache.solr.core.MemClassLoader;
-import org.apache.solr.core.PluginInfo;
-import org.apache.solr.core.RuntimeLib;
+import org.apache.solr.common.MapSerializable;
 import org.apache.solr.core.SolrConfig;
-import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.DOMUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 
-public class CacheConfig implements MapWriter {
-  final PluginInfo args;
-  private CacheRegenerator defRegen;
-  private final String name;
-  private String cacheImpl, regenImpl;
-  Object[] persistence = new Object[1];
+/**
+ * Contains the knowledge of how cache config is
+ * stored in the solrconfig.xml file, and implements a
+ * factory to create caches.
+ *
+ *
+ */
+public class CacheConfig implements MapSerializable{
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  
+  private String nodeName;
 
+  private Class<? extends SolrCache> clazz;
+  private Map<String,String> args;
+  private CacheRegenerator regenerator;
 
-  public CacheConfig(Map<String, String> args, String path) {
-    this.args = new PluginInfo(SolrCache.TYPE, (Map) copyValsAsString(args));
-    this.name = args.get(NAME);
-    this.cacheImpl = args.getOrDefault("class", "solr.LRUCache");
-    this.regenImpl = args.get("regenerator");
-    this.args.pathInConfig = StrUtils.splitSmart(path, '/', true);
-  }
+  private String cacheImpl;
 
-  static Map<String, String> copyValsAsString(Map m) {
-    Map<String, String> copy = new LinkedHashMap(m.size());
-    m.forEach((k, v) -> copy.put(String.valueOf(k), String.valueOf(v)));
-    return copy;
-  }
+  private Object[] persistence = new Object[1];
 
-  public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
-    Node node = solrConfig.getNode(xpath, false);
-    if (node == null || !"true".equals(DOMUtil.getAttrOrDefault(node, "enabled", "true"))) {
-      Map<String, String> m = solrConfig.getOverlay().getEditableSubProperties(xpath);
-      if (m == null) return null;
-      List<String> pieces = StrUtils.splitSmart(xpath, '/');
-      String name = pieces.get(pieces.size() - 1);
-      m = Utils.getDeepCopy(m, 2);
-      m.put(NAME, name);
-      return new CacheConfig(m, xpath);
-    } else {
-      Map<String, String> attrs = DOMUtil.toMap(node.getAttributes());
-      attrs.put(NAME, node.getNodeName());
-      return new CacheConfig(applyOverlay(xpath, solrConfig.getOverlay(), attrs), xpath);
+  private String regenImpl;
 
-    }
+  public CacheConfig() {}
 
+  public CacheConfig(Class<? extends SolrCache> clazz, Map<String,String> args, CacheRegenerator regenerator) {
+    this.clazz = clazz;
+    this.args = args;
+    this.regenerator = regenerator;
+  }
 
+  public CacheRegenerator getRegenerator() {
+    return regenerator;
   }
 
-  private static Map applyOverlay(String xpath, ConfigOverlay overlay, Map args) {
-    Map<String, String> map = xpath == null ? null : overlay.getEditableSubProperties(xpath);
-    if (map != null) {
-      HashMap<String, String> mapCopy = new HashMap<>(args);
-      for (Map.Entry<String, String> e : map.entrySet()) {
-        mapCopy.put(e.getKey(), String.valueOf(e.getValue()));
-      }
-      return mapCopy;
-    }
-    return args;
+  public void setRegenerator(CacheRegenerator regenerator) {
+    this.regenerator = regenerator;
   }
 
-  public static Map<String, CacheConfig> getConfigs(SolrConfig solrConfig, String configPath) {
+  public static Map<String, CacheConfig> getMultipleConfigs(SolrConfig solrConfig, String configPath) {
     NodeList nodes = (NodeList) solrConfig.evaluate(configPath, XPathConstants.NODESET);
     if (nodes == null || nodes.getLength() == 0) return new LinkedHashMap<>();
     Map<String, CacheConfig> result = new HashMap<>(nodes.getLength());
     for (int i = 0; i < nodes.getLength(); i++) {
-      Map<String, String> args = DOMUtil.toMap(nodes.item(i).getAttributes());
-      result.put(args.get(NAME), new CacheConfig(args, configPath+"/"+args.get(NAME)));
+      CacheConfig config = getConfig(solrConfig, nodes.item(i).getNodeName(), DOMUtil.toMap(nodes.item(i).getAttributes()), configPath);
+      result.put(config.args.get(NAME), config);
     }
     return result;
   }
 
-  public String getName() {
-    return name;
+
+  public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
+    Node node = solrConfig.getNode(xpath, false);
+    if(node == null || !"true".equals(DOMUtil.getAttrOrDefault(node, "enabled", "true"))) {
+      Map<String, String> m = solrConfig.getOverlay().getEditableSubProperties(xpath);
+      if(m==null) return null;
+      List<String> parts = StrUtils.splitSmart(xpath, '/');
+      return getConfig(solrConfig,parts.get(parts.size()-1) , Collections.EMPTY_MAP,xpath);
+    }
+    return getConfig(solrConfig, node.getNodeName(),DOMUtil.toMap(node.getAttributes()), xpath);
   }
 
 
-  public <K, V> SolrCacheHolder<K, V> newInstance(SolrCore core) {
-    return new SolrCacheHolder(new CacheInfo(this, core));
-  }
+  public static CacheConfig getConfig(SolrConfig solrConfig, String nodeName, Map<String,String> attrs, String xpath) {
+    CacheConfig config = new CacheConfig();
+    config.nodeName = nodeName;
+    Map attrsCopy = new LinkedHashMap<>(attrs.size());
+    for (Map.Entry<String, String> e : attrs.entrySet()) {
+      attrsCopy.put(e.getKey(), String.valueOf(e.getValue()));
+    }
+    attrs = attrsCopy;
+    config.args = attrs;
 
-  static class CacheInfo {
-    final CacheConfig cfg;
-    SolrCore core;
-    SolrCache cache = null;
-    String pkg;
-    RuntimeLib runtimeLib;
-    CacheRegenerator regen = null;
-
-
-    CacheInfo(CacheConfig cfg, SolrCore core) {
-      this.core = core;
-      this.cfg = cfg;
-      pkg = cfg.args.attributes.get(CommonParams.PACKAGE);
-      ResourceLoader loader = pkg == null ? core.getResourceLoader() :
-          core.getCoreContainer().getPackageManager().getResourceLoader(pkg);
-
-      try {
-        cache = loader.findClass(cfg.cacheImpl, SolrCache.class).getConstructor().newInstance();
-        regen = null;
-        if (cfg.regenImpl != null) {
-          regen = loader.findClass(cfg.regenImpl, CacheRegenerator.class).getConstructor().newInstance();
-        }
-      } catch (Exception e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error loading cache " + cfg.jsonStr(), e);
-      }
-      if (regen == null && cfg.defRegen != null) regen = cfg.defRegen;
-      cfg.persistence[0] = cache.init(cfg.args.attributes, cfg.persistence[0], regen);
-      if (pkg!=null && loader instanceof MemClassLoader) {
-        MemClassLoader memClassLoader = (MemClassLoader) loader;
-        runtimeLib = core.getCoreContainer().getPackageManager().getLib(pkg);
+    Map<String, String> map = xpath == null ? null : solrConfig.getOverlay().getEditableSubProperties(xpath);
+    if(map != null){
+      HashMap<String, String> mapCopy = new HashMap<>(config.args);
+      for (Map.Entry<String, String> e : map.entrySet()) {
+        mapCopy.put(e.getKey(),String.valueOf(e.getValue()));
       }
+      config.args = mapCopy;
+    }
+    String nameAttr = config.args.get(NAME);  // OPTIONAL
+    if (nameAttr==null) {
+      config.args.put(NAME, config.nodeName);
+    }
 
+    SolrResourceLoader loader = solrConfig.getResourceLoader();
+    config.cacheImpl = config.args.get("class");
+    if(config.cacheImpl == null) config.cacheImpl = "solr.LRUCache";
+    config.regenImpl = config.args.get("regenerator");
+    config.clazz = loader.findClass(config.cacheImpl, SolrCache.class);
+    if (config.regenImpl != null) {
+      config.regenerator = loader.newInstance(config.regenImpl, CacheRegenerator.class);
     }
+    
+    return config;
   }
 
-
-  public void setDefaultRegenerator(CacheRegenerator regen) {
-    this.defRegen = regen;
+  public SolrCache newInstance() {
+    try {
+      SolrCache cache = clazz.getConstructor().newInstance();
+      persistence[0] = cache.init(args, persistence[0], regenerator);
+      return cache;
+    } catch (Exception e) {
+      SolrException.log(log,"Error instantiating cache",e);
+      // we can carry on without a cache... but should we?
+      // in some cases (like an OOM) we probably should try to continue.
+      return null;
+    }
   }
 
   @Override
-  public void writeMap(EntryWriter ew) throws IOException {
-    args.attributes.forEach(ew.getBiConsumer());
+  public Map<String, Object> toMap(Map<String, Object> map) {
+    Map result = Collections.unmodifiableMap(args);
+    return result;
   }
+
+  public String getNodeName() {
+    return nodeName;
+  }
+
+
 }
diff --git a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
index 1cec0aa..7d648fa 100644
--- a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
@@ -16,14 +16,6 @@
  */
 package org.apache.solr.search;
 
-import java.lang.invoke.MethodHandles;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeUnit;
-
 import com.codahale.metrics.MetricRegistry;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -34,6 +26,14 @@ import org.apache.solr.util.ConcurrentLRUCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.lang.invoke.MethodHandles;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+
 /**
  * SolrCache based on ConcurrentLRUCache implementation.
  * <p>
@@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
  * @see org.apache.solr.search.SolrCache
  * @since solr 1.4
  */
-public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K, V>, Accountable {
+public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V>, Accountable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FastLRUCache.class);
@@ -64,7 +64,7 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K, V>
   private long warmupTime = 0;
 
   private String description = "Concurrent LRU Cache";
-  private ConcurrentLRUCache<K, V> cache;
+  private ConcurrentLRUCache<K,V> cache;
   private int showItems = 0;
 
   private long maxRamBytes;
@@ -105,7 +105,7 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K, V>
     str = (String) args.get(INITIAL_SIZE_PARAM);
     initialSize = str == null ? maxSize : Integer.parseInt(str);
     str = (String) args.get(CLEANUP_THREAD_PARAM);
-    cleanupThread = str != null && Boolean.parseBoolean(str);
+    cleanupThread = str == null ? false : Boolean.parseBoolean(str);
 
     str = (String) args.get(SHOW_ITEMS_PARAM);
     showItems = str == null ? 0 : Integer.parseInt(str);
@@ -120,7 +120,7 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K, V>
     str = (String) args.get(MAX_RAM_MB_PARAM);
     long maxRamMB = str == null ? -1 : (long) Double.parseDouble(str);
     this.maxRamBytes = maxRamMB < 0 ? Long.MAX_VALUE : maxRamMB * 1024L * 1024L;
-    if (maxRamBytes != Long.MAX_VALUE) {
+    if (maxRamBytes != Long.MAX_VALUE)  {
       ramLowerWatermark = Math.round(maxRamBytes * 0.8);
       description = generateDescription(maxRamBytes, ramLowerWatermark, cleanupThread);
       cache = new ConcurrentLRUCache<>(ramLowerWatermark, maxRamBytes, cleanupThread, null, maxIdleTimeSec);
@@ -159,7 +159,7 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K, V>
    */
   protected String generateDescription(int limit, int initialSize, int minLimit, int acceptableLimit, boolean newThread) {
     String description = "Concurrent LRU Cache(maxSize=" + limit + ", initialSize=" + initialSize +
-        ", minSize=" + minLimit + ", acceptableSize=" + acceptableLimit + ", cleanupThread=" + newThread;
+        ", minSize="+minLimit + ", acceptableSize="+acceptableLimit+", cleanupThread="+newThread;
     if (isAutowarmingOn()) {
       description += ", " + getAutowarmDescription();
     }
@@ -220,9 +220,10 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K, V>
       for (int i = itemsArr.length - 1; i >= 0; i--) {
         try {
           boolean continueRegen = regenerator.regenerateItem(searcher,
-              this, old, itemsArr[i].getKey(), itemsArr[i].getValue());
+                  this, old, itemsArr[i].getKey(), itemsArr[i].getValue());
           if (!continueRegen) break;
-        } catch (Exception e) {
+        }
+        catch (Exception e) {
           SolrException.log(log, "Error during auto-warming of key:" + itemsArr[i].getKey(), e);
         }
       }
@@ -302,14 +303,14 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K, V>
         map.put("cumulative_idleEvictions", cIdleEvictions);
 
         if (detailed && showItems != 0) {
-          Map items = cache.getLatestAccessedItems(showItems == -1 ? Integer.MAX_VALUE : showItems);
-          for (Map.Entry e : (Set<Map.Entry>) items.entrySet()) {
+          Map items = cache.getLatestAccessedItems( showItems == -1 ? Integer.MAX_VALUE : showItems );
+          for (Map.Entry e : (Set <Map.Entry>)items.entrySet()) {
             Object k = e.getKey();
             Object v = e.getValue();
 
             String ks = "item_" + k;
             String vs = v.toString();
-            map.put(ks, vs);
+            map.put(ks,vs);
           }
 
         }
diff --git a/solr/core/src/java/org/apache/solr/search/LFUCache.java b/solr/core/src/java/org/apache/solr/search/LFUCache.java
index b9a4820..20cf664 100644
--- a/solr/core/src/java/org/apache/solr/search/LFUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LFUCache.java
@@ -17,10 +17,10 @@
 package org.apache.solr.search;
 
 import java.lang.invoke.MethodHandles;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 
@@ -116,14 +116,14 @@ public class LFUCache<K, V> implements SolrCache<K, V>, Accountable {
     str = (String) args.get(AUTOWARM_COUNT_PARAM);
     autowarmCount = str == null ? 0 : Integer.parseInt(str);
     str = (String) args.get(CLEANUP_THREAD_PARAM);
-    cleanupThread = str != null && Boolean.parseBoolean(str);
+    cleanupThread = str == null ? false : Boolean.parseBoolean(str);
 
     str = (String) args.get(SHOW_ITEMS_PARAM);
     showItems = str == null ? 0 : Integer.parseInt(str);
 
     // Don't make this "efficient" by removing the test, default is true and omitting the param will make it false.
     str = (String) args.get(TIME_DECAY_PARAM);
-    timeDecay = (str == null) || Boolean.parseBoolean(str);
+    timeDecay = (str == null) ? true : Boolean.parseBoolean(str);
 
     str = (String) args.get(MAX_IDLE_TIME_PARAM);
     if (str == null) {
diff --git a/solr/core/src/java/org/apache/solr/search/LRUCache.java b/solr/core/src/java/org/apache/solr/search/LRUCache.java
index bcb56cf..c733c07 100644
--- a/solr/core/src/java/org/apache/solr/search/LRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LRUCache.java
@@ -18,11 +18,11 @@ package org.apache.solr.search;
 
 import java.lang.invoke.MethodHandles;
 import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.LongAdder;
 
@@ -234,8 +234,8 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
   }
 
   /**
-   *
-   * @return Returns the description of this cache.
+   * 
+   * @return Returns the description of this cache. 
    */
   private String generateDescription() {
     String description = "LRU Cache(maxSize=" + getMaxSize() + ", initialSize=" + initialSize;
@@ -341,9 +341,9 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
 
       // Don't do the autowarming in the synchronized block, just pull out the keys and values.
       synchronized (other.map) {
-
+        
         int sz = autowarm.getWarmCount(other.map.size());
-
+        
         keys = new Object[sz];
         vals = new Object[sz];
 
@@ -383,6 +383,7 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
 
   }
 
+
   //////////////////////// SolrInfoMBeans methods //////////////////////
 
 
diff --git a/solr/core/src/java/org/apache/solr/search/SolrCache.java b/solr/core/src/java/org/apache/solr/search/SolrCache.java
index 9fe186a..5988ad5 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrCache.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCache.java
@@ -16,17 +16,16 @@
  */
 package org.apache.solr.search;
 
-import java.util.Map;
-
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricProducer;
 
+import java.util.Map;
+
 
 /**
  * Primary API for dealing with Solr's internal caches.
  */
 public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer {
-  String TYPE = "cache";
 
   String HIT_RATIO_PARAM = "hitratio";
   String HITS_PARAM = "hits";
@@ -61,7 +60,7 @@ public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer {
    * regenerate an item in the new cache from an entry in the old cache.
    *
    */
-  Object init(Map args, Object persistence, CacheRegenerator regenerator);
+  public Object init(Map args, Object persistence, CacheRegenerator regenerator);
   // I don't think we need a factory for faster creation given that these
   // will be associated with slow-to-create SolrIndexSearchers.
   // change to NamedList when other plugins do?
@@ -77,29 +76,29 @@ public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer {
    *
    * :TODO: verify this.
    */
-  String name();
+  public String name();
 
 
   // Should SolrCache just extend the java.util.Map interface?
   // Following the conventions of the java.util.Map interface in any case.
 
   /** :TODO: copy from Map */
-  int size();
+  public int size();
 
   /** :TODO: copy from Map */
-  V put(K key, V value);
+  public V put(K key, V value);
 
   /** :TODO: copy from Map */
-  V get(K key);
+  public V get(K key);
 
   /** :TODO: copy from Map */
-  void clear();
+  public void clear();
 
   /** 
    * Enumeration of possible States for cache instances.
    * :TODO: only state that seems to ever be set is LIVE ?
   */
-  enum State {
+  public enum State { 
     /** :TODO */
     CREATED, 
     /** :TODO */
@@ -116,14 +115,14 @@ public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer {
    * The cache user (SolrIndexSearcher) will take care of switching
    * cache states.
    */
-  void setState(State state);
+  public void setState(State state);
 
   /**
    * Returns the last State set on this instance
    *
    * @see #setState
    */
-  State getState();
+  public State getState();
 
   /**
    * Warm this cache associated with <code>searcher</code> using the <code>old</code>
@@ -135,7 +134,7 @@ public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer {
 
 
   /** Frees any non-memory resources */
-  void close();
+  public void close();
 
   /** Returns maximum size limit (number of items) if set and supported, -1 otherwise. */
   int getMaxSize();
diff --git a/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java b/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
index 3b64e9d..66b8ab1 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
@@ -22,12 +22,8 @@ import java.util.Map;
 import java.util.Set;
 
 import com.codahale.metrics.MetricRegistry;
-import org.apache.solr.common.MapWriter;
-import org.apache.solr.core.PluginInfo;
-import org.apache.solr.core.RuntimeLib;
-import org.apache.solr.core.SolrCore;
+import org.apache.solr.common.util.Utils;
 import org.apache.solr.metrics.SolrMetricManager;
-import org.apache.solr.metrics.SolrMetricProducer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,53 +31,12 @@ public class SolrCacheHolder<K, V> implements SolrCache<K,V> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
 
-  private CacheConfig.CacheInfo info;
+  private final CacheConfig factory;
   protected volatile SolrCache<K, V> delegate;
 
-
-
-  public SolrCacheHolder(CacheConfig.CacheInfo cacheInfo) {
-    this.info = cacheInfo;
-    this.delegate = cacheInfo.cache;
-
-    if(info.pkg != null) {
-      info.core.addPackageListener(new SolrCore.PkgListener() {
-        @Override
-        public String packageName() {
-          return info.pkg;
-        }
-
-        @Override
-        public PluginInfo pluginInfo() {
-          return info.cfg.args;
-        }
-
-        @Override
-        public MapWriter lib() {
-          return info.runtimeLib;
-        }
-
-        @Override
-        public void changed(RuntimeLib lib) {
-          reloadCache(lib);
-        }
-      });
-    }
-  }
-
-  private void reloadCache(RuntimeLib lib) {
-    int znodeVersion = info.runtimeLib == null ? -1 : info.runtimeLib.getZnodeVersion();
-    if (lib.getZnodeVersion() > znodeVersion) {
-      log.info("Cache {} being reloaded, package: {} loaded from: {} ", delegate.getClass().getSimpleName(), info.pkg, lib.getUrl());
-      info = new CacheConfig.CacheInfo(info.cfg, info.core);
-      delegate.close();
-      delegate = info.cache;
-      if(metricsInfo != null){
-        metricsInfo.init(delegate);
-
-      }
-
-    }
+  public SolrCacheHolder(SolrCache<K, V> delegate, CacheConfig factory) {
+    this.delegate = delegate;
+    this.factory = factory;
   }
 
   public int size() {
@@ -186,31 +141,12 @@ public class SolrCacheHolder<K, V> implements SolrCache<K,V> {
     return delegate.getCategory();
   }
 
-
-  private MetricsInfo metricsInfo;
-
-  public static class MetricsInfo {
-    final SolrMetricManager manager;
-    final String registry;
-    final String tag;
-    final String scope;
-
-    MetricsInfo(SolrMetricManager manager, String registry, String tag, String scope) {
-      this.manager = manager;
-      this.registry = registry;
-      this.tag = tag;
-      this.scope = scope;
-    }
-
-    public void init(SolrMetricProducer metricProducer) {
-      metricProducer.initializeMetrics(manager,registry,tag,scope);
-    }
-  }
-
   @Override
   public void initializeMetrics(SolrMetricManager manager, String registry, String tag, String scope) {
-    this.metricsInfo = new MetricsInfo(manager, registry, tag, scope);
-    delegate.initializeMetrics(manager, registry, tag, scope);
+    log.debug("Going to register cachemetrics " + Utils.toJSONString(factory));
+
+    delegate.initializeMetrics(manager, registry, tag,scope);
 
   }
+
 }
diff --git a/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java b/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
index f7bc263..313d91b 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
@@ -111,8 +111,7 @@ public class SolrDocumentFetcher {
     this.searcher = searcher;
     this.enableLazyFieldLoading = solrConfig.enableLazyFieldLoading;
     if (cachingEnabled) {
-      documentCache = solrConfig.documentCacheConfig == null ? null :
-          solrConfig.documentCacheConfig.newInstance(searcher.getCore());
+      documentCache = solrConfig.documentCacheConfig == null ? null : solrConfig.documentCacheConfig.newInstance();
     } else {
       documentCache = null;
     }
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 192adb1..9b78431 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -267,12 +267,12 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     if (cachingEnabled) {
       final ArrayList<SolrCache> clist = new ArrayList<>();
       fieldValueCache = solrConfig.fieldValueCacheConfig == null ? null
-          : solrConfig.fieldValueCacheConfig.newInstance(core);
-      if (fieldValueCache != null) clist.add( fieldValueCache);
-      filterCache = solrConfig.filterCacheConfig == null ? null : solrConfig.filterCacheConfig.newInstance(core);
+          : solrConfig.fieldValueCacheConfig.newInstance();
+      if (fieldValueCache != null) clist.add(fieldValueCache);
+      filterCache = solrConfig.filterCacheConfig == null ? null : solrConfig.filterCacheConfig.newInstance();
       if (filterCache != null) clist.add(filterCache);
       queryResultCache = solrConfig.queryResultCacheConfig == null ? null
-          : solrConfig.queryResultCacheConfig.newInstance(core);
+          : solrConfig.queryResultCacheConfig.newInstance();
       if (queryResultCache != null) clist.add(queryResultCache);
       SolrCache<Integer, Document> documentCache = docFetcher.getDocumentCache();
       if (documentCache != null) clist.add(documentCache);
@@ -281,8 +281,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         cacheMap = NO_GENERIC_CACHES;
       } else {
         cacheMap = new HashMap<>(solrConfig.userCacheConfigs.size());
-        for (Map.Entry<String, CacheConfig> e : solrConfig.userCacheConfigs.entrySet()) {
-          SolrCache cache = e.getValue().newInstance(core);
+        for (Map.Entry<String,CacheConfig> e : solrConfig.userCacheConfigs.entrySet()) {
+          SolrCache cache = e.getValue().newInstance();
           if (cache != null) {
             cacheMap.put(cache.name(), cache);
             clist.add(cache);
@@ -502,8 +502,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   // Set default regenerators on filter and query caches if they don't have any
   //
   public static void initRegenerators(SolrConfig solrConfig) {
-    if (solrConfig.fieldValueCacheConfig != null) {
-      solrConfig.fieldValueCacheConfig.setDefaultRegenerator(new CacheRegenerator() {
+    if (solrConfig.fieldValueCacheConfig != null && solrConfig.fieldValueCacheConfig.getRegenerator() == null) {
+      solrConfig.fieldValueCacheConfig.setRegenerator(new CacheRegenerator() {
         @Override
         public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache,
             Object oldKey, Object oldVal) throws IOException {
@@ -515,8 +515,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       });
     }
 
-    if (solrConfig.filterCacheConfig != null ) {
-      solrConfig.filterCacheConfig.setDefaultRegenerator(new CacheRegenerator() {
+    if (solrConfig.filterCacheConfig != null && solrConfig.filterCacheConfig.getRegenerator() == null) {
+      solrConfig.filterCacheConfig.setRegenerator(new CacheRegenerator() {
         @Override
         public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache,
             Object oldKey, Object oldVal) throws IOException {
@@ -526,9 +526,9 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       });
     }
 
-    if (solrConfig.queryResultCacheConfig != null) {
+    if (solrConfig.queryResultCacheConfig != null && solrConfig.queryResultCacheConfig.getRegenerator() == null) {
       final int queryResultWindowSize = solrConfig.queryResultWindowSize;
-      solrConfig.queryResultCacheConfig.setDefaultRegenerator(new CacheRegenerator() {
+      solrConfig.queryResultCacheConfig.setRegenerator(new CacheRegenerator() {
         @Override
         public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache,
             Object oldKey, Object oldVal) throws IOException {
@@ -623,7 +623,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
   /** expert: internal API, subject to change */
   public SolrCache<String,UnInvertedField> getFieldValueCache() {
-    return fieldValueCache ;
+    return fieldValueCache;
   }
 
   /** Returns a weighted sort according to this searcher */
@@ -2468,7 +2468,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
     @Override
     public int hashCode() {
-      return classHash()
+      return classHash() 
           + 31 * Objects.hashCode(topFilter)
           + 31 * Objects.hashCode(weights);
     }
diff --git a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
index 31f5a74..5fd18a1 100644
--- a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
@@ -29,11 +29,12 @@ import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
-import org.apache.http.HttpRequest;
-import org.apache.http.protocol.HttpContext;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
+
+import org.apache.http.HttpRequest;
+import org.apache.http.protocol.HttpContext;
 import org.eclipse.jetty.client.api.Request;
 
 /**
@@ -65,7 +66,7 @@ public abstract class AuthenticationPlugin implements Closeable, SolrInfoBean, S
    * @param pluginConfig Config parameters, possibly from a ZK source
    */
   public abstract void init(Map<String, Object> pluginConfig);
-
+ 
   /**
    * This method attempts to authenticate the request. Upon a successful authentication, this
    * must call the next filter in the filter chain and set the user principal of the request,
@@ -106,10 +107,10 @@ public abstract class AuthenticationPlugin implements Closeable, SolrInfoBean, S
    * delegate to {@link PKIAuthenticationPlugin}. Return true to indicate that your plugin
    * did handle the request, or false to signal that PKI plugin should handle it. This method
    * will be called by {@link PKIAuthenticationPlugin}'s interceptor.
-   *
+   * 
    * <p>
    *   If not overridden, this method will return true for plugins implementing {@link HttpClientBuilderPlugin}.
-   *   This method can be overridden by subclasses e.g. to set HTTP headers, even if you don't use a clientBuilder.
+   *   This method can be overridden by subclasses e.g. to set HTTP headers, even if you don't use a clientBuilder. 
    * </p>
    * @param httpRequest the httpRequest that is about to be sent to another internal Solr node
    * @param httpContext the context of that request.
@@ -136,7 +137,7 @@ public abstract class AuthenticationPlugin implements Closeable, SolrInfoBean, S
   protected boolean interceptInternodeRequest(Request request) {
     return this instanceof HttpClientBuilderPlugin;
   }
-
+  
   /**
    * Cleanup any per request  data
    */
@@ -160,7 +161,7 @@ public abstract class AuthenticationPlugin implements Closeable, SolrInfoBean, S
     metricNames.addAll(Arrays.asList("errors", "requests", "authenticated", "passThrough",
         "failWrongCredentials", "failMissingCredentials", "requestTimes", "totalTime"));
   }
-
+  
   @Override
   public String getName() {
     return this.getClass().getName();
@@ -185,5 +186,5 @@ public abstract class AuthenticationPlugin implements Closeable, SolrInfoBean, S
   public MetricRegistry getMetricRegistry() {
     return registry;
   }
-
+  
 }
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index 16b39a4..79b4d29 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -39,7 +39,6 @@ public interface PermissionNameProvider {
     CORE_READ_PERM("core-admin-read", null),
     CORE_EDIT_PERM("core-admin-edit", null),
     READ_PERM("read", "*"),
-    CUSTOM_PERM("custom-op", null),//custom operation , user-defined
     UPDATE_PERM("update", "*"),
     CONFIG_EDIT_PERM("config-edit", unmodifiableSet(new HashSet<>(asList("*", null)))),
     CONFIG_READ_PERM("config-read", "*"),
@@ -52,8 +51,6 @@ public interface PermissionNameProvider {
     AUTOSCALING_WRITE_PERM("autoscaling-write", null),
     AUTOSCALING_HISTORY_READ_PERM("autoscaling-history-read", null),
     METRICS_HISTORY_READ_PERM("metrics-history-read", null),
-    BLOB_READ("blob-read", null),
-    BLOB_WRITE("blob-write", null),
     ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
     ;
     final String name;
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index a385479..90d6b17 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -82,9 +82,9 @@ import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
 import org.apache.solr.security.PublicKeyHandler;
 import org.apache.solr.util.SolrFileCleaningTracker;
+import org.apache.solr.util.tracing.GlobalTracer;
 import org.apache.solr.util.StartupLoggingUtils;
 import org.apache.solr.util.configuration.SSLConfigurationsFactory;
-import org.apache.solr.util.tracing.GlobalTracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -180,7 +180,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       final Path solrHomePath = solrHome == null ? SolrResourceLoader.locateSolrHome() : Paths.get(solrHome);
       coresInit = createCoreContainer(solrHomePath, extraProperties);
       SolrResourceLoader.ensureUserFilesDataDir(solrHomePath);
-      SolrResourceLoader.ensureBlobsDir(solrHomePath);
       this.httpClient = coresInit.getUpdateShardHandler().getDefaultHttpClient();
       setupJvmMetrics(coresInit);
       log.debug("user.dir=" + System.getProperty("user.dir"));
@@ -627,8 +626,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
             public void close() {
               // even though we skip closes, we let local tests know not to close so that a full understanding can take
               // place
-              assert !Thread.currentThread().getStackTrace()[2].getClassName().matches(
-                  "org\\.apache\\.(?:solr|lucene).*") : CLOSE_STREAM_MSG;
+              assert Thread.currentThread().getStackTrace()[2].getClassName().matches(
+                  "org\\.apache\\.(?:solr|lucene).*") ? false : true : CLOSE_STREAM_MSG;
               this.stream = ClosedServletInputStream.CLOSED_SERVLET_INPUT_STREAM;
             }
           };
@@ -662,8 +661,9 @@ public class SolrDispatchFilter extends BaseSolrFilter {
             public void close() {
               // even though we skip closes, we let local tests know not to close so that a full understanding can take
               // place
-              assert !Thread.currentThread().getStackTrace()[2].getClassName().matches(
-                  "org\\.apache\\.(?:solr|lucene).*") : CLOSE_STREAM_MSG;
+              assert Thread.currentThread().getStackTrace()[2].getClassName().matches(
+                  "org\\.apache\\.(?:solr|lucene).*") ? false
+                      : true : CLOSE_STREAM_MSG;
               stream = ClosedServletOutputStream.CLOSED_SERVLET_OUTPUT_STREAM;
             }
           };
diff --git a/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java b/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
index c45f0c6..eb3c08b 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
@@ -328,7 +328,7 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
   public static class LazyUpdateProcessorFactoryHolder extends PluginBag.PluginHolder<UpdateRequestProcessorFactory> {
     private volatile UpdateRequestProcessorFactory lazyFactory;
 
-    public LazyUpdateProcessorFactoryHolder(final PluginBag.PluginHolder<UpdateRequestProcessorFactory> holder) {
+    public LazyUpdateProcessorFactoryHolder(final PluginBag.LazyPluginHolder holder) {
       super(holder.getPluginInfo());
       lazyFactory = new LazyUpdateRequestProcessorFactory(holder);
     }
@@ -340,20 +340,26 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
     }
 
     public class LazyUpdateRequestProcessorFactory extends UpdateRequestProcessorFactory {
-      private final PluginBag.PluginHolder<UpdateRequestProcessorFactory> holder;
+      private final PluginBag.LazyPluginHolder holder;
+      UpdateRequestProcessorFactory delegate;
 
-      public LazyUpdateRequestProcessorFactory(PluginBag.PluginHolder holder) {
+      public LazyUpdateRequestProcessorFactory(PluginBag.LazyPluginHolder holder) {
         this.holder = holder;
       }
 
       public UpdateRequestProcessorFactory getDelegate() {
-        return holder.get();
+        return delegate;
       }
 
       @Override
       public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
-        return holder.get().getInstance(req, rsp, next);
+        if (delegate != null) return delegate.getInstance(req, rsp, next);
 
+        synchronized (this) {
+          if (delegate == null)
+            delegate = (UpdateRequestProcessorFactory) holder.get();
+        }
+        return delegate.getInstance(req, rsp, next);
       }
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/util/CryptoKeys.java b/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
index ede009a..0835b59 100644
--- a/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
+++ b/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
@@ -21,6 +21,7 @@ import javax.crypto.Cipher;
 import javax.crypto.IllegalBlockSizeException;
 import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
+
 import java.lang.invoke.MethodHandles;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
@@ -72,11 +73,11 @@ public final class CryptoKeys implements CLIO {
       boolean verified;
       try {
         verified = CryptoKeys.verify(entry.getValue(), Base64.base64ToByteArray(sig), data);
-        log.debug("verified {} ", verified);
+        log.info("verified {} ", verified);
         if (verified) return entry.getKey();
       } catch (Exception e) {
         exception = e;
-        log.debug("NOT verified  ");
+        log.info("NOT verified  ");
       }
 
     }
@@ -103,17 +104,22 @@ public final class CryptoKeys implements CLIO {
    * @param data      The data tha is signed
    */
   public static boolean verify(PublicKey publicKey, byte[] sig, ByteBuffer data) throws InvalidKeyException, SignatureException {
-    data = ByteBuffer.wrap(data.array(), data.arrayOffset(), data.limit());
+    int oldPos = data.position();
+    Signature signature = null;
     try {
-      Signature signature = Signature.getInstance("SHA1withRSA");
+      signature = Signature.getInstance("SHA1withRSA");
       signature.initVerify(publicKey);
       signature.update(data);
-      return signature.verify(sig);
+      boolean verify = signature.verify(sig);
+      return verify;
+
     } catch (NoSuchAlgorithmException e) {
-      //wil not happen
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      //will not happen
+    } finally {
+      //Signature.update resets the position. set it back to old
+      data.position(oldPos);
     }
-
+    return false;
   }
 
   private static byte[][] evpBytesTokey(int key_len, int iv_len, MessageDigest md,
diff --git a/solr/core/src/test-files/cryptokeys/priv_key2048.pem b/solr/core/src/test-files/cryptokeys/priv_key2048.pem
deleted file mode 100644
index 4d2c8c2..0000000
--- a/solr/core/src/test-files/cryptokeys/priv_key2048.pem
+++ /dev/null
@@ -1,27 +0,0 @@
------BEGIN RSA PRIVATE KEY-----
-MIIEpQIBAAKCAQEA1fSq/8iz1sIppHhSKrC0g2uzfFKZzWZAbcvVQbyS/pwxC7VB
-hR93DVINyGGT3XHnpemt/h0wrifCIEMyqSLTIhiu5bRJpfE7UO9vGgTcP5+i2wTe
-cKHqrxDvbQ4D7co96Gvu2cShySbOHsFjZXL4eaqU2W2x8S7U+OjRBwtwMxB4vstX
-5u75WtwVXwNRj+uXIfTTisplE/nA/slqByW4Q9QAg+du+Ejh4W7nF+Z9GRMR7MZe
-c1TeGOYZd8YzYku7WyUZ1SRQ6JjaZrdphlLtysMgqP0MMajEoFs/ajeNHef0iCz0
-TnB05PQd+GPO5+JrLPZ399mucl/jM+cbixn9pwIDAQABAoIBAQCpfA51XryvU9F+
-+t1D+gSU0p00z44MeoJxN3WwhDwBOyNS/ftwA/Pf9m76m+lxEwkIkesFkIP+gXoy
-6mhYOUD9EoaBaeapcEWnoHPnLZB3SfLNArdei7UHhyfSwLZ2CK4vzkg/85ahbH79
-N/6P35pbbrhI4K+DubB1mJ/0r6fqmh6/04L47/liAPvsSM9ZJIMwbuZbYY21ggI9
-ZGk+kO0C/CyzxplaVLJ8P86KnRloEfjSmMhP72z7bja/BE2NX42G12YbjY7tVMn7
-duTWU2F4JWYriWAHr+4GwODDdtvn/R5jPirDIJeHCd6Bg1t7KibHRTcgYgtwDBqG
-F65g4zqRAoGBAP2fry+6uXe3rAJDJrCSKPQVTv5QhOvG1466xsOaWRSe/rx1Mvnd
-Z4pe+T8bdvsvqFnNMAkZKzzPjJ+oCSVKHhcerzMm2Cw6Gpv2yywA/1VykIoZmdNM
-/vHjC7w35q7xwEUHxB/rt2vvijrAYnhaq86uIXzoiqTGaKJ/z34QsCppAoGBANf1
-1wsISnZPjIipMIYtC7Co3GCUhsQ+ksVBhtsOHaKfon3Q69Qbz93l7dbCwgFbL6td
-HW/ppnABZLVFHnoLJ5YrriVZ1Wizx90+RFGdNj74UTV8bfqr/C32UKTjqoYjPAZO
-vEOzHkmpc9I1mrxm1Mcff5EHDFmXGXoZ2GLCpEWPAoGAOXroVFPoVtacuEKJ0Ti+
-6Vqu9XpANcNx9RollA02JTNHnmSdcf2YysZtjLznwVPyvq9/NICsyPJs93443Geo
-3CqLIHesRJHCmBhdwZJUTART98iHkVkA6sc/UKAGux11Ku/wph9hCahXVqtlZct+
-5q+WTV3SljeVXUbEOtkDZAkCgYEArnd0R/xls5jmbs1IX01q4Ug56Wh0S3xFtEgQ
-u013EZcnfb9Xld72Gk0TzOlANDpHk4hBLNU02c22X188lNoIHCCjqpcdel2rPIh+
-RvTcCxku+ifQ7a8dpsAUPHGUpJM4fdwD6il9cYMNB6i4njXw9gDzXOW1y3bvZR4W
-GwsmDO8CgYEA5vG0TdwkvdDcsJYimm3WQJ/VnYidE6JfjnAxnPwFFPjQoDRIS32f
-TMMJFTHSSH4xgQLEhEfaAbrkptpPORM9QAjjRx2RXoa5yu2GMpDWua4MxpHdqiSY
-v/rOw+6fZbe8YC9bZ8AE+GPuHdJDQFoSU7ieCGiF/iwWB2jhwCm7OyY=
------END RSA PRIVATE KEY-----
diff --git a/solr/core/src/test-files/cryptokeys/priv_key512.pem b/solr/core/src/test-files/cryptokeys/priv_key512.pem
deleted file mode 100644
index 53c032c..0000000
--- a/solr/core/src/test-files/cryptokeys/priv_key512.pem
+++ /dev/null
@@ -1,9 +0,0 @@
------BEGIN RSA PRIVATE KEY-----
-MIIBOQIBAAJBAMgmSVfl+E2Nj/oKAP2TceWh17pk6Ugsw5A5nLH+OeMB/WeWJZg/
-NEDda8SXfQDEVRmw5P+2IZypPASzfCrc6yECAwEAAQJAbZFwEztky+fUSrhRRIAE
-GQaZV4PIpWdEA99WJaabv+YsWN5UUd7y+Evu50mhH3RQIxQd+R6SYs1ke9OlHlV2
-cQIhAP8367gybVEu2A+Cg1fE9vbHfnHrurpDQrh9r0ZKooTtAiEAyMMxvlHlSh6Q
-2cUTSxuyUEaQfN+W4efehgfIWBVlzIUCIEHBMZ0qeNnCvO36DUbuu0ZHjb9iIaDd
-tXH9B8yPbCHdAiAaV3o0ZZx3MDGDUVdpuHWaENguekva0kihP24rGIul3QIgNqZS
-EzA2aoQdNPl5oDfkhqAGjs5pb7qLgtmXJvVhi/Q=
------END RSA PRIVATE KEY-----
diff --git a/solr/core/src/test-files/cryptokeys/pub_key2048.der b/solr/core/src/test-files/cryptokeys/pub_key2048.der
deleted file mode 100644
index 0e0e36b..0000000
Binary files a/solr/core/src/test-files/cryptokeys/pub_key2048.der and /dev/null differ
diff --git a/solr/core/src/test-files/cryptokeys/pub_key512.der b/solr/core/src/test-files/cryptokeys/pub_key512.der
deleted file mode 100644
index 4c926dd..0000000
Binary files a/solr/core/src/test-files/cryptokeys/pub_key512.der and /dev/null differ
diff --git a/solr/core/src/test-files/runtimecode/MyDocCache.java b/solr/core/src/test-files/runtimecode/MyDocCache.java
deleted file mode 100644
index 406b950..0000000
--- a/solr/core/src/test-files/runtimecode/MyDocCache.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package runtimecode;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.StoredField;
-import org.apache.solr.search.LRUCache;
-
-public  class MyDocCache<K,V> extends LRUCache<K,V> {
-
-  static String fld_name= "my_synthetic_fld_s";
-  @Override
-  public V put(K key, V value) {
-    if(value instanceof Document){
-      Document d = (Document) value;
-      d.add(new StoredField(fld_name, "version_2"));
-    }
-    return super.put(key, value);
-  }
-}
diff --git a/solr/core/src/test-files/runtimecode/cache.jar.bin b/solr/core/src/test-files/runtimecode/cache.jar.bin
deleted file mode 100644
index 0729896..0000000
Binary files a/solr/core/src/test-files/runtimecode/cache.jar.bin and /dev/null differ
diff --git a/solr/core/src/test-files/runtimecode/cache_v2.jar.bin b/solr/core/src/test-files/runtimecode/cache_v2.jar.bin
deleted file mode 100644
index 6105993..0000000
Binary files a/solr/core/src/test-files/runtimecode/cache_v2.jar.bin and /dev/null differ
diff --git a/solr/core/src/test-files/runtimecode/runtimelibs_v3.jar.bin b/solr/core/src/test-files/runtimecode/runtimelibs_v3.jar.bin
deleted file mode 100644
index 6b5bad6..0000000
Binary files a/solr/core/src/test-files/runtimecode/runtimelibs_v3.jar.bin and /dev/null differ
diff --git a/solr/core/src/test-files/runtimecode/sig.txt b/solr/core/src/test-files/runtimecode/sig.txt
deleted file mode 100644
index 29dbb47..0000000
--- a/solr/core/src/test-files/runtimecode/sig.txt
+++ /dev/null
@@ -1,97 +0,0 @@
-================priv_key2048.pem===================
-
-openssl dgst -sha1 -sign ../cryptokeys/priv_key2048.pem runtimelibs.jar.bin | openssl enc -base64
-
-NaTm3+i99/ZhS8YRsLc3NLz2Y6VuwEbu7DihY8GAWwWIGm+jpXgn1JiuaenfxFCc
-fNKCC9WgZmEgbTZTzmV/OZMVn90u642YJbF3vTnzelW1pHB43ZRAJ1iesH0anM37
-w03n3es+vFWQtuxc+2Go888fJoMkUX2C6Zk6Jn116KE45DWjeyPM4mp3vvGzwGvd
-RxP5K9Q3suA+iuI/ULXM7m9mV4ruvs/MZvL+ELm5Jnmk1bBtixVJhQwJP2z++8tQ
-KJghhyBxPIC/2fkAHobQpkhZrXu56JjP+v33ul3Ku4bbvfVMY/LVwCAEnxlvhk+C
-6uRCKCeFMrzQ/k5inasXLw==
-
-
-openssl dgst -sha1 -sign ../cryptokeys/priv_key2048.pem runtimelibs_v2.jar.bin | openssl enc -base64
-
-jsPpNMs74ogRbx9M4n/OH3j3s85KOq9dOtgGJkUf6O5D8T9d9zU2lDwxnTYjQCaW
-cRTLGH3Z8vpc0wyT3g4aXepgLUTSnrepbPffSFhQtFrCNxurPOLzbp6ERhwjZ0RL
-GvZrlbbjR2SxqZ3BpHiGxslj0tPCkdevNCEy1glLhl8RWG5xsLCrRL1mrEtLg97A
-53oCCrfGAHLEvW+olGeB1r7jqUaSrbfAUfDMSIvZfOIV+xdlvabkNiuzvsAc+B6Q
-pXWm+Em2f5TO/bkOh2m/UInGXcNHCa0oqRMGKP1H252Cv9eXm/d0h3Dqxv+f80Gz
-LfyA6/OKQ9FfskY4pltCsQ==
-
-openssl dgst -sha1 -sign ../cryptokeys/priv_key2048.pem runtimelibs_v3.jar.bin | openssl enc -base64
-
-
-YxFr6SpYrDwG85miDfRWHTjU9UltjtIWQZEhcV55C2rczRUVowCYBxmsDv5mAM8j
-0CTv854xpI1DtBT86wpoTdbF95LQuP9FJId4TS1j8bZ9cxHP5Cqyz1uBHFfUUNUr
-npzTHQkVTp02O9NAjh3c2W41bL4U7j6jQ32+4CW2M+x00TDG0y0H75rQDR8zbLt3
-1oWCz+sBOdZ3rGKJgAvdoGm/wVCTmsabZN+xoz4JaDeBXF16O9Uk9SSq4G0dz5YX
-FuLxHK7ciB5t0+q6pXlF/tdlDqF76Abze0R3d2/0MhXBzyNp3UxJmj6DiprgysfB
-0TbQtJG0XGfdSmx0VChvcA==
-
-YxFr6SpYrDwG85miDfRWHTjU9UltjtIWQZEhcV55C2rczRUVowCYBxmsDv5mAM8j0CTv854xpI1DtBT86wpoTdbF95LQuP9FJId4TS1j8bZ9cxHP5Cqyz1uBHFfUUNUrnpzTHQkVTp02O9NAjh3c2W41bL4U7j6jQ32+4CW2M+x00TDG0y0H75rQDR8zbLt31oWCz+sBOdZ3rGKJgAvdoGm/wVCTmsabZN+xoz4JaDeBXF16O9Uk9SSq4G0dz5YXFuLxHK7ciB5t0+q6pXlF/tdlDqF76Abze0R3d2/0MhXBzyNp3UxJmj6DiprgysfB0TbQtJG0XGfdSmx0VChvcA==
-
-=====================priv_key512.pem=====================
-openssl dgst -sha1 -sign ../cryptokeys/priv_key512.pem runtimelibs.jar.bin | openssl enc -base64
-
-L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1
-f/U3bOlMPINlSOM6LK3JpQ==
-L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1f/U3bOlMPINlSOM6LK3JpQ==
-
-openssl dgst -sha1 -sign ../cryptokeys/priv_key512.pem runtimelibs_v2.jar.bin | openssl enc -base64
-
-j+Rflxi64tXdqosIhbusqi6GTwZq8znunC/dzwcWW0/dHlFGKDurOaE1Nz9FSPJu
-XbHkVLj638yZ0Lp1ssnoYA==
-
-openssl dgst -sha1 -sign ../cryptokeys/priv_key512.pem runtimelibs_v3.jar.bin | openssl enc -base64
-
-a400n4T7FT+2gM0SC6+MfSOExjud8MkhTSFylhvwNjtWwUgKdPFn434Wv7Qc4QEq
-DVLhQoL3WqYtQmLPti0G4Q==
-
-
-====================sha512====================
-
-openssl dgst -sha512 runtimelibs.jar.bin
-
-d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420
-
-
-openssl dgst -sha512 runtimelibs_v2.jar.bin
-
-bc5ce45ad281b6a08fb7e529b1eb475040076834816570902acb6ebdd809410e31006efdeaa7f78a6c35574f3504963f5f7e4d92247d0eb4db3fc9abdda5d417
-
-openssl dgst -sha512 runtimelibs_v3.jar.bin
-
-60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922
-
-openssl dgst -sha512 cache.jar.bin
-
-8946650ba88919cea2f81e4771c418411f61837b2a276088c2f2c86ef2d730f152ccf5975fa8a2c7035a1f00de1994a7788676d95dc7ccea6aaf28c7fff1f46b
-
-openssl dgst -sha512 cache_v2.jar.bin
-
-873337e67b90b7ff99df012b2e9093c63079c37a564643d34861a88c4cbaf0698ebb096905929d69cdbde3b4d29d55e31db24ee05c01b39c0b75a16e54eb4335
-
-=============sha256============================
-
-openssl dgst -sha256 runtimelibs.jar.bin
-
-e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc
-
-openssl dgst -sha512 runtimelibs_v2.jar.bin
-
-79298d7d5c3e60d91154efe7d72f4536eac46698edfa22ab894b85492d562ed4
-
-openssl dgst -sha256 runtimelibs_v3.jar.bin
-
-20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3
-
-openssl dgst -sha256 cache.jar.bin
-
-32e8b5b2a95ea306538b52017f0954aa1b0f8a8b2d0acbc498fd0e66a223f7bd
-
-openssl dgst -sha256 cache_v2.jar.bin
-
-0f670f6dcc2b00f9a448a7ebd457d4ff985ab702c85cdb3608dcae9889e8d702
-
-
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestClusterProperties.java b/solr/core/src/test/org/apache/solr/cloud/TestClusterProperties.java
index 1bc54f2..c082e37 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestClusterProperties.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestClusterProperties.java
@@ -27,7 +27,7 @@ import org.junit.Test;
 public class TestClusterProperties extends SolrCloudTestCase {
 
   private ClusterProperties props;
-
+  
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(1).configure();
@@ -49,7 +49,7 @@ public class TestClusterProperties extends SolrCloudTestCase {
     CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false").process(cluster.getSolrClient());
     assertEquals("false", props.getClusterProperty(ZkStateReader.LEGACY_CLOUD, "true"));
   }
-
+  
   @Test
   public void testSetPluginClusterProperty() throws Exception {
     String propertyName = ClusterProperties.EXT_PROPRTTY_PREFIX + "pluginA.propertyA";
@@ -57,7 +57,7 @@ public class TestClusterProperties extends SolrCloudTestCase {
         .process(cluster.getSolrClient());
     assertEquals("valueA", props.getClusterProperty(propertyName, null));
   }
-
+  
   @Test(expected = SolrException.class)
   public void testSetInvalidPluginClusterProperty() throws Exception {
     String propertyName = "pluginA.propertyA";
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCryptoKeys.java b/solr/core/src/test/org/apache/solr/cloud/TestCryptoKeys.java
index ca172e9..146ad82 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCryptoKeys.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCryptoKeys.java
@@ -195,7 +195,7 @@ public class TestCryptoKeys extends AbstractFullDistribZkTestBase {
   }
 
 
-  public static byte[] readFile(String fname) throws IOException {
+  private byte[] readFile(String fname) throws IOException {
     byte[] buf = null;
     try (FileInputStream fis = new FileInputStream(getFile(fname))) {
       buf = new byte[fis.available()];
diff --git a/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java b/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
index ce1f68e..4a0f1ba 100644
--- a/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
+++ b/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
@@ -36,6 +36,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.eq;
 import static org.mockito.Mockito.mock;
@@ -62,7 +63,6 @@ public class BlobRepositoryMockingTest {
   boolean blobFetched = false;
   String blobKey = "";
   String url = null;
-  String sha256 = null;
   ByteBuffer filecontent = null;
   
   @BeforeClass
@@ -92,14 +92,6 @@ public class BlobRepositoryMockingTest {
       }
 
       @Override
-      BlobContentRef getBlobIncRef(String key, Decoder decoder, String url, String sha256) {
-        if(!Objects.equals(sha256, BlobRepositoryMockingTest.this.sha256)) return null;
-        blobKey = key;
-        blobFetched = true;
-        return new BlobContentRef(new BlobContent(key, filecontent)) ;
-      }
-
-      @Override
       ConcurrentHashMap<String, BlobContent> createMap() {
         return mapMock;
       }
@@ -138,13 +130,21 @@ public class BlobRepositoryMockingTest {
     when(mockContainer.isZooKeeperAware()).thenReturn(true);
     filecontent = TestDynamicLoading.getFileContent("runtimecode/runtimelibs_v2.jar.bin");
     url = "http://localhost:8080/myjar/location.jar";
-    sha256 = "79298d7d5c3e60d91154efe7d72f4536eac46698edfa22ab894b85492d562ed4";
     BlobRepository.BlobContentRef ref = repository.getBlobIncRef( "filefoo",null,url,
-        "79298d7d5c3e60d91154efe7d72f4536eac46698edfa22ab894b85492d562ed4");
+        "bc5ce45ad281b6a08fb7e529b1eb475040076834816570902acb6ebdd809410e31006efdeaa7f78a6c35574f3504963f5f7e4d92247d0eb4db3fc9abdda5d417");
     assertTrue("filefoo".equals(blobKey));
     assertTrue(blobFetched);
     assertNotNull(ref.blob);
     assertEquals(filecontent, ref.blob.get());
+    verify(mockContainer).isZooKeeperAware();
+    try {
+      repository.getBlobIncRef( "filefoo",null,url,
+          "WRONG-SHA512-KEY");
+      fail("expected exception");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains(" expected sha512 hash : WRONG-SHA512-KEY , actual :"));
+    }
+
     url = null;
     filecontent = null;
   }
diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
index 3a8f2e6..22ee299 100644
--- a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
+++ b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
@@ -48,7 +48,6 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase {
   // 12-Jun-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
   //17-Aug-2018 commented @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 20-Jul-2018
   public void testDynamicLoading() throws Exception {
-
     System.setProperty("enable.runtime.lib", "true");
     setupRestTestHarnesses();
 
@@ -98,7 +97,7 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase {
 
 
     assertNotNull(map = (Map) map.get("error"));
-    assertTrue("full output " + map, map.get("msg").toString().contains("no such resource available: colltest/1" ));
+    assertTrue("full output " + map, map.get("msg").toString().contains("no such blob or version available: colltest/1" ));
     payload = " {\n" +
         "  'set' : {'watched': {" +
         "                    'x':'X val',\n" +
@@ -129,6 +128,9 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase {
     }
     ByteBuffer jar = null;
 
+//     jar = persistZip("/tmp/runtimelibs.jar.bin", TestDynamicLoading.class, RuntimeLibReqHandler.class, RuntimeLibResponseWriter.class, RuntimeLibSearchComponent.class);
+//    if(true) return;
+
     jar = getFileContent("runtimecode/runtimelibs.jar.bin");
     TestBlobHandler.postAndCheck(cloudClient, baseURL, blobName, jar, 1);
 
@@ -282,8 +284,4 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase {
     return bos.getByteBuffer();
   }
 
-/*  public static void main(String[] args) throws Exception {
-    persistZip("/tmp/runtimelibs_v3.jar.bin", TestDynamicLoading.class, RuntimeLibReqHandler.class, RuntimeLibResponseWriter.class, RuntimeLibSearchComponent.class);
-    if(true) return;
-  }*/
 }
diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java b/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java
index 8fec3a4..575cf9e 100644
--- a/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java
+++ b/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java
@@ -77,7 +77,7 @@ public class TestDynamicLoadingUrl extends AbstractFullDistribZkTestBase {
     try {
       String payload = "{\n" +
           "'add-runtimelib' : { 'name' : 'urljar', url : 'http://localhost:" + port + "/jar1.jar'" +
-          "  'sha256':'e01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}" +
+          "  'sha512':'e01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}" +
           "}";
       RestTestHarness client = randomRestTestHarness();
       TestSolrConfigHandler.runConfigCommandExpectFailure(client, "/config", payload, "Invalid jar");
@@ -85,7 +85,7 @@ public class TestDynamicLoadingUrl extends AbstractFullDistribZkTestBase {
 
       payload = "{\n" +
           "'add-runtimelib' : { 'name' : 'urljar', url : 'http://localhost:" + port + "/jar1.jar'" +
-          "  'sha256':'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}" +
+          "  'sha512':'d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}" +
           "}";
       client = randomRestTestHarness();
       TestSolrConfigHandler.runConfigCommand(client, "/config", payload);
@@ -93,8 +93,8 @@ public class TestDynamicLoadingUrl extends AbstractFullDistribZkTestBase {
           null,
           "/config/overlay",
           null,
-          Arrays.asList("overlay", "runtimeLib", "urljar", "sha256"),
-          "e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc", 120);
+          Arrays.asList("overlay", "runtimeLib", "urljar", "sha512"),
+          "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420", 120);
 
       payload = "{\n" +
           "'create-requesthandler' : { 'name' : '/runtime', 'class': 'org.apache.solr.core.RuntimeLibReqHandler', 'runtimeLib' : true}" +
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
index 5f6a1c2..17494e0 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
@@ -41,7 +41,7 @@ import org.apache.solr.handler.DumpRequestHandler;
 import org.apache.solr.handler.TestSolrConfigHandlerConcurrent;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.search.SolrCacheHolder;
+import org.apache.solr.search.SolrCache;
 import org.apache.solr.util.RESTfulServerProvider;
 import org.apache.solr.util.RestTestBase;
 import org.apache.solr.util.RestTestHarness;
@@ -543,8 +543,8 @@ public class TestSolrConfigHandler extends RestTestBase {
         HashMap m = new HashMap();
         rsp.add("caches", m);
         for (String c : caches) {
-          SolrCacheHolder cache = (SolrCacheHolder) req.getSearcher().getCache(c);
-          if(cache != null) m.put(c, cache.get().getClass().getName());
+          SolrCache cache = req.getSearcher().getCache(c);
+          if(cache != null) m.put(c, cache.getClass().getName());
         }
       }
     }
diff --git a/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java b/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
index 30e8e20..2af16c5 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
@@ -18,48 +18,28 @@
 package org.apache.solr.handler;
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.Reader;
 import java.lang.invoke.MethodHandles;
-import java.net.URL;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.function.Predicate;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.GenericSolrRequest;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.request.V2Request;
-import org.apache.solr.client.solrj.response.SimpleSolrResponse;
 import org.apache.solr.client.solrj.response.V2Response;
-import org.apache.solr.cloud.ConfigRequest;
 import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.ConfigOverlay;
 import org.apache.solr.core.MemClassLoader;
 import org.apache.solr.core.RuntimeLib;
 import org.apache.solr.request.SolrRequestHandler;
@@ -72,16 +52,15 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
+import com.google.common.collect.ImmutableMap;
+
 import static org.apache.solr.cloud.TestCryptoKeys.readFile;
-import static org.apache.solr.common.params.CommonParams.JAVABIN;
-import static org.apache.solr.common.params.CommonParams.WT;
 import static org.apache.solr.common.util.Utils.getObjectByPath;
 import static org.apache.solr.core.TestDynamicLoading.getFileContent;
 import static org.apache.solr.core.TestDynamicLoadingUrl.runHttpServer;
 
 @SolrTestCaseJ4.SuppressSSL
-@LogLevel("org.apache.solr.common.cloud.ZkStateReader=DEBUG;org.apache.solr.handler.admin.CollectionHandlerApi=DEBUG;org.apache.solr.core.PackageManager=DEBUG;org.apache.solr.common.cloud.ClusterProperties=DEBUG")
+@LogLevel("org.apache.solr.common.cloud.ZkStateReader=DEBUG;org.apache.solr.handler.admin.CollectionHandlerApi=DEBUG;org.apache.solr.core.LibListener=DEBUG;org.apache.solr.common.cloud.ClusterProperties=DEBUG")
 public class TestContainerReqHandler extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
@@ -92,39 +71,40 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
   }
 
-  static SolrResponse assertResponseValues(int repeats, SolrClient client, SolrRequest req, Map vals) throws Exception {
-    SolrResponse rsp = null;
-
+  static void assertResponseValues(int repeats, SolrClient client, SolrRequest req, Map vals) throws Exception {
     for (int i = 0; i < repeats; i++) {
       if (i > 0) {
         Thread.sleep(100);
       }
       try {
-        rsp = req.process(client);
+        SolrResponse rsp = req.process(client);
+        try {
+          for (Object e : vals.entrySet()) {
+            Map.Entry entry = (Map.Entry) e;
+            String key = (String) entry.getKey();
+            Object val = entry.getValue();
+            Predicate p = val instanceof Predicate ? (Predicate) val : o -> {
+              String v = o == null ? null : String.valueOf(o);
+              return Objects.equals(val, o);
+            };
+            assertTrue("attempt: " + i + " Mismatch for value : '" + key + "' in response " + Utils.toJSONString(rsp),
+                p.test(rsp.getResponse()._get(key, null)));
+
+          }
+          return;
+        } catch (Exception e) {
+          if (i >= repeats - 1) throw e;
+          continue;
+        }
+
       } catch (Exception e) {
         if (i >= repeats - 1) throw e;
+        log.error("exception in request", e);
         continue;
       }
-      for (Object e : vals.entrySet()) {
-        Map.Entry entry = (Map.Entry) e;
-        String k = (String) entry.getKey();
-        List<String> key = StrUtils.split(k, '/');
-
-        Object val = entry.getValue();
-        Predicate p = val instanceof Predicate ? (Predicate) val : o -> {
-          String v = o == null ? null : String.valueOf(o);
-          return Objects.equals(val, o);
-        };
-        boolean isPass = p.test(rsp._get(key, null));
-        if (isPass) return rsp;
-        else if (i >= repeats - 1) {
-          fail("attempt: " + i + " Mismatch for value : '" + key + "' in response " + Utils.toJSONString(rsp));
-        }
+    }
 
-      }
 
-    }
-    return rsp;
   }
 
   private static Map<String, Object> assertVersionInSync(SolrZkClient zkClient, SolrClient solrClient) throws SolrServerException, IOException {
@@ -142,7 +122,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
   @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-13781")
   @Test
-  public void testPackageAPI() throws Exception {
+  public void testRuntimeLib() throws Exception {
     Map<String, Object> jars = Utils.makeMap(
         "/jar1.jar", getFileContent("runtimecode/runtimelibs.jar.bin"),
         "/jar2.jar", getFileContent("runtimecode/runtimelibs_v2.jar.bin"),
@@ -154,20 +134,20 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
     try {
       String payload = null;
       try {
-        payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
-            "sha256 : 'wrong-sha256'}}";
+        payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
+            "sha512 : 'wrong-sha512'}}";
         new V2Request.Builder("/cluster")
             .withPayload(payload)
             .withMethod(SolrRequest.METHOD.POST)
             .build().process(cluster.getSolrClient());
         fail("Expected error");
       } catch (BaseHttpSolrClient.RemoteExecutionException e) {
-        assertTrue("actual output : " + Utils.toJSONString(e.getMetaData()), e.getMetaData()._getStr("error/details[0]/errorMessages[0]", "").contains("expected sha256 hash :"));
+        assertTrue("actual output : " + Utils.toJSONString(e.getMetaData()), e.getMetaData()._getStr("error/details[0]/errorMessages[0]", "").contains("expected sha512 hash :"));
       }
 
       try {
-        payload = "{add-package:{name : 'foo', url: 'http://localhost:" + port + "/jar0.jar', " +
-            "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
+        payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar0.jar', " +
+            "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
         new V2Request.Builder("/cluster")
             .withPayload(payload)
             .withMethod(SolrRequest.METHOD.POST)
@@ -177,18 +157,18 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
         assertTrue("Actual output : " + Utils.toJSONString(e.getMetaData()), e.getMetaData()._getStr("error/details[0]/errorMessages[0]", "").contains("no such resource available: foo"));
       }
 
-      payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
-          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
+      payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
+          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-runtimelib/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
 
 
       new V2Request.Builder("/cluster")
-          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler', package : global}}")
+          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler'}}")
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
       Map<String, Object> map = new ClusterProperties(cluster.getZkClient()).getClusterProperties();
@@ -207,14 +187,14 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           getObjectByPath(map, true, Arrays.asList("requestHandler", "bar", "class")));
 
 
-      payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
-          "sha256 : '20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3'}}";
+      payload = "{update-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar3.jar', " +
+          "sha512 : 'f67a7735a89b4348e273ca29e4651359d6d976ba966cb871c4b468ea1dbd452e42fcde9d188b7788e5a1ef668283c690606032922364759d19588666d5862653'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-runtimelib/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -237,23 +217,16 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       assertResponseValues(10, cluster.getSolrClient(), request, ImmutableMap.of(SolrRequestHandler.TYPE,
           (Predicate<Object>) o -> o instanceof List && ((List) o).isEmpty()));
       new V2Request.Builder("/cluster")
-          .withPayload("{delete-package : 'global'}")
+          .withPayload("{delete-runtimelib : 'foo'}")
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
       assertResponseValues(10, cluster.getSolrClient(), request, ImmutableMap.of(RuntimeLib.TYPE,
           (Predicate<Object>) o -> o instanceof List && ((List) o).isEmpty()));
 
 
-      URL baseUrl = cluster.getRandomJetty(random()).getBaseUrl();
-      try(HttpSolrClient client = new HttpSolrClient.Builder(baseUrl.toString()).build()){
-        SimpleSolrResponse rsp = new GenericSolrRequest(SolrRequest.METHOD.GET, "/____v2/node/blob", new ModifiableSolrParams()).process(client);
-        List l = (List) rsp.nl.get("blob");
-        assertTrue(l.contains("e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc"));
-        assertTrue(l.contains("20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3"));
-      }
     } finally {
-      cluster.shutdown();
       server.first().stop();
+      cluster.shutdown();
     }
   }
 
@@ -266,7 +239,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
     Pair<Server, Integer> server = runHttpServer(jars);
     int port = server.second();
-    MiniSolrCloudCluster cluster = configureCluster(4).configure();
+    MiniSolrCloudCluster cluster =  configureCluster(4).configure();
 
     try {
 
@@ -276,9 +249,9 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       String signature = "NaTm3+i99/ZhS8YRsLc3NLz2Y6VuwEbu7DihY8GAWwWIGm+jpXgn1JiuaenfxFCcfNKCC9WgZmEgbTZTzmV/OZMVn90u642YJbF3vTnzelW1pHB43ZRAJ1iesH0anM37w03n3es+vFWQtuxc+2Go888fJoMkUX2C6Zk6Jn116KE45DWjeyPM4mp3vvGzwGvdRxP5K9Q3suA+iuI/ULXM7m9mV4ruvs/MZvL+ELm5Jnmk1bBtixVJhQwJP2z++8tQKJghhyBxPIC/2fkAHobQpkhZrXu56JjP+v33ul3Ku4bbvfVMY/LVwCAEnxlvhk+C6uRCKCeFMrzQ/k5inasXLw==";
 
-      String payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
+      String payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : 'EdYkvRpMZbvElN93/xUmyKXcj6xHP16AVk71TlTascEwCb5cFQ2AeKhPIlwYpkLWXEOcLZKfeXoWwOLaV5ZNhg==' ," +
-          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
+          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
       try {
         new V2Request.Builder("/cluster")
             .withPayload(payload)
@@ -290,19 +263,19 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       }
 
 
-      payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
+      payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : '" + signature + "'," +
-          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
+          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-runtimelib/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
 
       new V2Request.Builder("/cluster")
-          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler' package : global}}")
+          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler'}}")
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
       Map<String, Object> map = new ClusterProperties(cluster.getZkClient()).getClusterProperties();
@@ -320,16 +293,16 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       assertEquals("org.apache.solr.core.RuntimeLibReqHandler",
           getObjectByPath(map, true, Arrays.asList("requestHandler", "bar", "class")));
 
-      payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
-          "sig : 'YxFr6SpYrDwG85miDfRWHTjU9UltjtIWQZEhcV55C2rczRUVowCYBxmsDv5mAM8j0CTv854xpI1DtBT86wpoTdbF95LQuP9FJId4TS1j8bZ9cxHP5Cqyz1uBHFfUUNUrnpzTHQkVTp02O9NAjh3c2W41bL4U7j6jQ32+4CW2M+x00TDG0y0H75rQDR8zbLt31oWCz+sBOdZ3rGKJgAvdoGm/wVCTmsabZN+xoz4JaDeBXF16O9Uk9SSq4G0dz5YXFuLxHK7ciB5t0+q6pXlF/tdlDqF76Abze0R3d2/0MhXBzyNp3UxJmj6DiprgysfB0TbQtJG0XGfdSmx0VChvcA==' ," +
-          "sha256 : '20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3'}}";
+      payload = "{update-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar3.jar', " +
+          "sig : 'BSx/v0eKWX+LzkWF+iIAzwGL9rezWMePsyRzi4TvV6boATZ9cSfeUAqUgRW50f/hAHX4/hrHr2Piy8za9tIUoXbLqn3xJNNroOqpcVEgwh1Zii4c7zPwUSB9gtd9zlAK4LAPLdjxILS8NXpTD2zLycc8kSpcyTpSTITqz6HA3HsPGC81WIq2k3IRqYAkacn46viW+nnEjA7OxDCOqoL//evjxDWQ6R1YggTGh4u5MSWZJCiCPJNQnTlPRzUZOAJjtX7PblDrKeiunKGbjtiOhFLYkupe1lSlIRLiJV/qqopO4TQGO1bhbxeCKAX2vEz5Ch5bGOa+VZLJJGaDo318UQ==' ," +
+          "sha512 : 'f67a7735a89b4348e273ca29e4651359d6d976ba966cb871c4b468ea1dbd452e42fcde9d188b7788e5a1ef668283c690606032922364759d19588666d5862653'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-runtimelib/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -357,7 +330,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
     Pair<Server, Integer> server = runHttpServer(jars);
     int port = server.second();
-    MiniSolrCloudCluster cluster = configureCluster(4).configure();
+    MiniSolrCloudCluster cluster =  configureCluster(4).configure();
 
     try {
 
@@ -367,19 +340,19 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       String signature = "L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1f/U3bOlMPINlSOM6LK3JpQ==";
 
-      String payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
+      String payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : '" + signature + "'," +
-          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
+          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-runtimelib/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
 
       new V2Request.Builder("/cluster")
-          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler' package : global }}")
+          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler'}}")
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
       Map<String, Object> map = new ClusterProperties(cluster.getZkClient()).getClusterProperties();
@@ -397,16 +370,16 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       assertEquals("org.apache.solr.core.RuntimeLibReqHandler",
           getObjectByPath(map, true, Arrays.asList("requestHandler", "bar", "class")));
 
-      payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
-          "sig : 'a400n4T7FT+2gM0SC6+MfSOExjud8MkhTSFylhvwNjtWwUgKdPFn434Wv7Qc4QEqDVLhQoL3WqYtQmLPti0G4Q==' ," +
-          "sha256 : '20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3'}}";
+      payload = "{update-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar3.jar', " +
+          "sig : 'pnH8uDHsTF0HWyQqABqVWmvo3rM/Mp2qpuo6S9YXZA9Ifg8NjHX8WzPe6EzlaqBcYcusrEV0b+5NCBx4AS0TGA==' ," +
+          "sha512 : 'f67a7735a89b4348e273ca29e4651359d6d976ba966cb871c4b468ea1dbd452e42fcde9d188b7788e5a1ef668283c690606032922364759d19588666d5862653'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-runtimelib/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -457,326 +430,4 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
     }
 
   }
-
-  public void testPluginFrompackage() throws Exception {
-    String COLLECTION_NAME = "globalLoaderColl";
-    Map<String, Object> jars = Utils.makeMap(
-        "/jar1.jar", getFileContent("runtimecode/runtimelibs.jar.bin"),
-        "/jar2.jar", getFileContent("runtimecode/runtimelibs_v2.jar.bin"),
-        "/jar3.jar", getFileContent("runtimecode/runtimelibs_v3.jar.bin"));
-
-    Pair<Server, Integer> server = runHttpServer(jars);
-    int port = server.second();
-    System.setProperty("enable.runtime.lib", "true");
-    MiniSolrCloudCluster cluster = configureCluster(4)
-        .addConfig("conf", configset("cloud-minimal"))
-        .configure();
-    try {
-      CollectionAdminRequest
-          .createCollection(COLLECTION_NAME, "conf", 2, 1)
-          .setMaxShardsPerNode(100)
-          .process(cluster.getSolrClient());
-
-
-      cluster.waitForActiveCollection(COLLECTION_NAME, 2, 2);
-      String payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
-          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
-      new V2Request.Builder("/cluster")
-          .withPayload(payload)
-          .withMethod(SolrRequest.METHOD.POST)
-          .build().process(cluster.getSolrClient());
-      String sha256 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256");
-      String url = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/url");
-
-      assertEquals(sha256,
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
-
-
-      payload = "{\n" +
-          "'create-requesthandler' : { 'name' : '/runtime', 'class': 'org.apache.solr.core.RuntimeLibReqHandler' , 'package':global }," +
-          "'create-searchcomponent' : { 'name' : 'get', 'class': 'org.apache.solr.core.RuntimeLibSearchComponent' , 'package':global }," +
-          "'create-queryResponseWriter' : { 'name' : 'json1', 'class': 'org.apache.solr.core.RuntimeLibResponseWriter' , 'package':global }" +
-          "}";
-      cluster.getSolrClient().request(new ConfigRequest(payload) {
-        @Override
-        public String getCollection() {
-          return COLLECTION_NAME;
-        }
-      });
-
-      SolrParams params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-          WT, JAVABIN,
-          "meta","true"));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/queryResponseWriter/json1", params),
-          Utils.makeMap(
-              "/config/queryResponseWriter/json1/_packageinfo_/url", url,
-              "/config/queryResponseWriter/json1/_meta_/sha256", sha256
-          ));
-
-      params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-          WT, JAVABIN,
-          "meta","true"));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/searchComponent/get", params),
-          Utils.makeMap(
-              "config/searchComponent/get/_packageinfo_/url", url,
-              "config/searchComponent/get/_packageinfo_/sha256", sha256
-          ));
-
-      params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-          WT, JAVABIN,
-          "meta","true"));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/requestHandler/runtime", params),
-          Utils.makeMap(
-              ":config:requestHandler:/runtime:_packageinfo_:url", url,
-              ":config:requestHandler:/runtime:_packageinfo_:sha256", sha256
-          ));
-
-
-      params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME, WT, JAVABIN));
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/overlay", params),
-          Utils.makeMap(
-              "overlay/queryResponseWriter/json1/class", "org.apache.solr.core.RuntimeLibResponseWriter",
-              "overlay/searchComponent/get/class", "org.apache.solr.core.RuntimeLibSearchComponent"
-          ));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/runtime", params),
-          Utils.makeMap("class", "org.apache.solr.core.RuntimeLibReqHandler",
-              "loader", MemClassLoader.class.getName()));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/get?abc=xyz", params),
-          Utils.makeMap("get", "org.apache.solr.core.RuntimeLibSearchComponent",
-              "loader", MemClassLoader.class.getName()));
-
-      GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/runtime",
-          new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME, WT, "json1")));
-      req.setResponseParser(new ResponseParser() {
-        @Override
-        public String getWriterType() {
-          return "json1";
-        }
-
-        @Override
-        public NamedList<Object> processResponse(InputStream body, String encoding) {
-          return new NamedList<>((Map) Utils.fromJSON(body));
-        }
-
-        @Override
-        public NamedList<Object> processResponse(Reader reader) {
-          return new NamedList<>((Map) Utils.fromJSON(reader));
-
-        }
-
-      });
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          req,
-          Utils.makeMap("wt", "org.apache.solr.core.RuntimeLibResponseWriter",
-              "loader", MemClassLoader.class.getName()));
-
-
-      payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar2.jar', " +
-          "sha256 : '79298d7d5c3e60d91154efe7d72f4536eac46698edfa22ab894b85492d562ed4'}}";
-      new V2Request.Builder("/cluster")
-          .withPayload(payload)
-          .withMethod(SolrRequest.METHOD.POST)
-          .build().process(cluster.getSolrClient());
-      sha256 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256");
-      url = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/url");
-
-      assertEquals(sha256,
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
-
-      params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-          WT, JAVABIN,
-          "meta","true"));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/queryResponseWriter/json1", params),
-          Utils.makeMap(
-              "/config/queryResponseWriter/json1/_packageinfo_/url", url,
-              "/config/queryResponseWriter/json1/_packageinfo_/sha256", sha256
-          ));
-
-      params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-          WT, JAVABIN,
-          "meta","true"));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/searchComponent/get", params),
-          Utils.makeMap(
-              "/config/searchComponent/get/_packageinfo_/url", url,
-              "/config/searchComponent/get/_packageinfo_/sha256", sha256
-          ));
-
-      params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-          WT, JAVABIN,
-          "meta","true"));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/requestHandler/runtime", params),
-          Utils.makeMap(
-              ":config:requestHandler:/runtime:_packageinfo_:url", url,
-              ":config:requestHandler:/runtime:_packageinfo_:sha256", sha256
-          ));
-
-
-
-      try {
-        new V2Request.Builder("/cluster")
-            .withPayload(payload)
-            .withMethod(SolrRequest.METHOD.POST)
-            .build().process(cluster.getSolrClient());
-        fail("should have failed");
-      } catch (BaseHttpSolrClient.RemoteExecutionException e) {
-        assertTrue("actual output : " + Utils.toJSONString(e.getMetaData()), e.getMetaData()._getStr("error/details[0]/errorMessages[0]", "").contains("Trying to update a jar with the same sha256"));
-      }
-
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/get?abc=xyz", params),
-          Utils.makeMap("get", "org.apache.solr.core.RuntimeLibSearchComponent",
-              "loader", MemClassLoader.class.getName(),
-              "Version", "2"));
-    } finally {
-      cluster.deleteAllCollections();
-      cluster.shutdown();
-      server.first().stop();
-    }
-
-  }
-
-//  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-13650")
-  public void testCacheLoadFromPackage() throws Exception {
-    String COLLECTION_NAME = "globalCacheColl";
-    Map<String, Object> jars = Utils.makeMap(
-        "/jar1.jar", getFileContent("runtimecode/cache.jar.bin"),
-        "/jar2.jar", getFileContent("runtimecode/cache_v2.jar.bin"));
-
-    Pair<Server, Integer> server = runHttpServer(jars);
-    int port = server.second();
-
-    String overlay = "{" +
-        "    \"props\":{\"query\":{\"documentCache\":{\n" +
-        "          \"class\":\"org.apache.solr.core.MyDocCache\",\n" +
-        "          \"size\":\"512\",\n" +
-        "          \"initialSize\":\"512\" , \"package\":\"cache_pkg\"}}}}";
-    MiniSolrCloudCluster cluster = configureCluster(4)
-        .addConfig("conf", configset("cloud-minimal"),
-            Collections.singletonMap(ConfigOverlay.RESOURCE_NAME, overlay.getBytes(UTF_8)))
-        .configure();
-    try {
-      String payload = "{add-package:{name : 'cache_pkg', url: 'http://localhost:" + port + "/jar1.jar', " +
-          "sha256 : '32e8b5b2a95ea306538b52017f0954aa1b0f8a8b2d0acbc498fd0e66a223f7bd'}}";
-
-      new V2Request.Builder("/cluster")
-          .withPayload(payload)
-          .withMethod(SolrRequest.METHOD.POST)
-          .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/cache_pkg/sha256"));
-
-      CollectionAdminRequest
-          .createCollection(COLLECTION_NAME, "conf", 2, 1)
-          .setMaxShardsPerNode(100)
-          .process(cluster.getSolrClient());
-
-
-      cluster.waitForActiveCollection(COLLECTION_NAME, 2, 2);
-      SolrParams params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME, WT, JAVABIN));
-
-      NamedList<Object> rsp = cluster.getSolrClient().request(new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/overlay", params));
-      assertEquals("org.apache.solr.core.MyDocCache", rsp._getStr("overlay/props/query/documentCache/class", null));
-
-      String sha256 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256");
-      String url = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/url");
-
-
-      params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-          WT, JAVABIN,
-          "meta","true"));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/query/documentCache", params),
-          Utils.makeMap(
-              "/config/query/documentCache/_packageinfo_/url", url,
-              "/config/query/documentCache/_packageinfo_/sha256", sha256
-          ));
-
-
-      UpdateRequest req = new UpdateRequest();
-
-      req.add("id", "1", "desc_s", "document 1")
-          .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true)
-          .setWaitSearcher(true);
-      cluster.getSolrClient().request(req, COLLECTION_NAME);
-
-      SolrQuery solrQuery = new SolrQuery("q", "id:1", "collection", COLLECTION_NAME);
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new QueryRequest(solrQuery),
-          Utils.makeMap("/response[0]/my_synthetic_fld_s", "version_1"));
-
-
-      payload = "{update-package:{name : 'cache_pkg', url: 'http://localhost:" + port + "/jar2.jar', " +
-          "sha256 : '0f670f6dcc2b00f9a448a7ebd457d4ff985ab702c85cdb3608dcae9889e8d702'}}";
-      new V2Request.Builder("/cluster")
-          .withPayload(payload)
-          .withMethod(SolrRequest.METHOD.POST)
-          .build().process(cluster.getSolrClient());
-      sha256 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256");
-      url = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/url");
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/cache_pkg/sha256"));
-
-      params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-          WT, JAVABIN,
-          "meta","true"));
-
-      assertResponseValues(10,
-          cluster.getSolrClient(),
-          new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/query/documentCache", params),
-          Utils.makeMap(
-              "/config/query/documentCache/_packageinfo_/url", url,
-              "/config/query/documentCache/_packageinfo_/sha256", sha256
-          ));
-      req = new UpdateRequest();
-      req.add("id", "2", "desc_s", "document 1")
-          .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true)
-          .setWaitSearcher(true);
-      cluster.getSolrClient().request(req, COLLECTION_NAME);
-
-
-      solrQuery = new SolrQuery("q", "id:2", "collection", COLLECTION_NAME);
-      SolrResponse result = assertResponseValues(10,
-          cluster.getSolrClient(),
-          new QueryRequest(solrQuery),
-          Utils.makeMap("response[0]/my_synthetic_fld_s", "version_2"));
-
-    } finally {
-      cluster.deleteAllCollections();
-      cluster.shutdown();
-      server.first().stop();
-    }
-  }
-
 }
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
index 356e865..a6dbd9e 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -53,12 +53,12 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
   @AfterClass
   public static void cleanupMetrics() throws Exception {
     if (null != h) {
-      h.getCoreContainer().getMetricManager().registry("solr.jvm").remove("solrtest_foo");
+      h.getCoreContainer().getMetricManager().registry("solr.jvm"  ).remove("solrtest_foo");
       h.getCoreContainer().getMetricManager().registry("solr.jetty").remove("solrtest_foo");
       h.getCoreContainer().getMetricManager().registry("solr.jetty").remove("solrtest_foo:bar");
     }
   }
-
+  
   @Test
   public void test() throws Exception {
     MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
@@ -145,7 +145,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNotNull(values.get("metrics"));
     values = (NamedList) values.get("metrics");
     assertEquals(1, values.size());
-    assertEquals(13, ((NamedList) values.get("solr.node")).size());
+    assertEquals(13, ((NamedList)values.get("solr.node")).size());
     assertNotNull(values.get("solr.node"));
     values = (NamedList) values.get("solr.node");
     assertNotNull(values.get("CONTAINER.cores.lazy")); // this is a gauge node
@@ -171,7 +171,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNotNull(values.get("solr.core.collection1"));
     values = (NamedList) values.get("solr.core.collection1");
     assertEquals(1, values.size());
-    Map m = (Map) values.get("CACHE.core.fieldCache");
+    Map m = (Map)values.get("CACHE.core.fieldCache");
     assertNotNull(m);
     assertNotNull(m.get("entries_count"));
 
@@ -223,7 +223,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertTrue(nl.size() > 0);
     nl.forEach((k, v) -> {
       assertTrue(v instanceof Map);
-      Map map = (Map) v;
+      Map map = (Map)v;
       assertTrue(map.size() > 2);
     });
 
@@ -238,7 +238,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertTrue(nl.size() > 0);
     nl.forEach((k, v) -> {
       assertTrue(v instanceof Map);
-      Map map = (Map) v;
+      Map map = (Map)v;
       assertEquals(2, map.size());
       assertNotNull(map.get("inserts"));
       assertNotNull(map.get("size"));
@@ -257,7 +257,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     Object val = values.findRecursive("metrics", key1);
     assertNotNull(val);
     assertTrue(val instanceof Map);
-    assertTrue(((Map) val).size() >= 2);
+    assertTrue(((Map)val).size() >= 2);
 
     String key2 = "solr.core.collection1:CACHE.core.fieldCache:entries_count";
     resp = new SolrQueryResponse();
@@ -276,7 +276,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     val = values.findRecursive("metrics", key3);
     assertNotNull(val);
     assertTrue(val instanceof Number);
-    assertEquals(3, ((Number) val).intValue());
+    assertEquals(3, ((Number)val).intValue());
 
     // test multiple keys
     resp = new SolrQueryResponse();
@@ -306,7 +306,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json",
         MetricsHandler.KEY_PARAM, "foo", MetricsHandler.KEY_PARAM, "foo:bar:baz:xyz"), resp);
     values = resp.getValues();
-    NamedList metrics = (NamedList) values.get("metrics");
+    NamedList metrics = (NamedList)values.get("metrics");
     assertEquals(0, metrics.size());
     assertNotNull(values.findRecursive("errors", "foo"));
     assertNotNull(values.findRecursive("errors", "foo:bar:baz:xyz"));
@@ -316,7 +316,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json",
         MetricsHandler.KEY_PARAM, "foo:bar:baz"), resp);
     values = resp.getValues();
-    metrics = (NamedList) values.get("metrics");
+    metrics = (NamedList)values.get("metrics");
     assertEquals(0, metrics.size());
     assertNotNull(values.findRecursive("errors", "foo:bar:baz"));
 
@@ -325,7 +325,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json",
         MetricsHandler.KEY_PARAM, "solr.jetty:unknown:baz"), resp);
     values = resp.getValues();
-    metrics = (NamedList) values.get("metrics");
+    metrics = (NamedList)values.get("metrics");
     assertEquals(0, metrics.size());
     assertNotNull(values.findRecursive("errors", "solr.jetty:unknown:baz"));
   }
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
index 1762ec6..01a5bca 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -19,20 +19,14 @@ package org.apache.solr.handler.admin;
 
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.api.Api;
 import org.apache.solr.api.ApiBag;
-import org.apache.solr.api.V2HttpCall;
 import org.apache.solr.api.V2HttpCall.CompositeApi;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.util.CommandOperation;
-import org.apache.solr.common.util.PathTrie;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.common.util.ValidatingJsonMap;
@@ -45,6 +39,10 @@ import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.V2HttpCall;
+import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.PathTrie;
 
 import static org.apache.solr.api.ApiBag.EMPTY_SPEC;
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
@@ -73,7 +71,7 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     Map<String, String> parts = new HashMap<>();
     String fullPath = "/collections/hello/shards";
     Api api = V2HttpCall.getApiInfo(containerHandlers, fullPath, "POST",
-        fullPath, parts);
+       fullPath, parts);
     assertNotNull(api);
     assertConditions(api.getSpec(), Utils.makeMap(
         "/methods[0]", "POST",
@@ -83,7 +81,7 @@ public class TestApiFramework extends SolrTestCaseJ4 {
 
     parts = new HashMap<>();
     api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards", "POST",
-        null, parts);
+      null, parts);
     assertConditions(api.getSpec(), Utils.makeMap(
         "/methods[0]", "POST",
         "/commands/split", NOT_NULL,
@@ -104,7 +102,7 @@ public class TestApiFramework extends SolrTestCaseJ4 {
 
     parts = new HashMap<>();
     api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello", "POST",
-        null, parts);
+       null, parts);
     assertConditions(api.getSpec(), Utils.makeMap(
         "/methods[0]", "POST",
         "/commands/add-replica-property", NOT_NULL,
@@ -113,7 +111,7 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     assertEquals("hello", parts.get("collection"));
 
     api = V2HttpCall.getApiInfo(containerHandlers, "/collections/hello/shards/shard1/replica1", "DELETE",
-        null, parts);
+       null, parts);
     assertConditions(api.getSpec(), Utils.makeMap(
         "/methods[0]", "DELETE",
         "/url/params/onlyIfDown/type", "boolean"
@@ -124,21 +122,18 @@ public class TestApiFramework extends SolrTestCaseJ4 {
 
     SolrQueryResponse rsp = invoke(containerHandlers, null, "/collections/_introspect", GET, mockCC);
 
-    Set<String> methodNames = new HashSet<>();
-    methodNames.add(rsp.getValues()._getStr("/spec[0]/methods[0]", null));
-    methodNames.add(rsp.getValues()._getStr("/spec[1]/methods[0]", null));
-    methodNames.add(rsp.getValues()._getStr("/spec[2]/methods[0]", null));
-    assertTrue(methodNames.contains("DELETE"));
-    assertTrue(methodNames.contains("POST"));
-    assertTrue(methodNames.contains("GET"));
+    assertConditions(rsp.getValues().asMap(2), Utils.makeMap(
+        "/spec[0]/methods[0]", "DELETE",
+        "/spec[1]/methods[0]", "POST",
+        "/spec[2]/methods[0]", "GET"
 
-    methodNames = new HashSet<>();
+    ));
 
     rsp = invoke(coreHandlers, "/schema/_introspect", "/collections/hello/schema/_introspect", GET, mockCC);
-    methodNames.add(rsp.getValues()._getStr("/spec[0]/methods[0]", null));
-    methodNames.add(rsp.getValues()._getStr("/spec[1]/methods[0]", null));
-    assertTrue(methodNames.contains("POST"));
-    assertTrue(methodNames.contains("GET"));
+    assertConditions(rsp.getValues().asMap(2), Utils.makeMap(
+        "/spec[0]/methods[0]", "POST",
+        "/spec[0]/commands", NOT_NULL,
+        "/spec[1]/methods[0]", "GET"));
 
     rsp = invoke(coreHandlers, "/", "/collections/hello/_introspect", GET, mockCC);
     assertConditions(rsp.getValues().asMap(2), Utils.makeMap(
@@ -151,17 +146,16 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     ));
 
   }
-
-  public void testTrailingTemplatePaths() {
-    PathTrie<Api> registry = new PathTrie<>();
+  public void testTrailingTemplatePaths(){
+    PathTrie<Api> registry =  new PathTrie<>();
     Api api = new Api(EMPTY_SPEC) {
       @Override
       public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
 
       }
     };
-    Api intropsect = new ApiBag.IntrospectApi(api, false);
-    ApiBag.registerIntrospect(Collections.emptyMap(), registry, "/c/.system/blob/{name}", intropsect);
+    Api intropsect = new ApiBag.IntrospectApi(api,false);
+    ApiBag.registerIntrospect(Collections.emptyMap(),registry,"/c/.system/blob/{name}",intropsect);
     ApiBag.registerIntrospect(Collections.emptyMap(), registry, "/c/.system/{x}/{name}", intropsect);
     assertEquals(intropsect, registry.lookup("/c/.system/blob/random_string/_introspect", new HashMap<>()));
     assertEquals(intropsect, registry.lookup("/c/.system/blob/_introspect", new HashMap<>()));
@@ -169,7 +163,6 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     assertEquals(intropsect, registry.lookup("/c/.system/v1/_introspect", new HashMap<>()));
     assertEquals(intropsect, registry.lookup("/c/.system/v1/v2/_introspect", new HashMap<>()));
   }
-
   private SolrQueryResponse invoke(PluginBag<SolrRequestHandler> reqHandlers, String path,
                                    String fullPath, SolrRequest.METHOD method,
                                    CoreContainer mockCC) {
@@ -191,14 +184,14 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     }
 
     SolrQueryResponse rsp = new SolrQueryResponse();
-    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, new MapSolrParams(new HashMap<>())) {
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, new MapSolrParams(new HashMap<>())){
       @Override
       public List<CommandOperation> getCommands(boolean validateInput) {
         return Collections.emptyList();
       }
     };
 
-    api.call(req, rsp);
+    api.call(req,rsp);
     return rsp;
 
   }
@@ -208,12 +201,12 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     for (Object o : conditions.entrySet()) {
       Map.Entry e = (Map.Entry) o;
       String path = (String) e.getKey();
-      List<String> parts = StrUtils.splitSmart(path, path.charAt(0) == '/' ? '/' : ' ', true);
+      List<String> parts = StrUtils.splitSmart(path, path.charAt(0) == '/' ?  '/':' ', true);
       Object val = Utils.getObjectByPath(root, false, parts);
       if (e.getValue() instanceof ValidatingJsonMap.PredicateWithErrMsg) {
         ValidatingJsonMap.PredicateWithErrMsg value = (ValidatingJsonMap.PredicateWithErrMsg) e.getValue();
         String err = value.test(val);
-        if (err != null) {
+        if(err != null){
           assertEquals(err + " for " + e.getKey() + " in :" + Utils.toJSONString(root), e.getValue(), val);
         }
 
diff --git a/solr/core/src/test/org/apache/solr/search/TestLFUCache.java b/solr/core/src/test/org/apache/solr/search/TestLFUCache.java
index 05e7557..7989d8e 100644
--- a/solr/core/src/test/org/apache/solr/search/TestLFUCache.java
+++ b/solr/core/src/test/org/apache/solr/search/TestLFUCache.java
@@ -63,7 +63,7 @@ public class TestLFUCache extends SolrTestCaseJ4 {
   @Test
   public void testTimeDecayParams() throws IOException {
     h.getCore().withSearcher(searcher -> {
-      LFUCache cacheDecayTrue = (LFUCache) ((SolrCacheHolder) searcher.getCache("lfuCacheDecayTrue")).get();
+      LFUCache cacheDecayTrue = (LFUCache) searcher.getCache("lfuCacheDecayTrue");
       assertNotNull(cacheDecayTrue);
       Map<String,Object> stats = cacheDecayTrue.getMetricsMap().getValue();
       assertTrue((Boolean) stats.get("timeDecay"));
@@ -74,7 +74,7 @@ public class TestLFUCache extends SolrTestCaseJ4 {
       addCache(cacheDecayTrue, 11, 12, 13, 14, 15);
       assertCache(cacheDecayTrue, 1, 2, 3, 4, 5, 12, 13, 14, 15);
 
-      LFUCache cacheDecayDefault = (LFUCache) ((SolrCacheHolder) searcher.getCache("lfuCacheDecayDefault")).get();
+      LFUCache cacheDecayDefault = (LFUCache) searcher.getCache("lfuCacheDecayDefault");
       assertNotNull(cacheDecayDefault);
       stats = cacheDecayDefault.getMetricsMap().getValue();
       assertTrue((Boolean) stats.get("timeDecay"));
@@ -88,7 +88,7 @@ public class TestLFUCache extends SolrTestCaseJ4 {
       addCache(cacheDecayDefault, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21);
       assertCache(cacheDecayDefault, 1, 2, 3, 4, 5, 17, 18, 19, 20, 21);
 
-      LFUCache cacheDecayFalse = (LFUCache) ((SolrCacheHolder) searcher.getCache("lfuCacheDecayFalse")).get();
+      LFUCache cacheDecayFalse = (LFUCache) searcher.getCache("lfuCacheDecayFalse");
       assertNotNull(cacheDecayFalse);
       stats = cacheDecayFalse.getMetricsMap().getValue();
       assertFalse((Boolean) stats.get("timeDecay"));
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 96c82ab..39c5e1c 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -138,9 +138,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
 
       final SolrRequest genericReq;
       if (isUseV2Api) {
-        genericReq = new V2Request.Builder("/cluster/security/authentication")
-            .withPayload(command)
-            .withMethod(SolrRequest.METHOD.POST).build();
+        genericReq = new V2Request.Builder("/cluster/security/authentication").withMethod(SolrRequest.METHOD.POST).build();
       } else {
         genericReq = new GenericSolrRequest(SolrRequest.METHOD.POST, authcPrefix, new ModifiableSolrParams());
         ((GenericSolrRequest)genericReq).setContentWriter(new StringPayloadContentWriter(command, CommonParams.JSON_MIME));
diff --git a/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java b/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java
index 6cee3d9..889b0bf 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java
@@ -31,7 +31,7 @@ public class RuntimeUrp extends SimpleUpdateProcessorFactory {
     List<String>  names = new ArrayList<>();
     for (UpdateRequestProcessorFactory p : processorChain.getProcessors()) {
       if (p instanceof UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder.LazyUpdateRequestProcessorFactory) {
-        p = ((UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder.LazyUpdateRequestProcessorFactory) p).getDelegate();
+        p = ((UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder.LazyUpdateRequestProcessorFactory) p).delegate;
       }
       names.add(p.getClass().getSimpleName());
     }
diff --git a/solr/solr-ref-guide/src/adding-custom-plugins-in-solrcloud-mode.adoc b/solr/solr-ref-guide/src/adding-custom-plugins-in-solrcloud-mode.adoc
index 5738c6f..2f5ed58 100644
--- a/solr/solr-ref-guide/src/adding-custom-plugins-in-solrcloud-mode.adoc
+++ b/solr/solr-ref-guide/src/adding-custom-plugins-in-solrcloud-mode.adoc
@@ -128,11 +128,11 @@ Step 1: Download a jar from github to the current directory
 ----
  curl -o runtimelibs.jar   -LO https://github.com/apache/lucene-solr/blob/master/solr/core/src/test-files/runtimecode/runtimelibs.jar.bin?raw=true
 ----
-Step 2: Get the `sha256` hash of the jar
+Step 2: Get the `sha512` hash of the jar
 
 [source,bash]
 ----
- openssl dgst -sha256 runtimelibs.jar
+ openssl dgst -sha512 runtimelibs.jar
 ----
 
 Step 3 :  Start solr with runtime lib enabled
@@ -154,9 +154,9 @@ Step 5: Add the jar to your collection `gettingstarted`
 [source,bash]
 ----
  curl http://localhost:8983/solr/gettingstarted/config -H 'Content-type:application/json' -d '{
-    "add-package": { "name" : "my-pkg",
+    "add-runtimelib": { "name" : "testjar",
     "url":"http://localhost:8000/runtimelibs.jar" ,
-    "sha256" : "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420"}
+    "sha512" : "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420"}
     }'
 ----
 
@@ -166,7 +166,7 @@ Step  6 : Create a new request handler '/test' for the collection 'gettingstarte
 ----
 curl http://localhost:8983/solr/gettingstarted/config -H 'Content-type:application/json' -d '{
     "create-requesthandler": { "name" : "/test",
-    "class": "org.apache.solr.core.RuntimeLibReqHandler", "package" : "my-pkg" }
+    'class': 'org.apache.solr.core.RuntimeLibReqHandler', 'runtimeLib' : true}
     }'
 ----
 
@@ -198,15 +198,15 @@ output:
 Example:
 
 * Host the new jar to a new url. eg:  http://localhost:8000/runtimelibs_v2.jar
-* get the `sha256` hash of the new jar
+* get the `sha512` hash of the new jar
 * run the update-runtime lib command
 
 [source,bash]
 ----
  curl http://localhost:8983/solr/gettingstarted/config -H 'Content-type:application/json' -d '{
-    "update-package": { "name" : "my-pkg",
+    "update-runtimelib": { "name" : "testjar",
     "url":"http://localhost:8000/runtimelibs_v2.jar" ,
-    "sha256" : "<replace-the-new-sha256-digest-here>"}
+    "sha512" : "<replace-the-new-sha512-digest-here>"}
     }'
 ----
 NOTE: Always upload your jar to a new url as the Solr cluster is still referring to the old jar. If the existing jar is modified it can cause errors as the hash may not match
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrResponse.java b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrResponse.java
index 73eb863..ef52eb2 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrResponse.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrResponse.java
@@ -18,12 +18,10 @@ package org.apache.solr.client.solrj;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
-import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
 
-import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
@@ -34,7 +32,7 @@ import org.apache.solr.common.util.NamedList;
  * 
  * @since solr 1.3
  */
-public abstract class SolrResponse implements Serializable, MapWriter {
+public abstract class SolrResponse implements Serializable {
 
   /** Elapsed time in milliseconds for the request as seen from the client. */
   public abstract long getElapsedTime();
@@ -45,11 +43,6 @@ public abstract class SolrResponse implements Serializable, MapWriter {
   
   public abstract NamedList<Object> getResponse();
 
-  @Override
-  public void writeMap(EntryWriter ew) throws IOException {
-    getResponse().writeMap(ew);
-  }
-
   public Exception getException() {
     NamedList exp = (NamedList) getResponse().get("exception");
     if (exp == null) {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
index 43ba737..74d0bbc 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
@@ -207,23 +207,6 @@ public class CollectionApiMapping {
         POST,
         null,
         "set-obj-property", null),
-
-    ADD_PACKAGE(CLUSTER_CMD,
-        POST,null,
-        "add-package",null ),
-    UPDATE_PACKAGE(CLUSTER_CMD,
-        POST,null,
-        "update-package",null ),
-    DELETE_RUNTIME_LIB(CLUSTER_CMD,
-        POST,null,
-        "delete-package",null ),
-    ADD_REQ_HANDLER(CLUSTER_CMD,
-        POST,null,
-        "add-requesthandler",null ),
-    DELETE_REQ_HANDLER(CLUSTER_CMD,
-        POST,null,
-        "delete-requesthandler",null ),
-
     UTILIZE_NODE(CLUSTER_CMD,
         POST,
         UTILIZENODE,
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
index fa35e88..96e5371 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
@@ -89,18 +89,14 @@ public class ClusterProperties {
     return value;
   }
 
-  public Map<String, Object> getClusterProperties() throws IOException {
-    return getClusterProperties(new Stat());
-
-  }
   /**
    * Return the cluster properties
    * @throws IOException if there is an error reading properties from the cluster
    */
   @SuppressWarnings("unchecked")
-  public Map<String, Object> getClusterProperties(Stat stat) throws IOException {
+  public Map<String, Object> getClusterProperties() throws IOException {
     try {
-      Map<String, Object> properties = (Map<String, Object>) Utils.fromJSON(client.getData(ZkStateReader.CLUSTER_PROPS, null, stat, true));
+      Map<String, Object> properties = (Map<String, Object>) Utils.fromJSON(client.getData(ZkStateReader.CLUSTER_PROPS, null, new Stat(), true));
       return convertCollectionDefaultsToNestedFormat(properties);
     } catch (KeeperException.NoNodeException e) {
       return Collections.emptyMap();
@@ -109,12 +105,6 @@ public class ClusterProperties {
     }
   }
 
-  /**This applies the new map over the existing map. it's a merge operation, not an overwrite
-   * This applies the changes atomically over an existing object tree even if multiple nodes are
-   * trying to update this simultaneously
-   *
-   * @param properties The partial Object tree that needs to be applied
-   */
   public void setClusterProperties(Map<String, Object> properties) throws IOException, KeeperException, InterruptedException {
     client.atomicUpdate(ZkStateReader.CLUSTER_PROPS, zkData -> {
       if (zkData == null) return Utils.toJSON(convertCollectionDefaultsToNestedFormat(properties));
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 b9c66cf..dcf7d9e 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
@@ -194,7 +194,6 @@ public class ZkStateReader implements SolrCloseable {
   private final ConcurrentHashMap<String, PropsWatcher> collectionPropsWatchers = new ConcurrentHashMap<>();
 
   private volatile SortedSet<String> liveNodes = emptySortedSet();
-  private volatile int clusterPropsVersion = -1;
 
   private volatile Map<String, Object> clusterProperties = Collections.emptyMap();
 
@@ -494,20 +493,40 @@ public class ZkStateReader implements SolrCloseable {
     return collection.getZNodeVersion();
   }
 
-  private final Watcher clusterPropertiesWatcher = event -> {
-    // session events are not change events, and do not remove the watcher
-    if (Watcher.Event.EventType.None.equals(event.getType())) {
-      return;
+  public synchronized void createClusterStateWatchersAndUpdate() throws KeeperException,
+      InterruptedException {
+    // We need to fetch the current cluster state and the set of live nodes
+
+    log.debug("Updating cluster state from ZooKeeper... ");
+
+    // Sanity check ZK structure.
+    if (!zkClient.exists(CLUSTER_STATE, true)) {
+      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
+          "Cannot connect to cluster at " + zkClient.getZkServerAddress() + ": cluster not found/not ready");
     }
+
+    // on reconnect of SolrZkClient force refresh and re-add watches.
     loadClusterProperties();
-  };
+    refreshLiveNodes(new LiveNodeWatcher());
+    refreshLegacyClusterState(new LegacyClusterStateWatcher());
+    refreshStateFormat2Collections();
+    refreshCollectionList(new CollectionsChildWatcher());
+    refreshAliases(aliasesManager);
 
-  public void forceRefreshClusterProps(int expectedVersion) {
-    log.debug("Expected version of clusterprops.json is {} , my version is {}", expectedVersion, clusterPropsVersion);
-    if (expectedVersion > clusterPropsVersion) {
-      log.info("reloading clusterprops.json");
-      loadClusterProperties();
+    if (securityNodeListener != null) {
+      addSecurityNodeWatcher(pair -> {
+        ConfigData cd = new ConfigData();
+        cd.data = pair.first() == null || pair.first().length == 0 ? EMPTY_MAP : Utils.getDeepCopy((Map) fromJSON(pair.first()), 4, false);
+        cd.version = pair.second() == null ? -1 : pair.second().getVersion();
+        securityData = cd;
+        securityNodeListener.run();
+      });
+      securityData = getSecurityProps(true);
     }
+
+    collectionPropsObservers.forEach((k, v) -> {
+      collectionPropsWatchers.computeIfAbsent(k, PropsWatcher::new).refreshAndWatch(true);
+    });
   }
 
   private void addSecurityNodeWatcher(final Callable<Pair<byte[], Stat>> callback)
@@ -1083,52 +1102,22 @@ public class ZkStateReader implements SolrCloseable {
     return Collections.unmodifiableMap(clusterProperties);
   }
 
-  public synchronized void createClusterStateWatchersAndUpdate() throws KeeperException,
-      InterruptedException {
-    // We need to fetch the current cluster state and the set of live nodes
-
-    log.debug("Updating cluster state from ZooKeeper... ");
-
-    // Sanity check ZK structure.
-    if (!zkClient.exists(CLUSTER_STATE, true)) {
-      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
-          "Cannot connect to cluster at " + zkClient.getZkServerAddress() + ": cluster not found/not ready");
+  private final Watcher clusterPropertiesWatcher = event -> {
+    // session events are not change events, and do not remove the watcher
+    if (Watcher.Event.EventType.None.equals(event.getType())) {
+      return;
     }
-
-    // on reconnect of SolrZkClient force refresh and re-add watches.
     loadClusterProperties();
-    refreshLiveNodes(new LiveNodeWatcher());
-    refreshLegacyClusterState(new LegacyClusterStateWatcher());
-    refreshStateFormat2Collections();
-    refreshCollectionList(new CollectionsChildWatcher());
-    refreshAliases(aliasesManager);
-
-    if (securityNodeListener != null) {
-      addSecurityNodeWatcher(pair -> {
-        ConfigData cd = new ConfigData();
-        cd.data = pair.first() == null || pair.first().length == 0 ? EMPTY_MAP : Utils.getDeepCopy((Map) fromJSON(pair.first()), 4, false);
-        cd.version = pair.second() == null ? -1 : pair.second().getVersion();
-        securityData = cd;
-        securityNodeListener.run();
-      });
-      securityData = getSecurityProps(true);
-    }
-
-    collectionPropsObservers.forEach((k, v) -> {
-      collectionPropsWatchers.computeIfAbsent(k, PropsWatcher::new).refreshAndWatch(true);
-    });
-  }
+  };
 
   @SuppressWarnings("unchecked")
   private void loadClusterProperties() {
     try {
       while (true) {
         try {
-          Stat stat = new Stat();
-          byte[] data = zkClient.getData(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, stat, true);
+          byte[] data = zkClient.getData(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, new Stat(), true);
           this.clusterProperties = ClusterProperties.convertCollectionDefaultsToNestedFormat((Map<String, Object>) Utils.fromJSON(data));
-          this.clusterPropsVersion = stat.getVersion();
-          log.debug("Loaded cluster properties: {} to version {}", this.clusterProperties, clusterPropsVersion);
+          log.debug("Loaded cluster properties: {}", this.clusterProperties);
 
           for (ClusterPropertiesListener listener : clusterPropertiesListeners) {
             listener.onChange(getClusterProperties());
@@ -1136,7 +1125,6 @@ public class ZkStateReader implements SolrCloseable {
           return;
         } catch (KeeperException.NoNodeException e) {
           this.clusterProperties = Collections.emptyMap();
-          this.clusterPropsVersion = -1;
           log.debug("Loaded empty cluster properties");
           // set an exists watch, and if the node has been created since the last call,
           // read the data again
@@ -1149,10 +1137,6 @@ public class ZkStateReader implements SolrCloseable {
     }
   }
 
-  public int getClusterPropsVersion() {
-    return clusterPropsVersion;
-  }
-
   /**
    * Get collection properties for a given collection. If the collection is watched, simply return it from the cache,
    * otherwise fetch it directly from zookeeper. This is a convenience for {@code getCollectionProperties(collection,0)}
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
index d0c85a8..d8f1410 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
@@ -293,7 +293,5 @@ public interface CommonParams {
 
   String JAVABIN_MIME = "application/javabin";
 
-  String PACKAGE = "package";
-
 }
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java b/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
index 3804f78..277324a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
@@ -28,7 +28,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.noggit.JSONParser;
 import org.noggit.ObjectBuilder;
@@ -39,7 +38,7 @@ import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.util.StrUtils.formatString;
 import static org.apache.solr.common.util.Utils.toJSON;
 
-public class CommandOperation implements MapWriter {
+public class CommandOperation {
   public final String name;
   private Object commandData;//this is most often a map
   private List<String> errors = new ArrayList<>();
@@ -387,10 +386,4 @@ public class CommandOperation implements MapWriter {
     if (o == null) return null;
     return getInt(name, null);
   }
-
-  @Override
-  public void writeMap(EntryWriter ew) throws IOException {
-    ew.put(name, commandData);
-    ew.putIfNotNull("errors", errors);
-  }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java b/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
index e5bad27..a053a18 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
@@ -72,7 +72,6 @@ public class ExecutorUtil {
   }
 
   public static void shutdownAndAwaitTermination(ExecutorService pool) {
-    if(pool == null) return;
     pool.shutdown(); // Disable new tasks from being submitted
     awaitTermination(pool);
   }
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java b/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
index 9a68c3b..c0b19f5 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
@@ -40,15 +40,6 @@ public class StrUtils {
 
   }
 
-  static final String DELIM_CHARS = "/:;.,%#";
-  public static List<String> split(String s, char sep){
-    if(DELIM_CHARS.indexOf(s.charAt(0)) >-1){
-     sep = s.charAt(0);
-    }
-    return splitSmart(s,sep, true);
-
-  }
-
   public static List<String> splitSmart(String s, char separator, boolean trimEmpty) {
     List<String> l = splitSmart(s, separator);
     if(trimEmpty){
@@ -157,7 +148,7 @@ public class StrUtils {
    */
   public static List<String> splitFileNames(String fileNames) {
     if (fileNames == null)
-      return Collections.emptyList();
+      return Collections.<String>emptyList();
 
     List<String> result = new ArrayList<>();
     for (String file : fileNames.split("(?<!\\\\),")) {
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index 4eb3a3c..4a8d987 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -91,7 +91,7 @@ public class Utils {
   public static final Function NEW_SYNCHRONIZED_ARRAYLIST_FUN = o -> Collections.synchronizedList(new ArrayList<>());
   public static final Function NEW_HASHSET_FUN = o -> new HashSet<>();
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
+  
   public static Map getDeepCopy(Map map, int maxDepth) {
     return getDeepCopy(map, maxDepth, true, false);
   }
@@ -100,18 +100,21 @@ public class Utils {
     return getDeepCopy(map, maxDepth, mutable, false);
   }
 
-  public static final Function<JSONParser, ObjectBuilder> MAPWRITEROBJBUILDER = jsonParser -> {
-    try {
-      return new ObjectBuilder(jsonParser) {
-        @Override
-        public Object newObject() {
-          return new LinkedHashMapWriter();
-        }
-      };
-    } catch (IOException e) {
-      throw new RuntimeException(e);
+  public static Map getDeepCopy(Map map, int maxDepth, boolean mutable, boolean sorted) {
+    if(map == null) return null;
+    if (maxDepth < 1) return map;
+    Map copy;
+    if (sorted) {
+      copy = new TreeMap();
+    } else {
+      copy = map instanceof LinkedHashMap?  new LinkedHashMap(map.size()): new HashMap(map.size());
     }
-  };
+    for (Object o : map.entrySet()) {
+      Map.Entry e = (Map.Entry) o;
+      copy.put(e.getKey(), makeDeepCopy(e.getValue(),maxDepth, mutable, sorted));
+    }
+    return mutable ? copy : Collections.unmodifiableMap(copy);
+  }
 
   public static void forEachMapEntry(Object o, String path, BiConsumer fun) {
     Object val = Utils.getObjectByPath(o, false, path);
@@ -141,40 +144,6 @@ public class Utils {
       ((Map) o).forEach((k, v) -> fun.accept(k, v));
     }
   }
-  public static final Function<JSONParser, ObjectBuilder> MAPOBJBUILDER = jsonParser -> {
-    try {
-      return new ObjectBuilder(jsonParser) {
-        @Override
-        public Object newObject() {
-          return new HashMap();
-        }
-      };
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  };
-  public static final Pattern ARRAY_ELEMENT_INDEX = Pattern
-      .compile("(\\S*?)\\[([-]?\\d+)\\]");
-
-  public static Map getDeepCopy(Map map, int maxDepth, boolean mutable, boolean sorted) {
-    if (map == null) return null;
-    if (maxDepth < 1) return map;
-    Map copy;
-    if (sorted) {
-      copy = new TreeMap();
-    } else {
-      copy = map instanceof LinkedHashMap ? new LinkedHashMap(map.size()) : new HashMap(map.size());
-    }
-    for (Object o : map.entrySet()) {
-      Map.Entry e = (Map.Entry) o;
-      copy.put(e.getKey(), makeDeepCopy(e.getValue(), maxDepth, mutable, sorted));
-    }
-    return mutable ? copy : Collections.unmodifiableMap(copy);
-  }
-
-  public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable) {
-    return getDeepCopy(c, maxDepth, mutable, false);
-  }
 
   private static Object makeDeepCopy(Object v, int maxDepth, boolean mutable, boolean sorted) {
     if (v instanceof MapWriter && maxDepth > 1) {
@@ -182,7 +151,7 @@ public class Utils {
     } else if (v instanceof IteratorWriter && maxDepth > 1) {
       v = ((IteratorWriter) v).toList(new ArrayList<>());
       if (sorted) {
-        Collections.sort((List) v);
+        Collections.sort((List)v);
       }
     }
 
@@ -194,6 +163,29 @@ public class Utils {
     return v;
   }
 
+  public static InputStream toJavabin(Object o) throws IOException {
+    try (final JavaBinCodec jbc = new JavaBinCodec()) {
+      BinaryRequestWriter.BAOS baos = new BinaryRequestWriter.BAOS();
+      jbc.marshal(o,baos);
+      return new ByteBufferInputStream(ByteBuffer.wrap(baos.getbuf(),0,baos.size()));
+    }
+  }
+
+  public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable) {
+    return getDeepCopy(c, maxDepth, mutable, false);
+  }
+
+  public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable, boolean sorted) {
+    if (c == null || maxDepth < 1) return c;
+    Collection result = c instanceof Set ?
+        ( sorted? new TreeSet() : new HashSet()) : new ArrayList();
+    for (Object o : c) result.add(makeDeepCopy(o, maxDepth, mutable, sorted));
+    if (sorted && (result instanceof List)) {
+      Collections.sort((List)result);
+    }
+    return mutable ? result : result instanceof Set ? unmodifiableSet((Set) result) : unmodifiableList((List) result);
+  }
+
   public static void writeJson(Object o, OutputStream os, boolean indent) throws IOException {
     writeJson(o, new OutputStreamWriter(os, UTF_8), indent)
         .flush();
@@ -207,14 +199,37 @@ public class Utils {
     return writer;
   }
 
-  public static InputStream toJavabin(Object o) throws IOException {
-    try (final JavaBinCodec jbc = new JavaBinCodec()) {
-      BinaryRequestWriter.BAOS baos = new BinaryRequestWriter.BAOS();
-      jbc.marshal(o, baos);
-      return new ByteBufferInputStream(ByteBuffer.wrap(baos.getbuf(), 0, baos.size()));
+  private static class MapWriterJSONWriter extends JSONWriter {
+
+    public MapWriterJSONWriter(CharArr out, int indentSize) {
+      super(out, indentSize);
+    }
+
+    @Override
+    public void handleUnknownClass(Object o) {
+      if (o instanceof MapWriter) {
+        Map m = ((MapWriter)o).toMap(new LinkedHashMap<>());
+        write(m);
+      } else {
+        super.handleUnknownClass(o);
+      }
     }
   }
 
+  public static byte[] toJSON(Object o) {
+    if(o == null) return new byte[0];
+    CharArr out = new CharArr();
+    if (!(o instanceof List) && !(o instanceof Map)) {
+      if (o instanceof MapWriter)  {
+        o = ((MapWriter)o).toMap(new LinkedHashMap<>());
+      } else if(o instanceof IteratorWriter){
+        o = ((IteratorWriter)o).toList(new ArrayList<>());
+      }
+    }
+    new MapWriterJSONWriter(out, 2).write(o); // indentation by default
+    return toUTF8(out);
+  }
+
   public static String toJSONString(Object o) {
     return new String(toJSON(o), StandardCharsets.UTF_8);
   }
@@ -259,29 +274,15 @@ public class Utils {
     return propMap;
   }
 
-  public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable, boolean sorted) {
-    if (c == null || maxDepth < 1) return c;
-    Collection result = c instanceof Set ?
-        (sorted ? new TreeSet() : new HashSet()) : new ArrayList();
-    for (Object o : c) result.add(makeDeepCopy(o, maxDepth, mutable, sorted));
-    if (sorted && (result instanceof List)) {
-      Collections.sort((List) result);
-    }
-    return mutable ? result : result instanceof Set ? unmodifiableSet((Set) result) : unmodifiableList((List) result);
+  public static Object fromJSON(InputStream is){
+    return fromJSON(new InputStreamReader(is, UTF_8));
   }
-
-  public static byte[] toJSON(Object o) {
-    if (o == null) return new byte[0];
-    CharArr out = new CharArr();
-    if (!(o instanceof List) && !(o instanceof Map)) {
-      if (o instanceof MapWriter) {
-        o = ((MapWriter) o).toMap(new LinkedHashMap<>());
-      } else if (o instanceof IteratorWriter) {
-        o = ((IteratorWriter) o).toList(new ArrayList<>());
-      }
+  public static Object fromJSON(Reader is){
+    try {
+      return STANDARDOBJBUILDER.apply(getJSONParser(is)).getVal();
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error", e);
     }
-    new MapWriterJSONWriter(out, 2).write(o); // indentation by default
-    return toUTF8(out);
   }
 
 
@@ -292,14 +293,35 @@ public class Utils {
       throw new RuntimeException(e);
     }
   };
+  public static final Function<JSONParser, ObjectBuilder> MAPWRITEROBJBUILDER = jsonParser -> {
+    try {
+      return new ObjectBuilder(jsonParser){
+        @Override
+        public Object newObject() {
+          return new LinkedHashMapWriter();
+        }
+      };
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  };
 
-  public static Object fromJSON(InputStream is) {
-    return fromJSON(new InputStreamReader(is, UTF_8));
-  }
+  public static final Function<JSONParser, ObjectBuilder> MAPOBJBUILDER = jsonParser -> {
+    try {
+      return new ObjectBuilder(jsonParser){
+        @Override
+        public Object newObject() {
+          return new HashMap();
+        }
+      };
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  };
 
-  public static Object fromJSON(Reader is) {
+  public static Object fromJSON(InputStream is, Function<JSONParser, ObjectBuilder> objBuilderProvider) {
     try {
-      return STANDARDOBJBUILDER.apply(getJSONParser(is)).getVal();
+      return objBuilderProvider.apply(getJSONParser((new InputStreamReader(is, StandardCharsets.UTF_8)))).getVal();
     } catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error", e);
     }
@@ -314,19 +336,10 @@ public class Utils {
       return fromJSON(stream);
     } catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-          "Resource error: " + e.getMessage(), e);
+                              "Resource error: " + e.getMessage(), e);
     }
   }
-
-  public static Object fromJSON(InputStream is, Function<JSONParser, ObjectBuilder> objBuilderProvider) {
-    try {
-      return objBuilderProvider.apply(getJSONParser((new InputStreamReader(is, StandardCharsets.UTF_8)))).getVal();
-    } catch (IOException e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error", e);
-    }
-  }
-
-  public static JSONParser getJSONParser(Reader reader) {
+  public static JSONParser getJSONParser(Reader reader){
     JSONParser parser = new JSONParser(reader);
     parser.setFlags(parser.getFlags() |
         JSONParser.ALLOW_MISSING_COLON_COMMA_BEFORE_OBJECT |
@@ -334,11 +347,11 @@ public class Utils {
     return parser;
   }
 
-  public static Object fromJSONString(String json) {
+  public static Object fromJSONString(String json)  {
     try {
       return STANDARDOBJBUILDER.apply(getJSONParser(new StringReader(json))).getVal();
     } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error : " + json, e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error : "+ json, e );
     }
   }
 
@@ -350,10 +363,10 @@ public class Utils {
 
   public static boolean setObjectByPath(Object root, String hierarchy, Object value) {
     List<String> parts = StrUtils.splitSmart(hierarchy, '/', true);
-    return setObjectByPath(root, parts, value, true);
+    return setObjectByPath(root, parts, value);
   }
 
-  public static boolean setObjectByPath(Object root, List<String> hierarchy, Object value, boolean insertMissing) {
+  public static boolean setObjectByPath(Object root, List<String> hierarchy, Object value) {
     if (root == null) return false;
     if (!isMapLike(root)) throw new RuntimeException("must be a Map or NamedList");
     Object obj = root;
@@ -369,10 +382,7 @@ public class Utils {
       }
       if (i < hierarchy.size() - 1) {
         Object o = getVal(obj, s, -1);
-        if (o == null) {
-          if (insertMissing) insertItem(o = new LinkedHashMap<>(), obj, s);
-          else return false;
-        }
+        if (o == null) return false;
         if (idx > -1) {
           List l = (List) o;
           o = idx < l.size() ? l.get(idx) : null;
@@ -381,7 +391,14 @@ public class Utils {
         obj = o;
       } else {
         if (idx == -2) {
-          insertItem(value, obj, s);
+          if (obj instanceof NamedList) {
+            NamedList namedList = (NamedList) obj;
+            int location = namedList.indexOf(s, 0);
+            if (location == -1) namedList.add(s, value);
+            else namedList.setVal(location, value);
+          } else if (obj instanceof Map) {
+            ((Map) obj).put(s, value);
+          }
           return true;
         } else {
           Object v = getVal(obj, s, -1);
@@ -405,20 +422,10 @@ public class Utils {
 
   }
 
-  private static void insertItem(Object value, Object container, String name) {
-    if (container instanceof NamedList) {
-      NamedList namedList = (NamedList) container;
-      int location = namedList.indexOf(name, 0);
-      if (location == -1) namedList.add(name, value);
-      else namedList.setVal(location, value);
-    } else if (container instanceof Map) {
-      ((Map) container).put(name, value);
-    }
-  }
 
   public static Object getObjectByPath(Object root, boolean onlyPrimitive, List<String> hierarchy) {
-    if (root == null) return null;
-    if (!isMapLike(root)) return null;
+    if(root == null) return null;
+    if(!isMapLike(root)) return null;
     Object obj = root;
     for (int i = 0; i < hierarchy.size(); i++) {
       int idx = -1;
@@ -511,7 +518,6 @@ public class Utils {
       try {
         ((MapWriter) obj).writeMap(new MapWriter.EntryWriter() {
           int count = -1;
-
           @Override
           public MapWriter.EntryWriter put(CharSequence k, Object v) {
             if (result[0] != null) return this;
@@ -527,14 +533,15 @@ public class Utils {
         throw new RuntimeException(e);
       }
       return result[0];
-    } else if (obj instanceof Map) return ((Map) obj).get(key);
+    }
+    else if (obj instanceof Map) return ((Map) obj).get(key);
     else throw new RuntimeException("must be a NamedList or Map");
   }
 
   /**
    * If the passed entity has content, make sure it is fully
    * read and closed.
-   *
+   * 
    * @param entity to consume or null
    */
   public static void consumeFully(HttpEntity entity) {
@@ -555,21 +562,31 @@ public class Utils {
 
   /**
    * Make sure the InputStream is fully read.
-   *
+   * 
    * @param is to read
    * @throws IOException on problem with IO
    */
   private static void readFully(InputStream is) throws IOException {
     is.skip(is.available());
-    while (is.read() != -1) {
+    while (is.read() != -1) {}
+  }
+
+  public static Map<String, Object> getJson(DistribStateManager distribStateManager, String path) throws InterruptedException, IOException, KeeperException {
+    VersionedData data = null;
+    try {
+      data = distribStateManager.getData(path);
+    } catch (KeeperException.NoNodeException | NoSuchElementException e) {
+      return Collections.emptyMap();
     }
+    if (data == null || data.getData() == null || data.getData().length == 0) return Collections.emptyMap();
+    return (Map<String, Object>) Utils.fromJSON(data.getData());
   }
 
   /**
    * Assumes data in ZooKeeper is a JSON string, deserializes it and returns as a Map
    *
-   * @param zkClient        the zookeeper client
-   * @param path            the path to the znode being read
+   * @param zkClient the zookeeper client
+   * @param path the path to the znode being read
    * @param retryOnConnLoss whether to retry the operation automatically on connection loss, see {@link org.apache.solr.common.cloud.ZkCmdExecutor#retryOperation(ZkOperation)}
    * @return a Map if the node exists and contains valid JSON or an empty map if znode does not exist or has a null data
    */
@@ -585,23 +602,39 @@ public class Utils {
     return Collections.emptyMap();
   }
 
-  public static Map<String, Object> getJson(DistribStateManager distribStateManager, String path) throws InterruptedException, IOException, KeeperException {
-    VersionedData data = null;
-    try {
-      data = distribStateManager.getData(path);
-    } catch (KeeperException.NoNodeException | NoSuchElementException e) {
-      return Collections.emptyMap();
+  public static final Pattern ARRAY_ELEMENT_INDEX = Pattern
+      .compile("(\\S*?)\\[([-]?\\d+)\\]");
+
+  public static SpecProvider getSpec(final String name) {
+    return () -> {
+      return ValidatingJsonMap.parse(CommonParams.APISPEC_LOCATION + name + ".json", CommonParams.APISPEC_LOCATION);
+    };
+  }
+
+  public static String parseMetricsReplicaName(String collectionName, String coreName) {
+    if (collectionName == null || !coreName.startsWith(collectionName)) {
+      return null;
+    } else {
+      // split "collection1_shard1_1_replica1" into parts
+      if (coreName.length() > collectionName.length()) {
+        String str = coreName.substring(collectionName.length() + 1);
+        int pos = str.lastIndexOf("_replica");
+        if (pos == -1) { // ?? no _replicaN part ??
+          return str;
+        } else {
+          return str.substring(pos + 1);
+        }
+      } else {
+        return null;
+      }
     }
-    if (data == null || data.getData() == null || data.getData().length == 0) return Collections.emptyMap();
-    return (Map<String, Object>) Utils.fromJSON(data.getData());
   }
 
-  /**
-   * Applies one json over other. The 'input' is applied over the sink
-   * The values in input are applied over the values in 'sink' . If a value is 'null'
+  /**Applies one json over other. The 'input' is applied over the sink
+   * The values in input isapplied over the values in 'sink' . If a value is 'null'
    * that value is removed from sink
    *
-   * @param sink  the original json object to start with. Ensure that this Map is mutable
+   * @param sink the original json object to start with. Ensure that this Map is mutable
    * @param input the json with new values
    * @return whether there was any change made to sink or not.
    */
@@ -639,62 +672,20 @@ public class Utils {
     return isModified;
   }
 
-  public static SpecProvider getSpec(final String name) {
-    return () -> {
-      return ValidatingJsonMap.parse(CommonParams.APISPEC_LOCATION + name + ".json", CommonParams.APISPEC_LOCATION);
-    };
-  }
-
-  public static String parseMetricsReplicaName(String collectionName, String coreName) {
-    if (collectionName == null || !coreName.startsWith(collectionName)) {
-      return null;
-    } else {
-      // split "collection1_shard1_1_replica1" into parts
-      if (coreName.length() > collectionName.length()) {
-        String str = coreName.substring(collectionName.length() + 1);
-        int pos = str.lastIndexOf("_replica");
-        if (pos == -1) { // ?? no _replicaN part ??
-          return str;
-        } else {
-          return str.substring(pos + 1);
-        }
-      } else {
-        return null;
-      }
-    }
-  }
-
   public static String getBaseUrlForNodeName(final String nodeName, String urlScheme) {
     final int _offset = nodeName.indexOf("_");
     if (_offset < 0) {
       throw new IllegalArgumentException("nodeName does not contain expected '_' separator: " + nodeName);
     }
-    final String hostAndPort = nodeName.substring(0, _offset);
+    final String hostAndPort = nodeName.substring(0,_offset);
     try {
-      final String path = URLDecoder.decode(nodeName.substring(1 + _offset), "UTF-8");
+      final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
       return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
     } catch (UnsupportedEncodingException e) {
       throw new IllegalStateException("JVM Does not seem to support UTF-8", e);
     }
   }
 
-  private static class MapWriterJSONWriter extends JSONWriter {
-
-    public MapWriterJSONWriter(CharArr out, int indentSize) {
-      super(out, indentSize);
-    }
-
-    @Override
-    public void handleUnknownClass(Object o) {
-      if (o instanceof MapWriter) {
-        Map m = ((MapWriter) o).toMap(new LinkedHashMap<>());
-        write(m);
-      } else {
-        super.handleUnknownClass(o);
-      }
-    }
-  }
-
   public static long time(TimeSource timeSource, TimeUnit unit) {
     return unit.convert(timeSource.getTimeNs(), TimeUnit.NANOSECONDS);
   }
diff --git a/solr/solrj/src/resources/apispec/cluster.Commands.json b/solr/solrj/src/resources/apispec/cluster.Commands.json
index af7c19d..069cd1d 100644
--- a/solr/solrj/src/resources/apispec/cluster.Commands.json
+++ b/solr/solrj/src/resources/apispec/cluster.Commands.json
@@ -169,47 +169,6 @@
       "required": [
         "name"
       ]
-    },
-    "add-package": {
-      "documentation": "",
-      "description" : "Add a package to the classpath",
-      "#include": "cluster.Commands.runtimelib.properties"
-    },
-    "update-package": {
-      "documentation": "",
-      "description" : "Update the jar details",
-      "#include": "cluster.Commands.runtimelib.properties"
-    },
-    "delete-package": {
-      "documentation": "",
-      "description" : "delete a lib",
-      "type": "string"
-    },
-    "add-requesthandler": {
-      "type": "object",
-      "documentation": "",
-      "description" : "Create a node level request handler",
-      "properties": {
-        "name": {
-          "type": "string",
-          "description": "Name of the request handler. This is the path"
-        },
-        "class": {
-          "type": "string",
-          "description": "The class name"
-        },
-        "package" : {
-          "type": "string",
-          "description": " The package from where the plugin can be loaded from"
-        }
-      },
-      "required": ["name", "class"],
-      "additionalProperties": true
-    },
-    "delete-requesthandler" : {
-      "description" : "delete a requesthandler",
-      "type": "string"
     }
-
   }
 }
diff --git a/solr/solrj/src/resources/apispec/cluster.Commands.runtimelib.properties.json b/solr/solrj/src/resources/apispec/cluster.Commands.runtimelib.properties.json
deleted file mode 100644
index ab334b5..0000000
--- a/solr/solrj/src/resources/apispec/cluster.Commands.runtimelib.properties.json
+++ /dev/null
@@ -1,23 +0,0 @@
-{
-  "type": "object",
-  "properties": {
-    "name": {
-      "type": "string",
-      "description": "A name for the library"
-    },
-    "url": {
-      "type": "string",
-      "description": "The remote url"
-    },
-    "sha256": {
-      "type": "string",
-      "description": "The sha256 hash of the jar"
-    },
-    "sig": {
-      "type": "string",
-      "description": "the signature of the jar"
-    }
-  },
-  "required" : ["name","url","sha256"]
-
-}
\ No newline at end of file
diff --git a/solr/solrj/src/resources/apispec/core.config.Commands.addRequestHandler.properties.json b/solr/solrj/src/resources/apispec/core.config.Commands.addRequestHandler.properties.json
index 6ee1498..731c3d8 100644
--- a/solr/solrj/src/resources/apispec/core.config.Commands.addRequestHandler.properties.json
+++ b/solr/solrj/src/resources/apispec/core.config.Commands.addRequestHandler.properties.json
@@ -10,7 +10,7 @@
       "description": "The request handler class. Class names do not need to be fully qualified if they are included with Solr, so you can abbreviate the name as 'solr.SearchHandler'. Custom or third-party class names may need to be fully qualified, however."
     },
     "runtimeLib": {
-      "type": "string",
+      "type": "boolean",
       "description": "An optional parameter to use a custom .jar file that has been uploaded to Solr's blobstore. This additionally requires that the .jar has also been registered with the 'add-runtimelib' command, which is one of the available commands for the Config API."
     },
     "startup": {
diff --git a/solr/solrj/src/resources/apispec/core.config.Commands.generic.json b/solr/solrj/src/resources/apispec/core.config.Commands.generic.json
index 2ebfdf8..9d2b01d 100644
--- a/solr/solrj/src/resources/apispec/core.config.Commands.generic.json
+++ b/solr/solrj/src/resources/apispec/core.config.Commands.generic.json
@@ -10,7 +10,7 @@
       "description": "The configuration item class. Class names do not need to be fully qualified if they are included with Solr, so you can abbreviate the name as 'solr.SearchHandler'. Custom or third-party class names may need to be fully qualified, however."
    },
     "runtimeLib": {
-      "type": "string",
+      "type": "boolean",
       "description": "An optional parameter to use a custom .jar file that has been uploaded to Solr's blobstore. This additionally requires that the .jar has also been registered with the 'add-runtimelib' command, which is one of the available commands for the Config API."
    }
   },
diff --git a/solr/solrj/src/resources/apispec/core.config.json b/solr/solrj/src/resources/apispec/core.config.json
index 2324821..81e7d54 100644
--- a/solr/solrj/src/resources/apispec/core.config.json
+++ b/solr/solrj/src/resources/apispec/core.config.json
@@ -12,8 +12,7 @@
       "/config/jmx",
       "/config/requestDispatcher",
       "/config/znodeVersion",
-      "/config/{plugin}",
-      "/config/{plugin}/{pluginName}"
+      "/config/{plugin}"
     ]
   }
 }
diff --git a/solr/solrj/src/resources/apispec/node.blob.GET.json b/solr/solrj/src/resources/apispec/node.blob.GET.json
deleted file mode 100644
index 273333e..0000000
--- a/solr/solrj/src/resources/apispec/node.blob.GET.json
+++ /dev/null
@@ -1,11 +0,0 @@
-{
-  "methods": [
-    "GET"
-  ],
-  "url": {
-    "paths": [
-      "/node/blob",
-      "/node/blob/{sha256}"
-    ]
-  }
-}
diff --git a/solr/solrj/src/resources/apispec/node.ext.json b/solr/solrj/src/resources/apispec/node.ext.json
deleted file mode 100644
index 161b2aa..0000000
--- a/solr/solrj/src/resources/apispec/node.ext.json
+++ /dev/null
@@ -1,13 +0,0 @@
-{
-  "methods": [
-    "POST",
-    "GET",
-    "DELETE"
-  ],
-  "url": {
-    "paths": [
-      "/node/ext/{handlerName}",
-      "/node/ext"
-    ]
-  }
-}
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
index 2ef2659..d9de129 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.cloud;
 
+import static org.apache.solr.common.util.Utils.makeMap;
+
 import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
@@ -106,8 +108,6 @@ import org.noggit.JSONWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.util.Utils.makeMap;
-
 /**
  * TODO: we should still test this works as a custom update chain as well as
  * what we test now - the default update chain
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
index 23283cc..cb66ae9 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
@@ -55,14 +55,11 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.zookeeper.CreateMode;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.cloud.ZkConfigManager.CONFIGS_ZKNODE;
-
 /**
  * Base class for SolrCloud tests
  * <p>
@@ -90,12 +87,9 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
   private static class Config {
     final String name;
     final Path path;
-    final Map<String, byte[]> extraConfig;
-
-    private Config(String name, Path path, Map<String, byte[]> extraConfig) {
+    private Config(String name, Path path) {
       this.name = name;
       this.path = path;
-      this.extraConfig = extraConfig;
     }
   }
 
@@ -187,12 +181,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
      * @param configPath the path to the config files
      */
     public Builder addConfig(String configName, Path configPath) {
-      this.configs.add(new Config(configName, configPath, null));
-      return this;
-    }
-
-    public Builder addConfig(String configName, Path configPath, Map<String, byte[]> extraConfig) {
-      this.configs.add(new Config(configName, configPath, extraConfig));
+      this.configs.add(new Config(configName, configPath));
       return this;
     }
 
@@ -217,8 +206,8 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
      *
      * @throws Exception if an error occurs on startup
      */
-    public MiniSolrCloudCluster configure() throws Exception {
-      return cluster = build();
+    public void configure() throws Exception {
+      cluster = build();
     }
 
     /**
@@ -232,15 +221,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
           null, securityJson, trackJettyMetrics);
       CloudSolrClient client = cluster.getSolrClient();
       for (Config config : configs) {
-        ((ZkClientClusterStateProvider) client.getClusterStateProvider()).uploadConfig(config.path, config.name);
-        if (config.extraConfig != null) {
-          for (Map.Entry<String, byte[]> e : config.extraConfig.entrySet()) {
-            ((ZkClientClusterStateProvider) client.getClusterStateProvider()).getZkStateReader().getZkClient()
-                .create(CONFIGS_ZKNODE + "/" + config.name + "/" + e.getKey(), e.getValue(), CreateMode.PERSISTENT, true);
-
-          }
-
-        }
+        ((ZkClientClusterStateProvider)client.getClusterStateProvider()).uploadConfig(config.path, config.name);
       }
 
       if (clusterProperties.size() > 0) {