You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2019/08/20 01:57:24 UTC

[lucene-solr] branch branch_8x updated (ee50c9d -> 625e3f3)

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

noble pushed a change to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from ee50c9d  Harden TestExtendedDismaxParser.testBf
     new b0a11ba  SOLR-13650: Support for named global classloaders
     new 129231d  SOLR-13650: compile error fixed
     new ecfd0c1  SOLR-13650: cache reload listener
     new 6d3d3c7  SOLR-13650: precommit errors fixed
     new 625e3f3  SOLR-13650: CHANGES.txt

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/lucene/document/LongPoint.java |   2 +-
 .../apache/lucene/queryparser/xml/CoreParser.java  |  35 +++-
 .../lucene/queryparser/xml/TestCoreParser.java     |   6 +-
 solr/CHANGES.txt                                   |   3 +
 .../solr/cloud/autoscaling/IndexSizeTrigger.java   |   2 +-
 .../java/org/apache/solr/core/CoreContainer.java   |   4 +-
 .../java/org/apache/solr/core/MemClassLoader.java  |   8 +
 .../core/{LibListener.java => PackageManager.java} | 214 +++++++++++++++------
 .../src/java/org/apache/solr/core/PluginBag.java   |  58 ++----
 .../src/java/org/apache/solr/core/PluginInfo.java  |  67 ++++---
 .../src/java/org/apache/solr/core/RuntimeLib.java  |   7 +
 .../src/java/org/apache/solr/core/SolrCore.java    |  22 ++-
 .../org/apache/solr/handler/SolrConfigHandler.java |  14 +-
 .../solr/handler/admin/CollectionHandlerApi.java   |  34 ++--
 .../solr/handler/admin/CollectionsHandler.java     |   2 +-
 .../handler/component/QueryElevationComponent.java |   2 +-
 .../solr/rest/schema/FieldTypeXmlAdapter.java      |   5 +-
 .../java/org/apache/solr/search/CacheConfig.java   |  63 ++++--
 .../org/apache/solr/search/SolrCacheHolder.java    |  21 +-
 .../runtimecode/{TestURP.java => MyDocCache.java}  |  21 +-
 solr/core/src/test-files/runtimecode/cache.jar.bin | Bin 0 -> 820 bytes
 .../src/test-files/runtimecode/cache_v2.jar.bin    | Bin 0 -> 819 bytes
 solr/core/src/test-files/runtimecode/sig.txt       |  10 +-
 .../solr/handler/TestContainerReqHandler.java      | 177 ++++++++++++-----
 .../client/solrj/request/CollectionApiMapping.java |  10 +-
 .../apache/solr/common/params/CommonParams.java    |   2 +
 .../src/resources/apispec/cluster.Commands.json    |  12 +-
 .../org/apache/solr/cloud/SolrCloudTestCase.java   |  22 ++-
 28 files changed, 559 insertions(+), 264 deletions(-)
 rename solr/core/src/java/org/apache/solr/core/{LibListener.java => PackageManager.java} (53%)
 copy solr/core/src/test-files/runtimecode/{TestURP.java => MyDocCache.java} (65%)
 create mode 100644 solr/core/src/test-files/runtimecode/cache.jar.bin
 create mode 100644 solr/core/src/test-files/runtimecode/cache_v2.jar.bin


[lucene-solr] 01/05: SOLR-13650: Support for named global classloaders

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit b0a11ba5c7a3ff97414891228e626c47927f7865
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Tue Aug 20 05:00:38 2019 +1000

    SOLR-13650: Support for named global classloaders
---
 .../java/org/apache/lucene/document/LongPoint.java |   2 +-
 .../apache/lucene/queryparser/xml/CoreParser.java  |  35 +++-
 .../lucene/queryparser/xml/TestCoreParser.java     |   6 +-
 .../solr/cloud/autoscaling/IndexSizeTrigger.java   |   2 +-
 .../java/org/apache/solr/core/CoreContainer.java   |   4 +-
 .../java/org/apache/solr/core/MemClassLoader.java  |   8 +
 .../core/{LibListener.java => PackageManager.java} | 214 +++++++++++++++------
 .../src/java/org/apache/solr/core/PluginBag.java   |  58 ++----
 .../src/java/org/apache/solr/core/PluginInfo.java  |  67 ++++---
 .../src/java/org/apache/solr/core/RuntimeLib.java  |   7 +
 .../src/java/org/apache/solr/core/SolrCore.java    |  22 ++-
 .../org/apache/solr/handler/SolrConfigHandler.java |  14 +-
 .../solr/handler/admin/CollectionHandlerApi.java   |  34 ++--
 .../solr/handler/admin/CollectionsHandler.java     |   2 +-
 .../handler/component/QueryElevationComponent.java |   2 +-
 .../solr/rest/schema/FieldTypeXmlAdapter.java      |   5 +-
 .../java/org/apache/solr/search/CacheConfig.java   |  63 ++++--
 .../src/test-files/runtimecode/MyDocCache.java     |  35 ++++
 solr/core/src/test-files/runtimecode/cache.jar.bin | Bin 0 -> 820 bytes
 .../src/test-files/runtimecode/cache_v2.jar.bin    | Bin 0 -> 819 bytes
 solr/core/src/test-files/runtimecode/sig.txt       |  10 +-
 .../solr/handler/TestContainerReqHandler.java      | 176 ++++++++++++-----
 .../client/solrj/request/CollectionApiMapping.java |  10 +-
 .../apache/solr/common/params/CommonParams.java    |   2 +
 .../src/resources/apispec/cluster.Commands.json    |  12 +-
 .../org/apache/solr/cloud/SolrCloudTestCase.java   |  22 ++-
 26 files changed, 560 insertions(+), 252 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index 28a6a59..e6baa32 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -20,12 +20,12 @@ import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.lucene.index.PointValues;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.NumericUtils;
 
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
index b470312..77680cc 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
@@ -16,9 +16,34 @@
  */
 package org.apache.lucene.queryparser.xml;
 
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.InputStream;
+import java.util.Locale;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.queryparser.classic.QueryParser;
-import org.apache.lucene.queryparser.xml.builders.*;
+import org.apache.lucene.queryparser.xml.builders.BooleanQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.BoostingTermBuilder;
+import org.apache.lucene.queryparser.xml.builders.ConstantScoreQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.DisjunctionMaxQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.MatchAllDocsQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.PointRangeQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.RangeQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanFirstBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanNearBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanNotBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanOrBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanOrTermsBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanPositionRangeBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanQueryBuilderFactory;
+import org.apache.lucene.queryparser.xml.builders.SpanTermBuilder;
+import org.apache.lucene.queryparser.xml.builders.TermQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.TermsQueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.UserInputQueryBuilder;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.w3c.dom.Document;
@@ -27,14 +52,6 @@ import org.xml.sax.EntityResolver;
 import org.xml.sax.ErrorHandler;
 import org.xml.sax.SAXException;
 
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
-import java.io.InputStream;
-import java.util.Locale;
-
 /**
  * Assembles a QueryBuilder which uses only core Lucene Query objects
  */
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
index 4faf6e8..cb45dc9 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
@@ -16,6 +16,9 @@
  */
 package org.apache.lucene.queryparser.xml;
 
+import java.io.IOException;
+import java.io.InputStream;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenFilter;
@@ -32,9 +35,6 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.junit.AfterClass;
 import org.xml.sax.SAXException;
 
-import java.io.IOException;
-import java.io.InputStream;
-
 public class TestCoreParser extends LuceneTestCase {
 
   final private static String defaultField = "contents";
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 f32669c..6023f43 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/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 571c8d0..9a3d5b4 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -219,7 +219,7 @@ public class CoreContainer {
 
   protected volatile AutoscalingHistoryHandler autoscalingHistoryHandler;
 
-  private final LibListener clusterPropertiesListener = new LibListener(this);
+  private final PackageManager clusterPropertiesListener = new PackageManager(this);
 
 
   // Bits for the state variable.
@@ -1802,7 +1802,7 @@ public class CoreContainer {
     return containerHandlers;
   }
 
-  public LibListener getClusterPropertiesListener(){
+  public PackageManager getPackageManager(){
     return clusterPropertiesListener;
   }
 
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 c1267c2..56ff087 100644
--- a/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
@@ -54,6 +54,14 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
     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;
diff --git a/solr/core/src/java/org/apache/solr/core/LibListener.java b/solr/core/src/java/org/apache/solr/core/PackageManager.java
similarity index 53%
rename from solr/core/src/java/org/apache/solr/core/LibListener.java
rename to solr/core/src/java/org/apache/solr/core/PackageManager.java
index 463ad21..ab963c0 100644
--- a/solr/core/src/java/org/apache/solr/core/LibListener.java
+++ b/solr/core/src/java/org/apache/solr/core/PackageManager.java
@@ -17,8 +17,8 @@
 
 package org.apache.solr.core;
 
+import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -27,11 +27,9 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.stream.Collectors;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.api.Api;
 import org.apache.solr.api.V2HttpCall;
-import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterPropertiesListener;
@@ -50,43 +48,83 @@ 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.SHA512;
 
-public class LibListener implements ClusterPropertiesListener {
+public class PackageManager implements ClusterPropertiesListener {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final CoreContainer coreContainer;
 
-  Map<String, RuntimeLib> runtimeLibs = new HashMap<>();
-  MemClassLoader memClassLoader;
+  private Map<String, Package> pkgs = new HashMap<>();
+
   final ExtHandler extHandler;
   private int myversion = -1;
 
-  LibListener(CoreContainer coreContainer) {
+  public int getZNodeVersion(String pkg) {
+    Package p = pkgs.get(pkg);
+    return p == null ? -1 : p.lib.getZnodeVersion();
+  }
+
+  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) {
+      if ((Objects.equals(lib.getSha512(), (map).get(SHA512)) &&
+          Objects.equals(lib.getSig(), (map).get(SHA512)))) {
+        return false;
+      } else {
+        return true;
+      }
+    }
+  }
+
+  PackageManager(CoreContainer coreContainer) {
     this.coreContainer = coreContainer;
     extHandler = new ExtHandler(this);
   }
 
 
-  public <T> T newInstance(String cName, Class<T> expectedType) {
+  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) {
-      if (memClassLoader != null) {
+      Package p = pkgs.get(pkg);
+
+      if (p != null) {
         try {
-          Class<? extends T> klas = memClassLoader.findClass(cName, expectedType);
+          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 (!memClassLoader.getErrors().isEmpty()) {
+          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(memClassLoader.getErrors(), ','), ex);
+                "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
@@ -101,59 +139,72 @@ public class LibListener implements ClusterPropertiesListener {
   @Override
   public boolean onChange(Map<String, Object> properties) {
     log.info("clusterprops.json changed , version {}", coreContainer.getZkController().getZkStateReader().getClusterPropsVersion());
-    boolean forceReload = updateRuntimeLibs(properties);
-    extHandler.updateReqHandlers(properties, forceReload);
+    int v = coreContainer.getZkController().getZkStateReader().getClusterPropsVersion();
+    boolean modified = updatePackages(properties, v);
+    extHandler.updateReqHandlers(properties, modified);
     for (SolrCore core : coreContainer.solrCores.getCores()) {
-      core.globalClassLoaderChanged();
+      pkgs.forEach((s, pkg) -> core.packageUpdated(pkg.lib));
     }
-    myversion = coreContainer.getZkController().getZkStateReader().getClusterPropsVersion();
+    myversion = v;
     return false;
   }
 
 
-  private boolean updateRuntimeLibs(Map<String, Object> properties) {
-    Map m = (Map) properties.getOrDefault(RuntimeLib.TYPE, Collections.emptyMap());
-    if (runtimeLibs.isEmpty() && m.isEmpty()) 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() == runtimeLibs.size()) {
+    if (m.size() == pkgs.size()) {
       m.forEach((k, v) -> {
         if (v instanceof Map) {
-          if (!runtimeLibs.containsKey(k)) needsReload[0] = true;
-          RuntimeLib rtl = runtimeLibs.get(k);
-          if (rtl == null || !Objects.equals(rtl.getSha512(), ((Map) v).get(SHA512))) {
+          Package pkg = pkgs.get(k);
+          if (pkg == null || pkg.isModified((Map) v)) {
             needsReload[0] = true;
           }
         }
-
       });
     } else {
       needsReload[0] = true;
     }
     if (needsReload[0]) {
-      createNewClassLoader(m);
+      createNewClassLoader(m, ver);
     }
     return needsReload[0];
   }
-  public ResourceLoader getResourceLoader() {
-    return memClassLoader == null ? coreContainer.getResourceLoader() : memClassLoader;
+
+  public ResourceLoader getResourceLoader(String pkg) {
+    Package p = pkgs.get(pkg);
+    return p == null ? coreContainer.getResourceLoader() : p.loader;
   }
-  void createNewClassLoader(Map m) {
+
+  void createNewClassLoader(Map m, int ver) {
     boolean[] loadedAll = new boolean[1];
     loadedAll[0] = true;
-    Map<String, RuntimeLib> libMap = new LinkedHashMap<>();
+    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(null, map));
+          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();
-          libMap.put(lib.getName(), lib);
+
+          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;
@@ -163,20 +214,20 @@ public class LibListener implements ClusterPropertiesListener {
     });
 
     if (loadedAll[0]) {
-      log.info("Libraries changed. New memclassloader created with jars {}", libMap.values().stream().map(runtimeLib -> runtimeLib.getUrl()).collect(Collectors.toList()));
-      this.memClassLoader = new MemClassLoader(new ArrayList<>(libMap.values()), coreContainer.getResourceLoader());
-      this.runtimeLibs = libMap;
+      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 LibListener libListener;
+    final PackageManager packageManager;
 
-    private Map<String, SolrRequestHandler> customHandlers = new HashMap<>();
+    private Map<String, Handler> customHandlers = new HashMap<>();
 
-    ExtHandler(LibListener libListener) {
-      this.libListener = libListener;
+    ExtHandler(PackageManager packageManager) {
+      this.packageManager = packageManager;
     }
 
 
@@ -184,15 +235,19 @@ public class LibListener implements ClusterPropertiesListener {
     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, libListener.myversion );
-        ZkStateReader zkStateReader = libListener.coreContainer.getZkController().getZkStateReader();
-        zkStateReader.forceRefreshClusterProps(v);
+        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,
-          libListener.coreContainer.getZkController().zkStateReader.getClusterPropsVersion()));
-      rsp.add(RuntimeLib.TYPE, libListener.runtimeLibs.values());
-      rsp.add(SolrRequestHandler.TYPE,
-          (IteratorWriter) iw -> customHandlers.forEach((s, h) -> iw.addNoEx(ImmutableMap.of(s, h.getClass().getName()))));
+          packageManager.coreContainer.getZkController().zkStateReader.getClusterPropsVersion()));
+      rsp.add(RuntimeLib.TYPE, packageManager.pkgs.values());
+      rsp.add(SolrRequestHandler.TYPE, customHandlers.values());
 
     }
 
@@ -206,13 +261,13 @@ public class LibListener implements ClusterPropertiesListener {
             handleRequestBody(req, rsp);
             return;
           }
-          SolrRequestHandler handler = customHandlers.get(name);
+          Handler handler = customHandlers.get(name);
           if (handler == null) {
-            String err = StrUtils.formatString(" No such handler: {0}, available handlers : {1}" , name, customHandlers.keySet());
+            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.handleRequest(req, rsp);
+          handler.handler.handleRequest(req, rsp);
         }
       });
     }
@@ -224,24 +279,37 @@ public class LibListener implements ClusterPropertiesListener {
       if (customHandlers.size() == m.size() && customHandlers.keySet().containsAll(m.keySet())) hasChanged = false;
       if (forceReload || hasChanged) {
         log.debug("RequestHandlers being reloaded : {}", m.keySet());
-        Map<String, SolrRequestHandler> newCustomHandlers = new HashMap<>();
+        Map<String, Handler> newCustomHandlers = new HashMap<>();
         m.forEach((k, v) -> {
           if (v instanceof Map) {
-            String klas = (String) ((Map) v).get(FieldType.CLASS_NAME);
-            if (klas != null) {
-              SolrRequestHandler inst = libListener.newInstance(klas, SolrRequestHandler.class);
-              if (inst instanceof PluginInfoInitialized) {
-                ((PluginInfoInitialized) inst).init(new PluginInfo(SolrRequestHandler.TYPE, (Map) v));
+            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));
               }
-              newCustomHandlers.put((String) k, inst);
+
+            } else {
+              newCustomHandlers.put(name, existing);
             }
+
           } else {
             log.error("Invalid data for requestHandler : {} , {}", k, v);
           }
         });
 
         log.debug("Registering request handlers {} ", newCustomHandlers.keySet());
-        Map<String, SolrRequestHandler> old = customHandlers;
+        Map<String, Handler> old = customHandlers;
         customHandlers = newCustomHandlers;
         old.forEach((s, h) -> PluginBag.closeQuietly(h));
       }
@@ -265,9 +333,41 @@ public class LibListener implements ClusterPropertiesListener {
 
     @Override
     public Name getPermissionName(AuthorizationContext request) {
-      if(request.getResource().endsWith("/node/ext")) return Name.COLL_READ_PERM;
+      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);
+        if (meta.equals(metaData) && p != null && p.lib.getZnodeVersion() <= zkversion) {
+          //the metadata is same and the package has not changed since we last loaded
+          return false;
+        }
+        return true;
+      }
+    }
   }
 
 }
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 1a5e987..ed7cd90 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -34,6 +34,7 @@ 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.CommonParams;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.SearchComponent;
@@ -337,15 +338,15 @@ public class PluginBag<T> implements AutoCloseable {
   }
 
   public PluginHolder<T> createPlugin(PluginInfo info) {
-    String runtimeType = info.getRuntimeLibType();
-    if ("global".equals(runtimeType)) {
-      log.debug(" {} : '{}'  created with runtimeLib=global ", meta.getCleanTag(), info.name);
-      PluginHolder<T> holder = new GlobalLoaderPluginHolder<T>(info, core, meta);
+    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 ("core".equals(String.valueOf(runtimeType))) {
+    } 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() :
@@ -478,58 +479,37 @@ public class PluginBag<T> implements AutoCloseable {
         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 GlobalLoaderPluginHolder<T> extends PluginHolder<T> {
+  public class PackagePluginHolder<T> extends PluginHolder<T> {
     private final SolrCore core;
     private final SolrConfig.SolrPluginInfo pluginMeta;
-    private final LibListener libListener;
+    private final PackageManager packageManager;
+    private final String pkg;
+    private int znodeVersion =-1;
 
-    public GlobalLoaderPluginHolder(PluginInfo info, SolrCore core, SolrConfig.SolrPluginInfo pluginMeta) {
+    public PackagePluginHolder(PluginInfo info, SolrCore core, SolrConfig.SolrPluginInfo pluginMeta) {
       super(info);
       this.core = core;
       this.pluginMeta = pluginMeta;
-      this.core.addGlobalClassLoaderListener(() -> reload());
-      this.libListener = core.getCoreContainer().getClusterPropertiesListener();
+      this.pkg = info.attributes.get(CommonParams.PACKAGE);
+      this.core.addPackageListener(pkg, (lib) -> {
+        if(lib.getZnodeVersion() > znodeVersion) reload();
+      });
+      this.packageManager = core.getCoreContainer().getPackageManager();
       reload();
     }
 
 
     private void reload() {
       if(inst == null) log.info("reloading plugin {} ", pluginInfo.name);
-      inst = createInitInstance(pluginInfo, pluginMeta, core, core.getCoreContainer().getClusterPropertiesListener().getResourceLoader(), true);
+      inst = createInitInstance(pluginInfo, pluginMeta,
+          core, packageManager.getResourceLoader(this.pkg), true);
+      znodeVersion = packageManager.getZNodeVersion(pkg);
     }
 
 
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 b537105..4ab40de 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginInfo.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
@@ -25,6 +25,7 @@ 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;
@@ -39,8 +40,7 @@ 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());
@@ -57,7 +57,7 @@ public class PluginInfo implements MapSerializable {
     this.className = attrs.get(CLASS_NAME);
     this.initArgs = initArgs;
     attributes = unmodifiableMap(attrs);
-    this.children = children == null ? Collections.<PluginInfo>emptyList(): unmodifiableList(children);
+    this.children = children == null ? Collections.<PluginInfo>emptyList() : unmodifiableList(children);
     isFromSolrConfig = false;
   }
 
@@ -72,7 +72,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()) {
@@ -97,7 +97,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.<PluginInfo>emptyList();
+    this.children = Collections.<PluginInfo>emptyList();
     isFromSolrConfig = true;
   }
 
@@ -127,37 +127,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);
         }
       }
 
@@ -165,47 +165,46 @@ 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.<String,String>emptyMap(), new NamedList(),Collections.<PluginInfo>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;
     return result;
   }
-  public String getRuntimeLibType(){
-    Object runtimeType = attributes.get(RuntimeLib.TYPE);
-    if(runtimeType == null || "false".equals(runtimeType) || Boolean.FALSE.equals(runtimeType)) return null;
-    runtimeType = runtimeType.toString();
-    if ("true".equals(String.valueOf(runtimeType))) {
-      log.warn("runtimeLib = true is deprecated, use runtimeLib=core/global");
-      runtimeType = "core";
-    }
-    return runtimeType.toString();
+
+  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
index c6da679..507fe02 100644
--- a/solr/core/src/java/org/apache/solr/core/RuntimeLib.java
+++ b/solr/core/src/java/org/apache/solr/core/RuntimeLib.java
@@ -50,6 +50,7 @@ public class RuntimeLib implements PluginInfoInitialized, AutoCloseable, MapWrit
   private String name, version, sig, sha512, url;
   private BlobRepository.BlobContentRef<ByteBuffer> jarContent;
   private boolean verified = false;
+  int znodeVersion = -1;
 
   @Override
   public void writeMap(EntryWriter ew) throws IOException {
@@ -58,6 +59,12 @@ public class RuntimeLib implements PluginInfoInitialized, AutoCloseable, MapWrit
     ew.putIfNotNull(version, version);
     ew.putIfNotNull(sha512, sha512);
     ew.putIfNotNull("sig", sig);
+    if (znodeVersion > -1) {
+      ew.put(ConfigOverlay.ZNODEVER, znodeVersion);
+    }
+  }
+  public int getZnodeVersion(){
+    return znodeVersion;
   }
 
   public RuntimeLib(CoreContainer coreContainer) {
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 b7936a9..662a3e6 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -56,6 +56,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Consumer;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.MetricRegistry;
@@ -94,6 +95,7 @@ import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.ObjectReleaseTracker;
+import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.DirectoryFactory.DirContext;
@@ -238,7 +240,8 @@ 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<Runnable> globalClassLoaderListeners = new ArrayList<>();
+  private List<Pair<String ,Consumer<RuntimeLib>>> packageListeners = new ArrayList<>();
+
 
   public Set<String> getMetricNames() {
     return metricNames;
@@ -353,14 +356,14 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
           .getPath();
     }
   }
-  void globalClassLoaderChanged(){
-    for (Runnable r : globalClassLoaderListeners) {
-      r.run();
 
+  void packageUpdated(RuntimeLib lib) {
+    for (Pair<String, Consumer<RuntimeLib>> pair : packageListeners) {
+      if(lib.equals(pair.first())) pair.second().accept(lib);
     }
   }
-  void addGlobalClassLoaderListener(Runnable r){
-    globalClassLoaderListeners.add(r);
+  public void addPackageListener(String  pkg, Consumer<RuntimeLib> r){
+    packageListeners.add(new Pair<>(pkg, r));
   }
 
 
@@ -868,9 +871,10 @@ 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;
-    ResourceLoader resourceLoader = "global".equals(info.getRuntimeLibType())?
-        coreContainer.getClusterPropertiesListener().
-        getResourceLoader(): getResourceLoader();
+    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);
     if (o instanceof PluginInfoInitialized) {
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 6e917ea..2d4487f 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -217,15 +217,15 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
         timer.getTime(), prop, expectedVersion, collection);
   }
 
-  public  static void execInparallel( List<? extends PerReplicaCallable> concurrentTasks, Consumer<ExecutorService> fun) {
+  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);
+      fun.accept(parallelExecutor);
 
-    }  finally {
+    } finally {
       ExecutorUtil.shutdownAndAwaitTermination(parallelExecutor);
     }
   }
@@ -601,7 +601,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
         }
       } catch (Exception e) {
 
-        log.error( "error executing commands "+ Utils.toJSONString(ops) ,e);
+        log.error("error executing commands " + Utils.toJSONString(ops), e);
         resp.setException(e);
         resp.add(CommandOperation.ERR_MSGS, singletonList(SchemaManager.getErrorStr(e)));
       }
@@ -817,13 +817,13 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
     private boolean verifyClass(CommandOperation op, String clz, SolrConfig.SolrPluginInfo pluginMeta) {
       if (clz == null) return true;
       PluginInfo info = new PluginInfo(pluginMeta.getCleanTag(), op.getDataMap());
-      if(info.getRuntimeLibType() != null && !RuntimeLib.isEnabled()){
+
+      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.TYPE, 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, "");
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 5a88c90..1a1e2b6 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
@@ -95,9 +95,9 @@ 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_RUNTIME_LIB, wrap(CollectionHandlerApi::addUpdateRuntimeLib));
-    addApi(apiMapping, Meta.UPDATE_RUNTIME_LIB, wrap(CollectionHandlerApi::addUpdateRuntimeLib));
-    addApi(apiMapping, Meta.DELETE_RUNTIME_LIB, wrap(CollectionHandlerApi::deleteRuntimeLib));
+    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));
 
@@ -117,9 +117,17 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
       if (modified) {
         Stat stat = new Stat();
         Map<String, Object> clusterProperties = new ClusterProperties(cc.getZkController().getZkClient()).getClusterProperties(stat);
-        cc.getClusterPropertiesListener().onChange(clusterProperties);
+        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 );
+        log.debug("The current clusterprops.json:  {}", clusterProperties);
         ((CollectionHandlerApi) info.apiHandler).waitForStateSync(stat.getVersion(), cc);
 
       }
@@ -165,7 +173,8 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
     clusterProperties.setClusterProperties(m);
     return true;
   }
-  private static boolean deleteRuntimeLib(ApiInfo params) throws Exception {
+
+  private static boolean deletePackage(ApiInfo params) throws Exception {
     if (!RuntimeLib.isEnabled()) {
       params.op.addError("node not started with enable.runtime.lib=true");
       return false;
@@ -173,9 +182,9 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
     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(RuntimeLib.TYPE, name);
+    List<String> pathToLib = asList(CommonParams.PACKAGE, name);
     Map existing = (Map) Utils.getObjectByPath(props, false, pathToLib);
-    if(existing == null){
+    if (existing == null) {
       params.op.addError("No such runtimeLib : " + name);
       return false;
     }
@@ -185,7 +194,7 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
     return true;
   }
 
-  private static boolean addUpdateRuntimeLib(ApiInfo params) throws Exception {
+  private static boolean addUpdatePackage(ApiInfo params) throws Exception {
     if (!RuntimeLib.isEnabled()) {
       params.op.addError("node not started with enable.runtime.lib=true");
       return false;
@@ -197,9 +206,9 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
     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(RuntimeLib.TYPE, name);
+    List<String> pathToLib = asList(CommonParams.PACKAGE, name);
     Map existing = (Map) Utils.getObjectByPath(props, false, pathToLib);
-    if (Meta.ADD_RUNTIME_LIB.commandName.equals(op.name)) {
+    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;
@@ -209,7 +218,7 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
         op.addError(StrUtils.formatString("The jar with a name ''{0}'' does not exist", name));
         return false;
       }
-      if(Objects.equals( existing.get(SHA512) , op.getDataMap().get(SHA512))){
+      if (Objects.equals(existing.get(SHA512), op.getDataMap().get(SHA512))) {
         op.addError("Trying to update a jar with the same sha512");
         return false;
       }
@@ -323,6 +332,7 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
   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);
     }
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 1e31577..ee3bde0 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 3ce531b..b44320c 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.XmlConfigFile;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.XmlConfigFile;
 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/rest/schema/FieldTypeXmlAdapter.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
index d25ab89..0501328 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,12 +16,11 @@
  */
 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 f0d031c..a564512 100644
--- a/solr/core/src/java/org/apache/solr/search/CacheConfig.java
+++ b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
@@ -21,12 +21,17 @@ import javax.xml.xpath.XPathConstants;
 import java.io.IOException;
 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.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.util.DOMUtil;
@@ -40,7 +45,7 @@ public class CacheConfig implements MapWriter {
   private CacheRegenerator defRegen;
   private final String name;
   private String cacheImpl, regenImpl;
-  private Object[] persistence = new Object[1];
+  Object[] persistence = new Object[1];
 
 
   public CacheConfig(Map<String, String> args) {
@@ -49,8 +54,9 @@ public class CacheConfig implements MapWriter {
     this.cacheImpl = args.getOrDefault("class", "solr.LRUCache");
     this.regenImpl = args.get("regenerator");
   }
-  static Map<String,String> copyValsAsString(Map m){
-    Map<String,String> copy = new LinkedHashMap(m.size());
+
+  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;
   }
@@ -60,6 +66,10 @@ public class CacheConfig implements MapWriter {
     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);
     } else {
       Map<String, String> attrs = DOMUtil.toMap(node.getAttributes());
@@ -100,28 +110,45 @@ public class CacheConfig implements MapWriter {
 
 
   public <K, V> SolrCacheHolder<K, V> newInstance(SolrCore core) {
-    ResourceLoader loader = core.getResourceLoader();
+    return new SolrCacheHolder<>(new CacheInfo(this, core));
+  }
 
-    SolrCache inst = null;
+  static class CacheInfo {
+    final CacheConfig cfg;
+    SolrCore core;
+    SolrCache cache = null;
+    int znodeVersion = -1;
+    String pkg;
     CacheRegenerator regen = null;
-    try {
-      inst = loader.findClass(cacheImpl, SolrCache.class).getConstructor().newInstance();
-      regen = null;
-      if (regenImpl != null) {
-        regen = loader.findClass(regenImpl, CacheRegenerator.class).getConstructor().newInstance();
-      }
-    } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error loading cache " + jsonStr(), e);
-
-    }
-    if (regen == null && defRegen != null) regen = defRegen;
 
-    persistence[0] =  inst.init(args, persistence[0], regen);
 
-    return new SolrCacheHolder<>(inst, this);
+    CacheInfo(CacheConfig cfg, SolrCore core) {
+      this.core = core;
+      this.cfg = cfg;
+      pkg = cfg.args.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, cfg.persistence[0], regen);
+      if (loader instanceof MemClassLoader) {
+        MemClassLoader memClassLoader = (MemClassLoader) loader;
+        znodeVersion = memClassLoader.getZnodeVersion();
+      }
 
+    }
   }
 
+
   public void setDefaultRegenerator(CacheRegenerator regen) {
     this.defRegen = regen;
   }
diff --git a/solr/core/src/test-files/runtimecode/MyDocCache.java b/solr/core/src/test-files/runtimecode/MyDocCache.java
new file mode 100644
index 0000000..406b950
--- /dev/null
+++ b/solr/core/src/test-files/runtimecode/MyDocCache.java
@@ -0,0 +1,35 @@
+/*
+ * 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
new file mode 100644
index 0000000..2a046fd
Binary files /dev/null and b/solr/core/src/test-files/runtimecode/cache.jar.bin 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
new file mode 100644
index 0000000..e49eed7
Binary files /dev/null and b/solr/core/src/test-files/runtimecode/cache_v2.jar.bin differ
diff --git a/solr/core/src/test-files/runtimecode/sig.txt b/solr/core/src/test-files/runtimecode/sig.txt
index a004027..8e1ba98 100644
--- a/solr/core/src/test-files/runtimecode/sig.txt
+++ b/solr/core/src/test-files/runtimecode/sig.txt
@@ -62,4 +62,12 @@ bc5ce45ad281b6a08fb7e529b1eb475040076834816570902acb6ebdd809410e31006efdeaa7f78a
 
 openssl dgst -sha512 runtimelibs_v3.jar.bin
 
-60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922
\ No newline at end of file
+60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922
+
+openssl dgst -sha512 cache.jar.bin
+
+1a3739b629ce85895c9b2a8c12dd7d98161ff47634b0693f1e1c5b444fb38343f95c6ee955cd99103bd24cfde6c205234b63823818660ac08392cdc626caf585
+
+openssl dgst -sha512 cache_v2.jar.bin
+
+aa3f42fb640636dd8126beca36ac389486d0fcb1c3a2e2c387d043d57637535ce8db3b17983853322f78bb8f447ed75fe7b405675debe652ed826ee95e8ce328
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 af83cf5..ad8612e 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
@@ -22,20 +22,26 @@ import java.io.InputStream;
 import java.io.Reader;
 import java.lang.invoke.MethodHandles;
 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.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.V2Response;
 import org.apache.solr.cloud.ConfigRequest;
@@ -48,6 +54,7 @@ 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.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;
@@ -60,8 +67,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-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;
@@ -70,7 +75,7 @@ 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.LibListener=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.PackageManager=DEBUG;org.apache.solr.common.cloud.ClusterProperties=DEBUG")
 public class TestContainerReqHandler extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
@@ -81,12 +86,13 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
   }
 
-  static void assertResponseValues(int repeats, SolrClient client, SolrRequest req, Map vals) throws Exception {
+  static SolrResponse assertResponseValues(int repeats, SolrClient client, SolrRequest req, Map vals) throws Exception {
+    SolrResponse rsp = null;
+
     for (int i = 0; i < repeats; i++) {
       if (i > 0) {
         Thread.sleep(100);
       }
-      SolrResponse rsp = null;
       try {
         rsp = req.process(client);
       } catch (Exception e) {
@@ -101,7 +107,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           return Objects.equals(val, o);
         };
         boolean isPass = p.test(rsp.getResponse()._get(key, null));
-        if(isPass) return;
+        if (isPass) return rsp;
         else if (i >= repeats - 1) {
           fail("attempt: " + i + " Mismatch for value : '" + key + "' in response " + Utils.toJSONString(rsp));
         }
@@ -109,8 +115,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       }
 
     }
-
-
+    return rsp;
   }
 
   private static Map<String, Object> assertVersionInSync(SolrZkClient zkClient, SolrClient solrClient) throws SolrServerException, IOException {
@@ -127,7 +132,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
   }
 
   @Test
-  public void testRuntimeLib() throws Exception {
+  public void testPackageAPI() throws Exception {
     Map<String, Object> jars = Utils.makeMap(
         "/jar1.jar", getFileContent("runtimecode/runtimelibs.jar.bin"),
         "/jar2.jar", getFileContent("runtimecode/runtimelibs_v2.jar.bin"),
@@ -139,7 +144,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
     try {
       String payload = null;
       try {
-        payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
+        payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
             "sha512 : 'wrong-sha512'}}";
         new V2Request.Builder("/cluster")
             .withPayload(payload)
@@ -151,7 +156,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       }
 
       try {
-        payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar0.jar', " +
+        payload = "{add-package:{name : 'foo', url: 'http://localhost:" + port + "/jar0.jar', " +
             "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
         new V2Request.Builder("/cluster")
             .withPayload(payload)
@@ -162,18 +167,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-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
+      payload = "{add-package:{name : 'global', 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-runtimelib/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
 
 
       new V2Request.Builder("/cluster")
-          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler'}}")
+          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler', package : global}}")
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
       Map<String, Object> map = new ClusterProperties(cluster.getZkClient()).getClusterProperties();
@@ -192,14 +197,14 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           getObjectByPath(map, true, Arrays.asList("requestHandler", "bar", "class")));
 
 
-      payload = "{update-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar3.jar', " +
+      payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
           "sha512 : '60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-runtimelib/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -222,7 +227,7 @@ 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-runtimelib : 'foo'}")
+          .withPayload("{delete-package : 'global'}")
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
       assertResponseValues(10, cluster.getSolrClient(), request, ImmutableMap.of(RuntimeLib.TYPE,
@@ -254,7 +259,7 @@ 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-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
+      String payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : 'EdYkvRpMZbvElN93/xUmyKXcj6xHP16AVk71TlTascEwCb5cFQ2AeKhPIlwYpkLWXEOcLZKfeXoWwOLaV5ZNhg==' ," +
           "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
       try {
@@ -268,7 +273,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       }
 
 
-      payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
+      payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : '" + signature + "'," +
           "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
 
@@ -276,11 +281,11 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-runtimelib/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
 
       new V2Request.Builder("/cluster")
-          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler'}}")
+          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler' package : global}}")
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
       Map<String, Object> map = new ClusterProperties(cluster.getZkClient()).getClusterProperties();
@@ -298,7 +303,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       assertEquals("org.apache.solr.core.RuntimeLibReqHandler",
           getObjectByPath(map, true, Arrays.asList("requestHandler", "bar", "class")));
 
-      payload = "{update-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar3.jar', " +
+      payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
           "sig : 'YxFr6SpYrDwG85miDfRWHTjU9UltjtIWQZEhcV55C2rczRUVowCYBxmsDv5mAM8j0CTv854xpI1DtBT86wpoTdbF95LQuP9FJId4TS1j8bZ9cxHP5Cqyz1uBHFfUUNUrnpzTHQkVTp02O9NAjh3c2W41bL4U7j6jQ32+4CW2M+x00TDG0y0H75rQDR8zbLt31oWCz+sBOdZ3rGKJgAvdoGm/wVCTmsabZN+xoz4JaDeBXF16O9Uk9SSq4G0dz5YXFuLxHK7ciB5t0+q6pXlF/tdlDqF76Abze0R3d2/0MhXBzyNp3UxJmj6DiprgysfB0TbQtJG0XGfdSmx0VChvcA==' ," +
           "sha512 : '60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922'}}";
 
@@ -306,8 +311,8 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-runtimelib/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -345,7 +350,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       String signature = "L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1f/U3bOlMPINlSOM6LK3JpQ==";
 
-      String payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
+      String payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : '" + signature + "'," +
           "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
 
@@ -353,11 +358,11 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-runtimelib/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
 
       new V2Request.Builder("/cluster")
-          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler'}}")
+          .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler' package : global }}")
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
       Map<String, Object> map = new ClusterProperties(cluster.getZkClient()).getClusterProperties();
@@ -375,7 +380,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       assertEquals("org.apache.solr.core.RuntimeLibReqHandler",
           getObjectByPath(map, true, Arrays.asList("requestHandler", "bar", "class")));
 
-      payload = "{update-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar3.jar', " +
+      payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
           "sig : 'a400n4T7FT+2gM0SC6+MfSOExjud8MkhTSFylhvwNjtWwUgKdPFn434Wv7Qc4QEqDVLhQoL3WqYtQmLPti0G4Q==' ," +
           "sha512 : '60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922'}}";
 
@@ -383,8 +388,8 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-runtimelib/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -457,20 +462,20 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
 
       cluster.waitForActiveCollection(COLLECTION_NAME, 2, 2);
-      String payload = "{add-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar1.jar', " +
+      String payload = "{add-package:{name : 'global', 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-runtimelib/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
 
 
       payload = "{\n" +
-          "'create-requesthandler' : { 'name' : '/runtime', 'class': 'org.apache.solr.core.RuntimeLibReqHandler' , 'runtimeLib':global }," +
-          "'create-searchcomponent' : { 'name' : 'get', 'class': 'org.apache.solr.core.RuntimeLibSearchComponent' , 'runtimeLib':global }," +
-          "'create-queryResponseWriter' : { 'name' : 'json1', 'class': 'org.apache.solr.core.RuntimeLibResponseWriter' , 'runtimeLib':global }" +
+          "'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
@@ -528,14 +533,14 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
               "loader", MemClassLoader.class.getName()));
 
 
-      payload = "{update-runtimelib:{name : 'foo', url: 'http://localhost:" + port + "/jar2.jar', " +
+      payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar2.jar', " +
           "sha512 : 'bc5ce45ad281b6a08fb7e529b1eb475040076834816570902acb6ebdd809410e31006efdeaa7f78a6c35574f3504963f5f7e4d92247d0eb4db3fc9abdda5d417'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-runtimelib/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "runtimeLib/foo/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
 
       try {
         new V2Request.Builder("/cluster")
@@ -562,4 +567,89 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
   }
 
+  public void testCacheFromGlobalLoader() 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()))
+        .configure();
+    try {
+      String payload = "{add-package:{name : 'cache_pkg', url: 'http://localhost:" + port + "/jar1.jar', " +
+          "sha512 : '1a3739b629ce85895c9b2a8c12dd7d98161ff47634b0693f1e1c5b444fb38343f95c6ee955cd99103bd24cfde6c205234b63823818660ac08392cdc626caf585'}}";
+
+      new V2Request.Builder("/cluster")
+          .withPayload(payload)
+          .withMethod(SolrRequest.METHOD.POST)
+          .build().process(cluster.getSolrClient());
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/cache_pkg/sha512"));
+
+      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));
+      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", "*:*", "collection", COLLECTION_NAME);
+      assertResponseValues(10,
+          cluster.getSolrClient(),
+          new QueryRequest(solrQuery),
+          Utils.makeMap("response[0]/id", "1"));
+
+
+      payload = "{update-package:{name : 'cache_pkg', url: 'http://localhost:" + port + "/jar2.jar', " +
+          "sha512 : 'aa3f42fb640636dd8126beca36ac389486d0fcb1c3a2e2c387d043d57637535ce8db3b17983853322f78bb8f447ed75fe7b405675debe652ed826ee95e8ce328'}}";
+      new V2Request.Builder("/cluster")
+          .withPayload(payload)
+          .withMethod(SolrRequest.METHOD.POST)
+          .build().process(cluster.getSolrClient());
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/cache_pkg/sha512"));
+
+      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/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 964c0a2..43ba737 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
@@ -208,15 +208,15 @@ public class CollectionApiMapping {
         null,
         "set-obj-property", null),
 
-    ADD_RUNTIME_LIB(CLUSTER_CMD,
+    ADD_PACKAGE(CLUSTER_CMD,
         POST,null,
-        "add-runtimelib",null ),
-    UPDATE_RUNTIME_LIB(CLUSTER_CMD,
+        "add-package",null ),
+    UPDATE_PACKAGE(CLUSTER_CMD,
         POST,null,
-        "update-runtimelib",null ),
+        "update-package",null ),
     DELETE_RUNTIME_LIB(CLUSTER_CMD,
         POST,null,
-        "delete-runtimelib",null ),
+        "delete-package",null ),
     ADD_REQ_HANDLER(CLUSTER_CMD,
         POST,null,
         "add-requesthandler",null ),
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 39a0242..4ce1f50 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
@@ -296,5 +296,7 @@ public interface CommonParams {
 
   String JAVABIN_MIME = "application/javabin";
 
+  String PACKAGE = "package";
+
 }
 
diff --git a/solr/solrj/src/resources/apispec/cluster.Commands.json b/solr/solrj/src/resources/apispec/cluster.Commands.json
index 0b4f33f..af7c19d 100644
--- a/solr/solrj/src/resources/apispec/cluster.Commands.json
+++ b/solr/solrj/src/resources/apispec/cluster.Commands.json
@@ -170,17 +170,17 @@
         "name"
       ]
     },
-    "add-runtimelib": {
+    "add-package": {
       "documentation": "",
-      "description" : "Add a remote jar to the classpath",
+      "description" : "Add a package to the classpath",
       "#include": "cluster.Commands.runtimelib.properties"
     },
-    "update-runtimelib": {
+    "update-package": {
       "documentation": "",
       "description" : "Update the jar details",
       "#include": "cluster.Commands.runtimelib.properties"
     },
-    "delete-runtimelib": {
+    "delete-package": {
       "documentation": "",
       "description" : "delete a lib",
       "type": "string"
@@ -197,6 +197,10 @@
         "class": {
           "type": "string",
           "description": "The class name"
+        },
+        "package" : {
+          "type": "string",
+          "description": " The package from where the plugin can be loaded from"
         }
       },
       "required": ["name", "class"],
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 191f441..e592307 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
@@ -54,11 +54,14 @@ 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
  *
@@ -86,10 +89,12 @@ 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) {
+    private Config(String name, Path path, Map<String,byte[]> extraConfig) {
       this.name = name;
       this.path = path;
+      this.extraConfig = extraConfig;
     }
   }
 
@@ -178,7 +183,12 @@ 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));
+      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));
       return this;
     }
 
@@ -214,6 +224,14 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
       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);
+
+          }
+
+        }
       }
 
       if (clusterProperties.size() > 0) {


[lucene-solr] 05/05: SOLR-13650: CHANGES.txt

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 625e3f38b21fb8df04242bbc9febac829b7ce449
Author: noble <no...@apache.org>
AuthorDate: Tue Aug 20 11:52:41 2019 +1000

    SOLR-13650: CHANGES.txt
---
 solr/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 19f722e..bf202a1 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -59,6 +59,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)
+
 Improvements
 ----------------------
 


[lucene-solr] 02/05: SOLR-13650: compile error fixed

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 129231d4a7bf5fe316ce8840cd702f59a335ce6e
Author: noble <no...@apache.org>
AuthorDate: Tue Aug 20 05:12:58 2019 +1000

    SOLR-13650: compile error fixed
---
 solr/core/src/java/org/apache/solr/search/CacheConfig.java     |  2 +-
 solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java | 10 ++++++----
 2 files changed, 7 insertions(+), 5 deletions(-)

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 a564512..cc92f0d 100644
--- a/solr/core/src/java/org/apache/solr/search/CacheConfig.java
+++ b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
@@ -110,7 +110,7 @@ public class CacheConfig implements MapWriter {
 
 
   public <K, V> SolrCacheHolder<K, V> newInstance(SolrCore core) {
-    return new SolrCacheHolder<>(new CacheInfo(this, core));
+    return new SolrCacheHolder(new CacheInfo(this, core));
   }
 
   static class CacheInfo {
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 86f6b2d..9f1e124 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
@@ -30,12 +30,14 @@ public class SolrCacheHolder<K, V> implements SolrCache<K,V> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
 
-  private final CacheConfig factory;
+  private CacheConfig.CacheInfo info;
   protected volatile SolrCache<K, V> delegate;
 
-  public SolrCacheHolder(SolrCache<K, V> delegate, CacheConfig factory) {
-    this.delegate = delegate;
-    this.factory = factory;
+
+
+  public SolrCacheHolder(CacheConfig.CacheInfo cacheInfo) {
+    this.info = cacheInfo;
+    this.delegate = cacheInfo.cache;
   }
 
   public int size() {


[lucene-solr] 04/05: SOLR-13650: precommit errors fixed

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 6d3d3c7a6c5fcf6274f2b2a412aa1ea6ca9b7856
Author: noble <no...@apache.org>
AuthorDate: Tue Aug 20 11:48:56 2019 +1000

    SOLR-13650: precommit errors fixed
---
 .../core/src/test/org/apache/solr/handler/TestContainerReqHandler.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

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 ad8612e..44c1d1e 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
@@ -67,6 +67,7 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 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;
@@ -583,7 +584,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
         "          \"initialSize\":\"512\" , \"package\":\"cache_pkg\"}}}}";
     MiniSolrCloudCluster cluster = configureCluster(4)
         .addConfig("conf", configset("cloud-minimal"),
-            Collections.singletonMap(ConfigOverlay.RESOURCE_NAME, overlay.getBytes()))
+            Collections.singletonMap(ConfigOverlay.RESOURCE_NAME, overlay.getBytes(UTF_8)))
         .configure();
     try {
       String payload = "{add-package:{name : 'cache_pkg', url: 'http://localhost:" + port + "/jar1.jar', " +


[lucene-solr] 03/05: SOLR-13650: cache reload listener

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit ecfd0c14bced2235075a04569c944db6a793c164
Author: noble <no...@apache.org>
AuthorDate: Tue Aug 20 05:49:46 2019 +1000

    SOLR-13650: cache reload listener
---
 .../core/src/java/org/apache/solr/search/SolrCacheHolder.java | 11 +++++++++++
 1 file changed, 11 insertions(+)

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 9f1e124..266b743 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.Set;
 
 import com.codahale.metrics.MetricRegistry;
+import org.apache.solr.core.RuntimeLib;
 import org.apache.solr.metrics.SolrMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,6 +39,16 @@ public class SolrCacheHolder<K, V> implements SolrCache<K,V> {
   public SolrCacheHolder(CacheConfig.CacheInfo cacheInfo) {
     this.info = cacheInfo;
     this.delegate = cacheInfo.cache;
+    if(info.pkg != null){
+      info.core.addPackageListener(info.pkg, lib -> reloadCache(lib));
+    }
+  }
+
+  private void reloadCache(RuntimeLib lib) {
+    if (lib.getZnodeVersion() > info.znodeVersion) {
+      info = new CacheConfig.CacheInfo(info.cfg, info.core);
+      delegate = info.cache;
+    }
   }
 
   public int size() {