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/10/12 06:17:36 UTC

[lucene-solr] 01/01: SOLR-13822: Isolated class loading from packages

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

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

commit 08f9e293ca3db6115f4732f76fb32ad69713a070
Author: noble <no...@apache.org>
AuthorDate: Sat Oct 12 17:16:43 2019 +1100

    SOLR-13822: Isolated class loading from packages
---
 .../java/org/apache/solr/core/CoreContainer.java   |  10 +
 .../src/java/org/apache/solr/core/PluginBag.java   |  15 +-
 .../src/java/org/apache/solr/core/PluginInfo.java  |  39 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |  18 +-
 .../org/apache/solr/core/SolrResourceLoader.java   | 333 ++++++++++--------
 .../org/apache/solr/handler/SolrConfigHandler.java |  23 +-
 .../src/java/org/apache/solr/pkg/PackageAPI.java   | 367 +++++++++++++++++++
 .../java/org/apache/solr/pkg/PackageListeners.java |  89 +++++
 .../java/org/apache/solr/pkg/PackageLoader.java    | 254 +++++++++++++
 .../org/apache/solr/pkg/PackagePluginHolder.java   |  92 +++++
 .../solr/security/PermissionNameProvider.java      |   3 +
 .../src/test/org/apache/solr/pkg/TestPackages.java | 391 +++++++++++++++++++++
 .../org/apache/solr/common/cloud/SolrZkClient.java |   9 +-
 .../apache/solr/common/cloud/ZkStateReader.java    |   1 +
 14 files changed, 1471 insertions(+), 173 deletions(-)

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 054bd67..c32783c 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -100,6 +100,7 @@ import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.pkg.PackageLoader;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.search.SolrFieldCacheBean;
@@ -221,6 +222,7 @@ public class CoreContainer {
   protected volatile AutoscalingHistoryHandler autoscalingHistoryHandler;
 
   private PackageStoreAPI packageStoreAPI;
+  private PackageLoader packageLoader;
 
 
   // Bits for the state variable.
@@ -580,6 +582,10 @@ public class CoreContainer {
     return replayUpdatesExecutor;
   }
 
+  public PackageLoader getPackageLoader() {
+    return packageLoader;
+  }
+
   public PackageStoreAPI getPackageStoreAPI() {
     return packageStoreAPI;
   }
@@ -734,6 +740,10 @@ public class CoreContainer {
     if (isZooKeeperAware()) {
       metricManager.loadClusterReporters(metricReporters, this);
     }
+    packageLoader = new PackageLoader(this);
+    containerHandlers.getApiBag().register(new AnnotatedApi(packageLoader.getPackageAPI().editAPI), Collections.EMPTY_MAP);
+    containerHandlers.getApiBag().register(new AnnotatedApi(packageLoader.getPackageAPI().readAPI), Collections.EMPTY_MAP);
+
 
     // setup executor to load cores in parallel
     ExecutorService coreLoadExecutor = MetricUtils.instrumentedExecutorService(
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 6088f52..bfdfa20 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -43,6 +43,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.SearchComponent;
+import org.apache.solr.pkg.PackagePluginHolder;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
@@ -97,7 +98,7 @@ public class PluginBag<T> implements AutoCloseable {
     this(klass, core, false);
   }
 
-  static void initInstance(Object inst, PluginInfo info) {
+  public static void initInstance(Object inst, PluginInfo info) {
     if (inst instanceof PluginInfoInitialized) {
       ((PluginInfoInitialized) inst).init(info);
     } else if (inst instanceof NamedListInitializedPlugin) {
@@ -138,9 +139,13 @@ public class PluginBag<T> implements AutoCloseable {
       log.debug("{} : '{}' created with startup=lazy ", meta.getCleanTag(), info.name);
       return new LazyPluginHolder<T>(meta, info, core, core.getResourceLoader(), false);
     } else {
-      T inst = core.createInstance(info.className, (Class<T>) meta.clazz, meta.getCleanTag(), null, core.getResourceLoader());
-      initInstance(inst, info);
-      return new PluginHolder<>(info, inst);
+      if (info.pkgName != null) {
+        return new PackagePluginHolder<>(info, core, meta);
+      } else {
+        T inst = core.createInstance(info.className, (Class<T>) meta.clazz, meta.getCleanTag(), null, core.getResourceLoader(info.pkgName));
+        initInstance(inst, info);
+        return new PluginHolder<>(info, inst);
+      }
     }
   }
 
@@ -329,7 +334,7 @@ public class PluginBag<T> implements AutoCloseable {
    * subclasses may choose to lazily load the plugin
    */
   public static class PluginHolder<T> implements AutoCloseable {
-    private T inst;
+    protected T inst;
     protected final PluginInfo pluginInfo;
     boolean registerAPI = false;
 
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 1bc85ae..366309e 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginInfo.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
@@ -16,14 +16,20 @@
  */
 package org.apache.solr.core;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.solr.common.MapSerializable;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Pair;
 import org.apache.solr.util.DOMUtil;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
-import java.util.*;
-
 import static java.util.Arrays.asList;
 import static java.util.Collections.unmodifiableList;
 import static java.util.Collections.unmodifiableMap;
@@ -35,27 +41,46 @@ import static org.apache.solr.schema.FieldType.CLASS_NAME;
  *
  */
 public class PluginInfo implements MapSerializable {
-  public final String name, className, type;
+  public final String name, className, type, pkgName;
   public final NamedList initArgs;
   public final Map<String, String> attributes;
   public final List<PluginInfo> children;
   private boolean isFromSolrConfig;
 
+
+
   public PluginInfo(String type, Map<String, String> attrs, NamedList initArgs, List<PluginInfo> children) {
     this.type = type;
     this.name = attrs.get(NAME);
-    this.className = attrs.get(CLASS_NAME);
+    Pair<String, String> parsed = parseClassName(attrs.get(CLASS_NAME));
+    this.className = parsed.second();
+    this.pkgName = parsed.first();
     this.initArgs = initArgs;
     attributes = unmodifiableMap(attrs);
     this.children = children == null ? Collections.<PluginInfo>emptyList(): unmodifiableList(children);
     isFromSolrConfig = false;
   }
+  static Pair<String,String > parseClassName(String name) {
+    String pkgName = null;
+    String className = name;
+    if (name != null) {
+      int colonIdx = name.indexOf(':');
+      if (colonIdx > -1) {
+        pkgName = name.substring(0, colonIdx);
+        className = name.substring(colonIdx + 1);
+      }
+    }
+    return new Pair<>(pkgName, className);
+
+  }
 
 
   public PluginInfo(Node node, String err, boolean requireName, boolean requireClass) {
     type = node.getNodeName();
     name = DOMUtil.getAttr(node, NAME, requireName ? err : null);
-    className = DOMUtil.getAttr(node, CLASS_NAME, requireClass ? err : null);
+    Pair<String, String> parsed = parseClassName(DOMUtil.getAttr(node, CLASS_NAME, requireClass ? err : null));
+    className = parsed.second();
+    pkgName = parsed.first();
     initArgs = DOMUtil.childNodesToNamedList(node);
     attributes = unmodifiableMap(DOMUtil.toMap(node.getAttributes()));
     children = loadSubPlugins(node);
@@ -85,7 +110,9 @@ public class PluginInfo implements MapSerializable {
     }
     this.type = type;
     this.name = (String) m.get(NAME);
-    this.className = (String) m.get(CLASS_NAME);
+    Pair<String, String> parsed = parseClassName((String) m.get(CLASS_NAME));
+    this.className = parsed.second();
+    this.pkgName = parsed.first();
     attributes = unmodifiableMap(m);
     this.children =  Collections.<PluginInfo>emptyList();
     isFromSolrConfig = true;
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 3e2fb1e..59c9a7a 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -110,6 +110,8 @@ import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.pkg.PackageListeners;
+import org.apache.solr.pkg.PackageLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.BinaryResponseWriter;
@@ -237,6 +239,8 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
   public volatile boolean indexEnabled = true;
   public volatile boolean readOnly = false;
 
+  private PackageListeners packageListeners = new PackageListeners();
+
   public Set<String> getMetricNames() {
     return metricNames;
   }
@@ -261,6 +265,10 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
     return restManager;
   }
 
+  public PackageListeners getPackageListeners() {
+    return packageListeners;
+  }
+
   static int boolean_query_max_clause_count = Integer.MIN_VALUE;
 
   private ExecutorService coreAsyncTaskExecutor = ExecutorUtil.newMDCAwareCachedThreadPool("Core Async Task");
@@ -274,6 +282,14 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
     return resourceLoader;
   }
 
+  public SolrResourceLoader getResourceLoader(String pkg) {
+    if (pkg == null) {
+      return resourceLoader;
+    }
+    PackageLoader.Package aPackage = coreContainer.getPackageLoader().getPackage(pkg);
+    return aPackage.getLatest().getLoader();
+  }
+
   /**
    * Gets the configuration resource name used by this core instance.
    *
@@ -856,7 +872,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
 
   public <T extends Object> T createInitInstance(PluginInfo info, Class<T> cast, String msg, String defClassName) {
     if (info == null) return null;
-    T o = createInstance(info.className == null ? defClassName : info.className, cast, msg, this, getResourceLoader());
+    T o = createInstance(info.className == null ? defClassName : info.className, cast, msg, this, getResourceLoader(info.pkgName));
     if (o instanceof PluginInfoInitialized) {
       ((PluginInfoInitialized) o).init(info);
     } else if (o instanceof NamedListInitializedPlugin) {
diff --git a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
index 4132918..cc1ef7a 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
@@ -28,6 +28,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.lang.invoke.MethodHandles;
 import java.lang.reflect.Constructor;
+import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.nio.charset.CharacterCodingException;
@@ -81,11 +82,11 @@ import org.slf4j.LoggerFactory;
 
 /**
  * @since solr 1.3
- */ 
-public class SolrResourceLoader implements ResourceLoader,Closeable
-{
+ */
+public class SolrResourceLoader implements ResourceLoader, Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  private String name = "";
   static final String project = "solr";
   static final String base = "org.apache" + "." + project;
   static final String[] packages = {
@@ -100,7 +101,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   protected URLClassLoader classLoader;
   private final Path instanceDir;
   private String dataDir;
-  
+
   private final List<SolrCoreAware> waitingForCore = Collections.synchronizedList(new ArrayList<SolrCoreAware>());
   private final List<SolrInfoBean> infoMBeans = Collections.synchronizedList(new ArrayList<SolrInfoBean>());
   private final List<ResourceLoaderAware> waitingForResources = Collections.synchronizedList(new ArrayList<ResourceLoaderAware>());
@@ -109,18 +110,18 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   private final Properties coreProperties;
 
   private volatile boolean live;
-  
+
   // Provide a registry so that managed resources can register themselves while the XML configuration
   // documents are being parsed ... after all are registered, they are asked by the RestManager to
   // initialize themselves. This two-step process is required because not all resources are available
   // (such as the SolrZkClient) when XML docs are being parsed.    
   private RestManager.Registry managedResourceRegistry;
-  
+
   public synchronized RestManager.Registry getManagedResourceRegistry() {
     if (managedResourceRegistry == null) {
-      managedResourceRegistry = new RestManager.Registry();      
+      managedResourceRegistry = new RestManager.Registry();
     }
-    return managedResourceRegistry; 
+    return managedResourceRegistry;
   }
 
   public SolrResourceLoader() {
@@ -134,11 +135,20 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
    * found in the "lib/" directory in the specified instance directory.
    * If the instance directory is not specified (=null), SolrResourceLoader#locateInstanceDir will provide one.
    */
-  public SolrResourceLoader(Path instanceDir, ClassLoader parent)
-  {
+  public SolrResourceLoader(Path instanceDir, ClassLoader parent) {
     this(instanceDir, parent, null);
   }
 
+  public SolrResourceLoader(String name, List<Path> classpath, Path instanceDir, ClassLoader parent) throws MalformedURLException {
+    this(instanceDir, parent);
+    this.name = name;
+    for (Path path : classpath) {
+      addToClassLoader(path.toUri().normalize().toURL());
+    }
+
+  }
+
+
   public SolrResourceLoader(Path instanceDir) {
     this(instanceDir, null, null);
   }
@@ -157,7 +167,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
     if (instanceDir == null) {
       this.instanceDir = SolrResourceLoader.locateSolrHome().toAbsolutePath().normalize();
       log.debug("new SolrResourceLoader for deduced Solr Home: '{}'", this.instanceDir);
-    } else{
+    } else {
       this.instanceDir = instanceDir.toAbsolutePath().normalize();
       log.debug("new SolrResourceLoader for directory: '{}'", this.instanceDir);
     }
@@ -167,7 +177,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
     }
     this.classLoader = URLClassLoader.newInstance(new URL[0], parent);
 
-    /* 
+    /*
      * Skip the lib subdirectory when we are loading from the solr home.
      * Otherwise load it, so core lib directories still get loaded.
      * The default sharedLib will pick this up later, and if the user has
@@ -264,6 +274,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
 
   /**
    * Utility method to get the URLs of all paths under a given directory that match a filter
+   *
    * @param libDir the root directory
    * @param filter the filter
    * @return all matching URLs
@@ -296,8 +307,9 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
 
   /**
    * Utility method to get the URLs of all paths under a given directory that match a regex
+   *
    * @param libDir the root directory
-   * @param regex the regex as a String
+   * @param regex  the regex as a String
    * @return all matching URLs
    * @throws IOException on error
    */
@@ -310,15 +322,17 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
       }
     });
   }
-  
-  /** Ensures a directory name always ends with a '/'. */
+
+  /**
+   * Ensures a directory name always ends with a '/'.
+   */
   public static String normalizeDir(String path) {
-    return ( path != null && (!(path.endsWith("/") || path.endsWith("\\"))) )? path + File.separator : path;
+    return (path != null && (!(path.endsWith("/") || path.endsWith("\\")))) ? path + File.separator : path;
   }
-  
+
   public String[] listConfigDir() {
     File configdir = new File(getConfigDir());
-    if( configdir.exists() && configdir.isDirectory() ) {
+    if (configdir.exists() && configdir.isDirectory()) {
       return configdir.list();
     } else {
       return new String[0];
@@ -328,8 +342,8 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   public String getConfigDir() {
     return instanceDir.resolve("conf").toString();
   }
-  
-  public String getDataDir()    {
+
+  public String getDataDir() {
     return dataDir;
   }
 
@@ -341,23 +355,28 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
    * EXPERT
    * <p>
    * The underlying class loader.  Most applications will not need to use this.
+   *
    * @return The {@link ClassLoader}
    */
   public ClassLoader getClassLoader() {
     return classLoader;
   }
 
-  /** Opens a schema resource by its name.
+  /**
+   * Opens a schema resource by its name.
    * Override this method to customize loading schema resources.
-   *@return the stream for the named schema
+   *
+   * @return the stream for the named schema
    */
   public InputStream openSchema(String name) throws IOException {
     return openResource(name);
   }
-  
-  /** Opens a config resource by its name.
+
+  /**
+   * Opens a config resource by its name.
    * Override this method to customize loading config resources.
-   *@return the stream for the named configuration
+   *
+   * @return the stream for the named configuration
    */
   public InputStream openConfig(String name) throws IOException {
     return openResource(name);
@@ -372,14 +391,16 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
     throw new IOException("File " + pathToCheck + " is outside resource loader dir " + instanceDir +
         "; set -Dsolr.allow.unsafe.resourceloading=true to allow unsafe loading");
   }
-  
-  /** Opens any resource by its name.
+
+  /**
+   * Opens any resource by its name.
    * By default, this will look in multiple locations to load the resource:
    * $configDir/$resource (if resource is not absolute)
    * $CWD/$resource
    * otherwise, it will look for it in any jar accessible through the class loader.
    * Override this method to customize loading resources.
-   *@return the stream for the named resource
+   *
+   * @return the stream for the named resource
    */
   @Override
   public InputStream openResource(String resource) throws IOException {
@@ -461,22 +482,22 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
    * @throws IOException If there is a low-level I/O error.
    */
   public List<String> getLines(String resource,
-      String encoding) throws IOException {
+                               String encoding) throws IOException {
     return getLines(resource, Charset.forName(encoding));
   }
 
 
-  public List<String> getLines(String resource, Charset charset) throws IOException{
+  public List<String> getLines(String resource, Charset charset) throws IOException {
     try {
       return WordlistLoader.getLines(openResource(resource), charset);
     } catch (CharacterCodingException ex) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
-         "Error loading resource (wrong encoding?): " + resource, ex);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Error loading resource (wrong encoding?): " + resource, ex);
     }
   }
 
   /*
-   * A static map of short class name to fully qualified class name 
+   * A static map of short class name to fully qualified class name
    */
   private static final Map<String, String> classNameCache = new ConcurrentHashMap<>();
 
@@ -486,14 +507,14 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   }
 
   // Using this pattern, legacy analysis components from previous Solr versions are identified and delegated to SPI loader:
-  private static final Pattern legacyAnalysisPattern = 
-      Pattern.compile("((\\Q"+base+".analysis.\\E)|(\\Q"+project+".\\E))([\\p{L}_$][\\p{L}\\p{N}_$]+?)(TokenFilter|Filter|Tokenizer|CharFilter)Factory");
+  private static final Pattern legacyAnalysisPattern =
+      Pattern.compile("((\\Q" + base + ".analysis.\\E)|(\\Q" + project + ".\\E))([\\p{L}_$][\\p{L}\\p{N}_$]+?)(TokenFilter|Filter|Tokenizer|CharFilter)Factory");
 
   @Override
   public <T> Class<? extends T> findClass(String cname, Class<T> expectedType) {
     return findClass(cname, expectedType, empty);
   }
-  
+
   /**
    * This method loads a class either with its FQN or a short-name (solr.class-simplename or class-simplename).
    * It tries to load the class with the name that is given first and if it fails, it tries all the known
@@ -501,25 +522,25 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
    * for the same class faster. The caching is done only if the class is loaded by the webapp classloader and it
    * is loaded using a shortname.
    *
-   * @param cname The name or the short name of the class.
+   * @param cname       The name or the short name of the class.
    * @param subpackages the packages to be tried if the cname starts with solr.
    * @return the loaded class. An exception is thrown if it fails
    */
   public <T> Class<? extends T> findClass(String cname, Class<T> expectedType, String... subpackages) {
     if (subpackages == null || subpackages.length == 0 || subpackages == packages) {
       subpackages = packages;
-      String  c = classNameCache.get(cname);
-      if(c != null) {
+      String c = classNameCache.get(cname);
+      if (c != null) {
         try {
           return Class.forName(c, true, classLoader).asSubclass(expectedType);
         } catch (ClassNotFoundException | ClassCastException e) {
           // this can happen if the legacyAnalysisPattern below caches the wrong thing
-          log.warn("Unable to load cached class, attempting lookup. name={} shortname={} reason={}", c, cname, e);
+          log.warn( name + " Unable to load cached class, attempting lookup. name={} shortname={} reason={}", c, cname, e);
           classNameCache.remove(cname);
         }
       }
     }
-    
+
     Class<? extends T> clazz = null;
     try {
       // first try legacy analysis patterns, now replaced by Lucene's Analysis package:
@@ -537,43 +558,43 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
           } else {
             log.warn("'{}' looks like an analysis factory, but caller requested different class type: {}", cname, expectedType.getName());
           }
-        } catch (IllegalArgumentException ex) { 
+        } catch (IllegalArgumentException ex) {
           // ok, we fall back to legacy loading
         }
       }
-      
+
       // first try cname == full name
       try {
         return clazz = Class.forName(cname, true, classLoader).asSubclass(expectedType);
       } catch (ClassNotFoundException e) {
-        String newName=cname;
+        String newName = cname;
         if (newName.startsWith(project)) {
-          newName = cname.substring(project.length()+1);
+          newName = cname.substring(project.length() + 1);
         }
         for (String subpackage : subpackages) {
           try {
             String name = base + '.' + subpackage + newName;
             log.trace("Trying class name " + name);
-            return clazz = Class.forName(name,true,classLoader).asSubclass(expectedType);
+            return clazz = Class.forName(name, true, classLoader).asSubclass(expectedType);
           } catch (ClassNotFoundException e1) {
             // ignore... assume first exception is best.
           }
         }
-    
-        throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, "Error loading class '" + cname + "'", e);
+
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, name +" Error loading class '" + cname + "'", e);
       }
-      
+
     } finally {
       if (clazz != null) {
         //cache the shortname vs FQN if it is loaded by the webapp classloader  and it is loaded
         // using a shortname
         if (clazz.getClassLoader() == SolrResourceLoader.class.getClassLoader() &&
-              !cname.equals(clazz.getName()) &&
-              (subpackages.length == 0 || subpackages == packages)) {
+            !cname.equals(clazz.getName()) &&
+            (subpackages.length == 0 || subpackages == packages)) {
           //store in the cache
           classNameCache.put(cname, clazz.getName());
         }
-        
+
         // print warning if class is deprecated
         if (clazz.isAnnotationPresent(Deprecated.class)) {
           log.warn("Solr loaded a deprecated plugin/analysis class [{}]. Please consult documentation how to replace it accordingly.",
@@ -582,9 +603,9 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
       }
     }
   }
-  
+
   static final String empty[] = new String[0];
-  
+
   @Override
   public <T> T newInstance(String name, Class<T> expectedType) {
     return newInstance(name, expectedType, empty);
@@ -593,33 +614,32 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   private static final Class[] NO_CLASSES = new Class[0];
   private static final Object[] NO_OBJECTS = new Object[0];
 
-  public <T> T newInstance(String cname, Class<T> expectedType, String ... subpackages) {
+  public <T> T newInstance(String cname, Class<T> expectedType, String... subpackages) {
     return newInstance(cname, expectedType, subpackages, NO_CLASSES, NO_OBJECTS);
   }
 
-  public CoreAdminHandler newAdminHandlerInstance(final CoreContainer coreContainer, String cname, String ... subpackages) {
+  public CoreAdminHandler newAdminHandlerInstance(final CoreContainer coreContainer, String cname, String... subpackages) {
     Class<? extends CoreAdminHandler> clazz = findClass(cname, CoreAdminHandler.class, subpackages);
-    if( clazz == null ) {
-      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-          "Can not find class: "+cname + " in " + classLoader);
+    if (clazz == null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Can not find class: " + cname + " in " + classLoader);
     }
-    
+
     CoreAdminHandler obj = null;
     try {
       Constructor<? extends CoreAdminHandler> ctor = clazz.getConstructor(CoreContainer.class);
       obj = ctor.newInstance(coreContainer);
-    } 
-    catch (Exception e) {
-      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-          "Error instantiating class: '" + clazz.getName()+"'", e);
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Error instantiating class: '" + clazz.getName() + "'", e);
     }
 
     if (!live) {
       //TODO: Does SolrCoreAware make sense here since in a multi-core context
       // which core are we talking about ?
-      if( obj instanceof ResourceLoaderAware ) {
-        assertAwareCompatibility( ResourceLoaderAware.class, obj );
-        waitingForResources.add( (ResourceLoaderAware)obj );
+      if (obj instanceof ResourceLoaderAware) {
+        assertAwareCompatibility(ResourceLoaderAware.class, obj);
+        waitingForResources.add((ResourceLoaderAware) obj);
       }
     }
 
@@ -627,12 +647,11 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   }
 
 
-
-  public <T> T newInstance(String cName, Class<T> expectedType, String [] subPackages, Class[] params, Object[] args){
+  public <T> T newInstance(String cName, Class<T> expectedType, String[] subPackages, Class[] params, Object[] args) {
     Class<? extends T> clazz = findClass(cName, expectedType, subPackages);
-    if( clazz == null ) {
-      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-          "Can not find class: "+cName + " in " + classLoader);
+    if (clazz == null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Can not find class: " + cName + " in " + classLoader);
     }
 
     T obj = null;
@@ -653,25 +672,25 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
       }
 
     } catch (Error err) {
-      log.error("Loading Class " + cName + " ("+clazz.getName() + ") triggered serious java error: "
-                + err.getClass().getName(), err);
+      log.error("Loading Class " + cName + " (" + clazz.getName() + ") triggered serious java error: "
+          + err.getClass().getName(), err);
       throw err;
 
     } catch (Exception e) {
-      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-          "Error instantiating class: '" + clazz.getName()+"'", e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Error instantiating class: '" + clazz.getName() + "'", e);
     }
 
     if (!live) {
-      if( obj instanceof SolrCoreAware ) {
-        assertAwareCompatibility( SolrCoreAware.class, obj );
-        waitingForCore.add( (SolrCoreAware)obj );
+      if (obj instanceof SolrCoreAware) {
+        assertAwareCompatibility(SolrCoreAware.class, obj);
+        waitingForCore.add((SolrCoreAware) obj);
       }
-      if( obj instanceof ResourceLoaderAware ) {
-        assertAwareCompatibility( ResourceLoaderAware.class, obj );
-        waitingForResources.add( (ResourceLoaderAware)obj );
+      if (obj instanceof ResourceLoaderAware) {
+        assertAwareCompatibility(ResourceLoaderAware.class, obj);
+        waitingForResources.add((ResourceLoaderAware) obj);
       }
-      if (obj instanceof SolrInfoBean){
+      if (obj instanceof SolrInfoBean) {
         //TODO: Assert here?
         infoMBeans.add((SolrInfoBean) obj);
       }
@@ -680,12 +699,11 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
     return obj;
   }
 
-  
+
   /**
    * Tell all {@link SolrCoreAware} instances about the SolrCore
    */
-  public void inform(SolrCore core) 
-  {
+  public void inform(SolrCore core) {
     this.dataDir = core.getDataDir();
 
     // make a copy to avoid potential deadlock of a callback calling newInstance and trying to
@@ -698,22 +716,21 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
         waitingForCore.clear();
       }
 
-      for( SolrCoreAware aware : arr) {
-        aware.inform( core );
+      for (SolrCoreAware aware : arr) {
+        aware.inform(core);
       }
     }
 
     // this is the last method to be called in SolrCore before the latch is released.
     live = true;
   }
-  
+
   /**
    * Tell all {@link ResourceLoaderAware} instances about the loader
    */
-  public void inform( ResourceLoader loader ) throws IOException
-  {
+  public void inform(ResourceLoader loader) throws IOException {
 
-     // make a copy to avoid potential deadlock of a callback adding to the list
+    // make a copy to avoid potential deadlock of a callback adding to the list
     ResourceLoaderAware[] arr;
 
     while (waitingForResources.size() > 0) {
@@ -722,7 +739,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
         waitingForResources.clear();
       }
 
-      for( ResourceLoaderAware aware : arr) {
+      for (ResourceLoaderAware aware : arr) {
         aware.inform(loader);
       }
     }
@@ -730,6 +747,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
 
   /**
    * Register any {@link SolrInfoBean}s
+   *
    * @param infoRegistry The Info Registry
    */
   public void inform(Map<String, SolrInfoBean> infoRegistry) {
@@ -755,7 +773,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
       }
     }
   }
-  
+
   /**
    * Determines the solrhome from the environment.
    * Tries JNDI (java:comp/env/solr/home) then system property (solr.solr.home);
@@ -765,12 +783,13 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   /**
    * Finds the solrhome based on looking up the value in one of three places:
    * <ol>
-   *  <li>JNDI: via java:comp/env/solr/home</li>
-   *  <li>The system property solr.solr.home</li>
-   *  <li>Look in the current working directory for a solr/ directory</li> 
+   * <li>JNDI: via java:comp/env/solr/home</li>
+   * <li>The system property solr.solr.home</li>
+   * <li>Look in the current working directory for a solr/ directory</li>
    * </ol>
-   *
+   * <p>
    * The return value is normalized.  Normalization essentially means it ends in a trailing slash.
+   *
    * @return A normalized solrhome
    * @see #normalizeDir(String)
    */
@@ -780,27 +799,27 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
     // Try JNDI
     try {
       Context c = new InitialContext();
-      home = (String)c.lookup("java:comp/env/"+project+"/home");
-      logOnceInfo("home_using_jndi", "Using JNDI solr.home: "+home );
+      home = (String) c.lookup("java:comp/env/" + project + "/home");
+      logOnceInfo("home_using_jndi", "Using JNDI solr.home: " + home);
     } catch (NoInitialContextException e) {
-      log.debug("JNDI not configured for "+project+" (NoInitialContextEx)");
+      log.debug("JNDI not configured for " + project + " (NoInitialContextEx)");
     } catch (NamingException e) {
-      log.debug("No /"+project+"/home in JNDI");
-    } catch( RuntimeException ex ) {
+      log.debug("No /" + project + "/home in JNDI");
+    } catch (RuntimeException ex) {
       log.warn("Odd RuntimeException while testing for JNDI: " + ex.getMessage());
-    } 
-    
+    }
+
     // Now try system property
-    if( home == null ) {
+    if (home == null) {
       String prop = project + ".solr.home";
       home = System.getProperty(prop);
-      if( home != null ) {
-        logOnceInfo("home_using_sysprop", "Using system property "+prop+": " + home );
+      if (home != null) {
+        logOnceInfo("home_using_sysprop", "Using system property " + prop + ": " + home);
       }
     }
-    
+
     // if all else fails, try 
-    if( home == null ) {
+    if (home == null) {
       home = project + '/';
       logOnceInfo("home_default", project + " home defaulted to '" + home + "' (could not find system property or JNDI)");
     }
@@ -809,22 +828,23 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
 
   /**
    * Solr allows users to store arbitrary files in a special directory located directly under SOLR_HOME.
-   *
+   * <p>
    * This directory is generally created by each node on startup.  Files located in this directory can then be
    * manipulated using select Solr features (e.g. streaming expressions).
    */
   public static final String USER_FILES_DIRECTORY = "userfiles";
+
   public static void ensureUserFilesDataDir(Path solrHome) {
     final Path userFilesPath = getUserFilesPath(solrHome);
     final File userFilesDirectory = new File(userFilesPath.toString());
-    if (! userFilesDirectory.exists()) {
+    if (!userFilesDirectory.exists()) {
       try {
         final boolean created = userFilesDirectory.mkdir();
-        if (! created) {
+        if (!created) {
           log.warn("Unable to create [{}] directory in SOLR_HOME [{}].  Features requiring this directory may fail.", USER_FILES_DIRECTORY, solrHome);
         }
       } catch (Exception e) {
-          log.warn("Unable to create [" + USER_FILES_DIRECTORY + "] directory in SOLR_HOME [" + solrHome + "].  Features requiring this directory may fail.", e);
+        log.warn("Unable to create [" + USER_FILES_DIRECTORY + "] directory in SOLR_HOME [" + solrHome + "].  Features requiring this directory may fail.", e);
       }
     }
   }
@@ -847,72 +867,73 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
   public Path getInstancePath() {
     return instanceDir;
   }
-  
+
   /**
    * Keep a list of classes that are allowed to implement each 'Aware' interface
    */
   private static final Map<Class, Class[]> awareCompatibility;
+
   static {
     awareCompatibility = new HashMap<>();
-    awareCompatibility.put( 
-      SolrCoreAware.class, new Class[] {
-        // DO NOT ADD THINGS TO THIS LIST -- ESPECIALLY THINGS THAT CAN BE CREATED DYNAMICALLY
-        // VIA RUNTIME APIS -- UNTILL CAREFULLY CONSIDERING THE ISSUES MENTIONED IN SOLR-8311
-        CodecFactory.class,
-        DirectoryFactory.class,
-        ManagedIndexSchemaFactory.class,
-        QueryResponseWriter.class,
-        SearchComponent.class,
-        ShardHandlerFactory.class,
-        SimilarityFactory.class,
-        SolrRequestHandler.class,
-        UpdateRequestProcessorFactory.class
-      }
+    awareCompatibility.put(
+        SolrCoreAware.class, new Class[]{
+            // DO NOT ADD THINGS TO THIS LIST -- ESPECIALLY THINGS THAT CAN BE CREATED DYNAMICALLY
+            // VIA RUNTIME APIS -- UNTILL CAREFULLY CONSIDERING THE ISSUES MENTIONED IN SOLR-8311
+            CodecFactory.class,
+            DirectoryFactory.class,
+            ManagedIndexSchemaFactory.class,
+            QueryResponseWriter.class,
+            SearchComponent.class,
+            ShardHandlerFactory.class,
+            SimilarityFactory.class,
+            SolrRequestHandler.class,
+            UpdateRequestProcessorFactory.class
+        }
     );
 
     awareCompatibility.put(
-      ResourceLoaderAware.class, new Class[] {
-        // DO NOT ADD THINGS TO THIS LIST -- ESPECIALLY THINGS THAT CAN BE CREATED DYNAMICALLY
-        // VIA RUNTIME APIS -- UNTILL CAREFULLY CONSIDERING THE ISSUES MENTIONED IN SOLR-8311
-        CharFilterFactory.class,
-        TokenFilterFactory.class,
-        TokenizerFactory.class,
-        QParserPlugin.class,
-        FieldType.class
-      }
+        ResourceLoaderAware.class, new Class[]{
+            // DO NOT ADD THINGS TO THIS LIST -- ESPECIALLY THINGS THAT CAN BE CREATED DYNAMICALLY
+            // VIA RUNTIME APIS -- UNTILL CAREFULLY CONSIDERING THE ISSUES MENTIONED IN SOLR-8311
+            CharFilterFactory.class,
+            TokenFilterFactory.class,
+            TokenizerFactory.class,
+            QParserPlugin.class,
+            FieldType.class
+        }
     );
   }
 
   /**
    * Utility function to throw an exception if the class is invalid
    */
-  static void assertAwareCompatibility( Class aware, Object obj )
-  {
-    Class[] valid = awareCompatibility.get( aware );
-    if( valid == null ) {
-      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-          "Unknown Aware interface: "+aware );
-    }
-    for( Class v : valid ) {
-      if( v.isInstance( obj ) ) {
+  static void assertAwareCompatibility(Class aware, Object obj) {
+    Class[] valid = awareCompatibility.get(aware);
+    if (valid == null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unknown Aware interface: " + aware);
+    }
+    for (Class v : valid) {
+      if (v.isInstance(obj)) {
         return;
       }
     }
     StringBuilder builder = new StringBuilder();
-    builder.append( "Invalid 'Aware' object: " ).append( obj );
-    builder.append( " -- ").append( aware.getName() );
-    builder.append(  " must be an instance of: " );
-    for( Class v : valid ) {
-      builder.append( "[" ).append( v.getName() ).append( "] ") ;
+    builder.append("Invalid 'Aware' object: ").append(obj);
+    builder.append(" -- ").append(aware.getName());
+    builder.append(" must be an instance of: ");
+    for (Class v : valid) {
+      builder.append("[").append(v.getName()).append("] ");
     }
-    throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, builder.toString() );
+    throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, builder.toString());
   }
 
   @Override
   public void close() throws IOException {
     IOUtils.close(classLoader);
   }
-  public List<SolrInfoBean> getInfoMBeans(){
+
+  public List<SolrInfoBean> getInfoMBeans() {
     return Collections.unmodifiableList(infoMBeans);
   }
 
@@ -922,8 +943,8 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
     File confFile = new File(loader.getConfigDir(), resourceName);
     try {
       File parentDir = confFile.getParentFile();
-      if ( ! parentDir.isDirectory()) {
-        if ( ! parentDir.mkdirs()) {
+      if (!parentDir.isDirectory()) {
+        if (!parentDir.mkdirs()) {
           final String msg = "Can't create managed schema directory " + parentDir.getAbsolutePath();
           log.error(msg);
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
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 11c6404..2085221 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -68,6 +68,7 @@ import org.apache.solr.core.RequestParams;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.pkg.PackageListeners;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
@@ -245,8 +246,22 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
             if (componentName != null) {
               Map map = (Map) val.get(parts.get(1));
               if (map != null) {
-                val.put(parts.get(1), makeMap(componentName, map.get(componentName)));
+                Object o = map.get(componentName);
+                val.put(parts.get(1), makeMap(componentName, o));
+                if(req.getParams().getBool("meta", false)){
+                  for (PackageListeners.Listener listener :
+                      req.getCore().getPackageListeners().getListeners()) {
+                    PluginInfo info = listener.pluginInfo();
+                    if(info.type.equals(parts.get(1)) && info.name.equals(componentName)){
+                      if (o instanceof Map) {
+                        Map m1 = (Map) o;
+                        m1.put("_packageinfo_", listener.getPackageVersion());
+                      }
+                    }
+                  }
+                }
               }
+
             }
 
             resp.add("config", val);
@@ -488,6 +503,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
       }
       List errs = CommandOperation.captureErrors(ops);
       if (!errs.isEmpty()) {
+        log.error("ERRROR:" +Utils.toJSONString(errs));
         throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST,"error processing commands", errs);
       }
 
@@ -495,7 +511,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
       if (loader instanceof ZkSolrResourceLoader) {
         int latestVersion = ZkController.persistConfigResourceToZooKeeper((ZkSolrResourceLoader) loader, overlay.getZnodeVersion(),
             ConfigOverlay.RESOURCE_NAME, overlay.toByteArray(), true);
-        log.info("Executed config commands successfully and persisted to ZK {}", ops);
+        log.debug("Executed config commands successfully and persisted to ZK {}", ops);
         waitForAllReplicasState(req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName(),
             req.getCore().getCoreContainer().getZkController(),
             ConfigOverlay.NAME,
@@ -503,7 +519,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
       } else {
         SolrResourceLoader.persistConfLocally(loader, ConfigOverlay.RESOURCE_NAME, overlay.toByteArray());
         req.getCore().getCoreContainer().reload(req.getCore().getName());
-        log.info("Executed config commands successfully and persited to File System {}", ops);
+        log.debug("Executed config commands successfully and persited to File System {}", ops);
       }
 
     }
@@ -570,6 +586,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
         try {
           req.getCore().createInitInstance(new PluginInfo(SolrRequestHandler.TYPE, op.getDataMap()), expected, clz, "");
         } catch (Exception e) {
+          log.error("Error checking plugin : ",e);
           op.addError(e.getMessage());
           return false;
         }
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java b/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
new file mode 100644
index 0000000..0267f37
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
@@ -0,0 +1,367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.pkg;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.solr.api.Command;
+import org.apache.solr.api.EndPoint;
+import org.apache.solr.api.PayloadObj;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.filestore.PackageStoreAPI;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.ReflectMapWriter;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.SOLR_PKGS_PATH;
+import static org.apache.solr.security.PermissionNameProvider.Name.PACKAGE_EDIT_PERM;
+import static org.apache.solr.security.PermissionNameProvider.Name.PACKAGE_READ_PERM;
+
+public class PackageAPI {
+  public static final String PACKAGES = "packages";
+  public final boolean enablePackages = Boolean.parseBoolean(System.getProperty("enable.packages", "false"));
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  final CoreContainer coreContainer;
+  private ObjectMapper mapper = new ObjectMapper();
+  private final PackageLoader packageLoader;
+  Packages pkgs;
+
+  public final Edit editAPI = new Edit();
+  public final Read readAPI = new Read();
+
+  public PackageAPI(CoreContainer coreContainer, PackageLoader loader) {
+    this.coreContainer = coreContainer;
+    this.packageLoader = loader;
+    pkgs = new Packages();
+    SolrZkClient zkClient = coreContainer.getZkController().getZkClient();
+    try {
+      registerListener(zkClient);
+    } catch (KeeperException | InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private void registerListener(SolrZkClient zkClient)
+      throws KeeperException, InterruptedException {
+    String path = SOLR_PKGS_PATH;
+    zkClient.exists(path,
+        new Watcher() {
+
+          @Override
+          public void process(WatchedEvent event) {
+            // session events are not change events, and do not remove the watcher
+            if (Event.EventType.None.equals(event.getType())) {
+              return;
+            }
+            try {
+              synchronized (this) {
+                log.debug("Updating [{}] ... ", path);
+
+                // remake watch
+                final Watcher thisWatch = this;
+                final Stat stat = new Stat();
+                final byte[] data = zkClient.getData(path, thisWatch, stat, true);
+                pkgs = readPkgsFromZk(data, stat);
+                packageLoader.refreshPackageConf();
+              }
+            } catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
+              log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
+            } catch (KeeperException e) {
+              log.error("A ZK error has occurred", e);
+              throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
+            } catch (InterruptedException e) {
+              // Restore the interrupted status
+              Thread.currentThread().interrupt();
+              log.warn("Interrupted", e);
+            }
+          }
+
+        }, true);
+  }
+
+
+  private Packages readPkgsFromZk(byte[] data, Stat stat) throws KeeperException, InterruptedException {
+
+    if (data == null || stat == null) {
+      stat = new Stat();
+      data = coreContainer.getZkController().getZkClient()
+          .getData(ZkStateReader.CLUSTER_PROPS, null, stat, true);
+
+    }
+    Packages packages = null;
+    if (data == null || data.length == 0) {
+      packages = new Packages();
+    } else {
+      try {
+        packages = mapper.readValue(data, Packages.class);
+        packages.znodeVersion = stat.getVersion();
+      } catch (IOException e) {
+        //invalid data in packages
+        //TODO handle properly;
+        return new Packages();
+      }
+    }
+    return packages;
+  }
+
+
+  public static class Packages implements ReflectMapWriter {
+    @JsonProperty
+    public int znodeVersion = -1;
+
+    @JsonProperty
+    public Map<String, List<PkgVersion>> packages = new LinkedHashMap<>();
+
+
+    public Packages copy() {
+      Packages p = new Packages();
+      p.znodeVersion = this.znodeVersion;
+      p.packages = new LinkedHashMap<>();
+      packages.forEach((s, versions) ->
+          p.packages.put(s, new ArrayList<>(versions)));
+      return p;
+    }
+  }
+
+  public static class PkgVersion implements ReflectMapWriter {
+
+    @JsonProperty
+    public String version;
+
+    @JsonProperty
+    public List<String> files;
+
+    public PkgVersion() {
+    }
+
+    public PkgVersion(AddVersion addVersion) {
+      this.version = addVersion.version;
+      this.files = addVersion.files;
+    }
+
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof PkgVersion) {
+        PkgVersion that = (PkgVersion) obj;
+        return Objects.equals(this.version, that.version)
+            && Objects.equals(this.files, that.files);
+
+      }
+      return false;
+    }
+  }
+
+
+  @EndPoint(method = SolrRequest.METHOD.POST,
+      path = "/cluster/package",
+      permission = PACKAGE_EDIT_PERM)
+  public class Edit {
+
+    @Command(name = "add")
+    public void add(SolrQueryRequest req, SolrQueryResponse rsp, PayloadObj<AddVersion> payload) {
+      if (!checkEnabled(payload)) return;
+      AddVersion add = payload.get();
+      if (add.files.isEmpty()) {
+        payload.addError("No files specified");
+        return;
+      }
+      PackageStoreAPI packageStoreAPI = coreContainer.getPackageStoreAPI();
+      packageStoreAPI.validateFiles(add.files, true, s -> payload.addError(s));
+      if (payload.hasError()) return;
+      Packages[] finalState = new Packages[1];
+      try {
+        coreContainer.getZkController().getZkClient().atomicUpdate(SOLR_PKGS_PATH, (stat, bytes) -> {
+          Packages packages = null;
+          try {
+            packages = bytes == null ? new Packages() : mapper.readValue(bytes, Packages.class);
+            packages = packages.copy();
+          } catch (IOException e) {
+            log.error("Error deserializing packages.json", e);
+            packages = new Packages();
+          }
+          packages.packages.computeIfAbsent(add.pkg, Utils.NEW_ARRAYLIST_FUN).add(new PkgVersion(add));
+          packages.znodeVersion = stat.getVersion() + 1;
+          finalState[0] = packages;
+          return Utils.toJSON(packages);
+        });
+      } catch (KeeperException | InterruptedException e) {
+        finalState[0] = null;
+        handleZkErr(e);
+      }
+      if (finalState[0] != null) {
+//        succeeded in updating
+        pkgs = finalState[0];
+        notifyAllNodesToSync(pkgs.znodeVersion);
+        packageLoader.refreshPackageConf();
+      }
+
+    }
+
+    @Command(name = "delete")
+    public void del(SolrQueryRequest req, SolrQueryResponse rsp, PayloadObj<DelVersion> payload) {
+      if (!checkEnabled(payload)) return;
+      DelVersion delVersion = payload.get();
+      try {
+        coreContainer.getZkController().getZkClient().atomicUpdate(SOLR_PKGS_PATH, (stat, bytes) -> {
+          Packages packages = null;
+          try {
+            packages = mapper.readValue(bytes, Packages.class);
+            packages = packages.copy();
+          } catch (IOException e) {
+            packages = new Packages();
+          }
+
+          List<PkgVersion> versions = packages.packages.get(delVersion.pkg);
+          if (versions == null || versions.isEmpty()) {
+            payload.addError("No such package: " + delVersion.pkg);
+            return null;// no change
+          }
+          int idxToremove = -1;
+          for (int i = 0; i < versions.size(); i++) {
+            if (Objects.equals(versions.get(i).version, delVersion.version)) {
+              idxToremove = i;
+              break;
+            }
+          }
+          if (idxToremove == -1) {
+            payload.addError("No such version: " + delVersion.version);
+            return null;
+          }
+          versions.remove(idxToremove);
+          packages.znodeVersion = stat.getVersion() + 1;
+          return Utils.toJSON(packages);
+        });
+      } catch (KeeperException | InterruptedException e) {
+        handleZkErr(e);
+
+      }
+
+
+    }
+
+  }
+
+  private boolean checkEnabled(CommandOperation payload) {
+    if (!enablePackages) {
+      payload.addError("Package loading is not enabled , Start your nodes with -Denable.packages=true");
+      return false;
+    }
+    return true;
+  }
+
+  @EndPoint(
+      method = SolrRequest.METHOD.GET,
+      path = {"/cluster/package/",
+          "/cluster/package/{name}"},
+      permission = PACKAGE_READ_PERM
+  )
+  public class Read {
+    @Command()
+    public void get(SolrQueryRequest req, SolrQueryResponse rsp) {
+      int expectedVersion = req.getParams().getInt("expectedVersion", -1);
+      if (expectedVersion != -1) {
+        syncToVersion(expectedVersion);
+      }
+      String name = req.getPathTemplateValues().get("name");
+      if (name == null) {
+        rsp.add("result", pkgs);
+      } else {
+        rsp.add("result", Collections.singletonMap(name, pkgs.packages.get(name)));
+      }
+    }
+
+    private void syncToVersion(int expectedVersion) {
+      for (int i = 0; i < 10; i++) {
+        log.debug("my version is {} , and expected version {}", pkgs.znodeVersion, expectedVersion);
+        if (pkgs.znodeVersion >= expectedVersion) {
+          return;
+        }
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException e) {
+        }
+        try {
+          pkgs = readPkgsFromZk(null, null);
+        } catch (KeeperException | InterruptedException e) {
+          handleZkErr(e);
+
+        }
+
+      }
+
+    }
+
+
+  }
+
+  public static class AddVersion implements ReflectMapWriter {
+    @JsonProperty(value = "package", required = true)
+    public String pkg;
+    @JsonProperty(required = true)
+    public String version;
+    @JsonProperty(required = true)
+    public List<String> files;
+
+  }
+
+  public static class DelVersion implements ReflectMapWriter {
+    @JsonProperty(value = "package", required = true)
+    public String pkg;
+    @JsonProperty(required = true)
+    public String version;
+
+  }
+
+  void notifyAllNodesToSync(int expected) {
+    for (String s : coreContainer.getPackageStoreAPI().shuffledNodes()) {
+      Utils.executeGET(coreContainer.getUpdateShardHandler().getDefaultHttpClient(),
+          coreContainer.getZkController().zkStateReader.getBaseUrlForNodeName(s).replace("/solr", "/api") + "/cluster/package?wt=javabin&omitHeader=true&expectedVersion" + expected,
+          Utils.JAVABINCONSUMER);
+    }
+  }
+
+  public void handleZkErr(Exception e) {
+    log.error("Error reading package config from zookeeper", SolrZkClient.checkInterrupted(e));
+  }
+
+
+}
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java b/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
new file mode 100644
index 0000000..c6ebae5
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.pkg;
+
+import java.lang.ref.WeakReference;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.solr.core.PluginInfo;
+
+public class PackageListeners {
+  // this registry only keeps a weak reference because it does not want to
+  // cause a memory leak if the listener forgets to unregister itself
+  private List<WeakReference<Listener>> listeners = new ArrayList<>();
+
+  public synchronized void addListener(Listener listener) {
+    listeners.add(new WeakReference<>(listener));
+
+  }
+
+  public synchronized void removeListener(Listener listener) {
+    Iterator<WeakReference<Listener>> it = listeners.iterator();
+    while (it.hasNext()) {
+      WeakReference<Listener> ref = it.next();
+      Listener pkgListener = ref.get();
+      if(pkgListener == null || pkgListener == listener){
+        it.remove();
+      }
+
+    }
+
+  }
+
+  synchronized void packagesUpdated(List<PackageLoader.Package> pkgs){
+    for (PackageLoader.Package pkgInfo : pkgs) {
+      invokeListeners(pkgInfo);
+    }
+  }
+
+  private synchronized void invokeListeners(PackageLoader.Package pkg) {
+    for (WeakReference<Listener> ref : listeners) {
+      Listener listener = ref.get();
+      if (listener != null && listener.packageName().equals(pkg.name())) {
+        listener.changed(pkg);
+      }
+    }
+  }
+
+  public List<Listener> getListeners(){
+    List<Listener> result = new ArrayList<>();
+    for (WeakReference<Listener> ref : listeners) {
+      Listener l = ref.get();
+      if(l != null){
+        result.add(l);
+      }
+
+    }
+    return result;
+  }
+
+
+
+  public interface Listener {
+    String packageName();
+
+    PluginInfo pluginInfo();
+
+    void changed(PackageLoader.Package pkg);
+
+    PackageLoader.Package.Version getPackageVersion();
+
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java b/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
new file mode 100644
index 0000000..7efcc88
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.pkg;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.MalformedURLException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrResourceLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PackageLoader {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final CoreContainer coreContainer;
+  private final Map<String, Package> packageClassLoaders = new ConcurrentHashMap<>();
+
+  private PackageAPI.Packages myCopy;
+
+  private PackageAPI packageAPI;
+
+
+  public PackageLoader(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+    packageAPI = new PackageAPI(coreContainer, this);
+    myCopy = packageAPI.pkgs;
+
+  }
+
+  public PackageAPI getPackageAPI() {
+    return packageAPI;
+  }
+
+  public Package getPackage(String key) {
+    return packageClassLoaders.get(key);
+  }
+
+  public Map<String, Package> getPackages() {
+    return Collections.EMPTY_MAP;
+  }
+
+  public void refreshPackageConf() {
+    log.info("{} updated to version {}", ZkStateReader.SOLR_PKGS_PATH, packageAPI.pkgs.znodeVersion);
+
+    List<Package> updated = new ArrayList<>();
+    Map<String, List<PackageAPI.PkgVersion>> modified = getModified(myCopy, packageAPI.pkgs);
+    for (Map.Entry<String, List<PackageAPI.PkgVersion>> e : modified.entrySet()) {
+      if (e.getValue() != null) {
+        Package p = packageClassLoaders.get(e.getKey());
+        if (e.getValue() != null && p == null) {
+          packageClassLoaders.put(e.getKey(), p = new Package(e.getKey()));
+        }
+        p.updateVersions(e.getValue());
+        updated.add(p);
+      } else {
+        Package p = packageClassLoaders.remove(e.getKey());
+        if (p != null) {
+          //other classes are holding to a reference to this objecec
+          // they should know that this is removed
+          p.markDeleted();
+        }
+      }
+    }
+    for (SolrCore core : coreContainer.getCores()) {
+      core.getPackageListeners().packagesUpdated(updated);
+
+    }
+  }
+
+  public Map<String, List<PackageAPI.PkgVersion>> getModified(PackageAPI.Packages old, PackageAPI.Packages newPkgs) {
+    Map<String, List<PackageAPI.PkgVersion>> changed = new HashMap<>();
+    for (Map.Entry<String, List<PackageAPI.PkgVersion>> e : newPkgs.packages.entrySet()) {
+      List<PackageAPI.PkgVersion> versions = old.packages.get(e.getKey());
+      if (versions != null) {
+        if (!Objects.equals(e.getValue(), versions)) {
+          changed.put(e.getKey(), e.getValue());
+        }
+      } else {
+        changed.put(e.getKey(), e.getValue());
+      }
+    }
+    //some packages are deleted altogether
+    for (String s : old.packages.keySet()) {
+      if (!newPkgs.packages.keySet().contains(s)) {
+        changed.put(s, null);
+      }
+    }
+
+    return changed;
+
+  }
+
+  public SolrResourceLoader getResourceLoader(String pkg, String version) {
+    return null;
+  }
+
+
+  public class Package {
+    final String name;
+    final Map<String, Version> myVersions = new ConcurrentHashMap<>();
+    private List<String> sortedVersions = new CopyOnWriteArrayList<>();
+    String latest;
+    private boolean deleted;
+
+
+    public Package(String name) {
+      this.name = name;
+    }
+
+    public boolean isDeleted() {
+      return deleted;
+    }
+
+
+    private synchronized void updateVersions(List<PackageAPI.PkgVersion> modified) {
+      for (PackageAPI.PkgVersion v : modified) {
+        Version version = myVersions.get(v.version);
+        if (version == null) {
+          myVersions.put(v.version, new Version(this, v));
+          sortedVersions.add(v.version);
+        }
+      }
+
+      Set<String> newVersions = new HashSet<>();
+      for (PackageAPI.PkgVersion v : modified) {
+        newVersions.add(v.version);
+      }
+      for (String s : new HashSet<>(myVersions.keySet())) {
+        if (!newVersions.contains(s)) {
+          sortedVersions.remove(s);
+          myVersions.remove(s);
+        }
+      }
+
+      sortedVersions.sort(String::compareTo);
+      if (sortedVersions.size() > 0) {
+        latest = sortedVersions.get(sortedVersions.size() - 1);
+      } else {
+        latest = null;
+      }
+
+    }
+
+
+    public Version getLatest() {
+      return latest == null ? null : myVersions.get(latest);
+    }
+
+    public Version getLatest(String lessThan) {
+      String latest = null;
+      for (String v : (ArrayList<String>) new ArrayList(sortedVersions)) {
+        if (v.compareTo(lessThan) < 1) {
+          latest = v;
+        } else break;
+
+      }
+
+
+      return latest == null ? null : myVersions.get(latest);
+    }
+
+    public String name() {
+      return name;
+    }
+
+    private void markDeleted() {
+      deleted = true;
+      myVersions.clear();
+      sortedVersions.clear();
+      latest = null;
+
+    }
+
+    public class Version implements MapWriter {
+      private final Package parent;
+      private SolrResourceLoader loader;
+
+      private final PackageAPI.PkgVersion version;
+
+      @Override
+      public void writeMap(EntryWriter ew) throws IOException {
+        ew.put("package", parent.name());
+        version.writeMap(ew);
+      }
+
+      public Version(Package parent, PackageAPI.PkgVersion v) {
+        this.parent = parent;
+        this.version = v;
+        List<Path> paths = new ArrayList<>();
+        for (String file : version.files) {
+          paths.add(coreContainer.getPackageStoreAPI().getPackageStore().getRealpath(file));
+        }
+
+        try {
+          loader = new SolrResourceLoader(
+              "PACKAGE_LOADER:"+ parent.name()+ ":"+ version,
+              paths,
+              coreContainer.getResourceLoader().getInstancePath(),
+              coreContainer.getResourceLoader().getClassLoader());
+        } catch (MalformedURLException e) {
+          log.error("Could not load classloader ", e);
+        }
+      }
+
+      public String getVersion() {
+        return version.version;
+      }
+
+      public Collection getFiles() {
+        return Collections.unmodifiableList(version.files);
+      }
+
+      public SolrResourceLoader getLoader() {
+        return loader;
+
+      }
+
+    }
+  }
+
+
+}
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackagePluginHolder.java b/solr/core/src/java/org/apache/solr/pkg/PackagePluginHolder.java
new file mode 100644
index 0000000..f0364c4
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/pkg/PackagePluginHolder.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.pkg;
+
+import java.lang.invoke.MethodHandles;
+
+import org.apache.solr.core.PluginBag;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrCore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PackagePluginHolder<T> extends PluginBag.PluginHolder<T> {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final SolrCore core;
+  private final SolrConfig.SolrPluginInfo pluginMeta;
+  private PackageLoader.Package aPackage;
+  private PackageLoader.Package.Version pkgVersion;
+
+
+  public PackagePluginHolder(PluginInfo info, SolrCore core, SolrConfig.SolrPluginInfo pluginMeta) {
+    super(info);
+    this.core = core;
+    this.pluginMeta = pluginMeta;
+
+    reload(aPackage = core.getCoreContainer().getPackageLoader().getPackage(info.pkgName));
+    core.getPackageListeners().addListener(new PackageListeners.Listener() {
+      @Override
+      public String packageName() {
+        return info.pkgName;
+      }
+
+      @Override
+      public PluginInfo pluginInfo() {
+        return info;
+      }
+
+      @Override
+      public void changed(PackageLoader.Package pkg) {
+        reload(pkg);
+
+      }
+
+      @Override
+      public PackageLoader.Package.Version getPackageVersion() {
+        return pkgVersion;
+      }
+
+    });
+  }
+
+
+  private synchronized void reload(PackageLoader.Package pkg) {
+    if(pkgVersion != null && aPackage.getLatest() == pkgVersion ) return;
+
+    if (inst != null) log.info("reloading plugin {} ", pluginInfo.name);
+    PackageLoader.Package.Version newest = pkg.getLatest();
+    if(newest == null) return;
+    Object instance = SolrCore.createInstance(pluginInfo.className,
+        pluginMeta.clazz, pluginMeta.getCleanTag(), core, newest.getLoader());
+    PluginBag.initInstance(instance, pluginInfo);
+    T old = inst;
+    inst = (T) instance;
+    pkgVersion = newest;
+    if (old instanceof AutoCloseable) {
+      AutoCloseable closeable = (AutoCloseable) old;
+      try {
+        closeable.close();
+      } catch (Exception e) {
+        log.error("error closing plugin", e);
+      }
+    }
+  }
+
+}
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index a4c7c0d..b5d4099 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -53,6 +53,9 @@ public interface PermissionNameProvider {
     METRICS_HISTORY_READ_PERM("metrics-history-read", null),
     FILESTORE_READ_PERM("filestore-read", null),
     FILESTORE_WRITE_PERM("filestore-write", null),
+    PACKAGE_EDIT_PERM("package-edit", null),
+    PACKAGE_READ_PERM("package-read", null),
+
     ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
     ;
     final String name;
diff --git a/solr/core/src/test/org/apache/solr/pkg/TestPackages.java b/solr/core/src/test/org/apache/solr/pkg/TestPackages.java
new file mode 100644
index 0000000..5de69cf
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/pkg/TestPackages.java
@@ -0,0 +1,391 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.pkg;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.cloud.ConfigRequest;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.MapWriterMap;
+import org.apache.solr.common.NavigableObject;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.filestore.TestDistribPackageStore;
+import org.apache.solr.util.LogLevel;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.Stat;
+import org.junit.Test;
+
+import static org.apache.solr.common.cloud.ZkStateReader.SOLR_PKGS_PATH;
+import static org.apache.solr.common.params.CommonParams.JAVABIN;
+import static org.apache.solr.common.params.CommonParams.WT;
+import static org.apache.solr.core.TestDynamicLoading.getFileContent;
+import static org.apache.solr.filestore.TestDistribPackageStore.readFile;
+
+@LogLevel("org.apache.solr.pkg.PackageLoader=DEBUG;org.apache.solr.pkg.PackageAPI=DEBUG")
+public class TestPackages extends SolrCloudTestCase {
+
+  @Test
+  public void testPluginLoading() throws Exception {
+    System.setProperty("enable.packages", "true");
+    MiniSolrCloudCluster cluster =
+        configureCluster(4)
+            .withJettyConfig(jetty -> jetty.enableV2(true))
+            .addConfig("conf", configset("cloud-minimal"))
+            .configure();
+    try {
+      String FILE1 = "/mypkg/runtimelibs.jar";
+      String FILE2 = "/mypkg/runtimelibs_v2.jar";
+      String COLLECTION_NAME = "testPluginLoadingColl";
+      byte[] derFile = readFile("cryptokeys/pub_key512.der");
+      cluster.getZkClient().makePath("/keys/exe", true);
+      cluster.getZkClient().create("/keys/exe/pub_key512.der", derFile, CreateMode.PERSISTENT, true);
+      postFileAndWait(cluster, "runtimecode/runtimelibs.jar.bin", FILE1,
+          "L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1f/U3bOlMPINlSOM6LK3JpQ==");
+
+      PackageAPI.AddVersion add = new PackageAPI.AddVersion();
+      add.version = "1.0";
+      add.pkg = "mypkg";
+      add.files = Arrays.asList(new String[]{FILE1});
+      V2Request req = new V2Request.Builder("/cluster/package")
+          .forceV2(true)
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload(Collections.singletonMap("add", add))
+          .build();
+
+      req.process(cluster.getSolrClient());
+
+
+      CollectionAdminRequest
+          .createCollection(COLLECTION_NAME, "conf", 2, 2)
+          .setMaxShardsPerNode(100)
+          .process(cluster.getSolrClient());
+      cluster.waitForActiveCollection(COLLECTION_NAME, 2, 4);
+
+      TestDistribPackageStore.assertResponseValues(10,
+          () -> new V2Request.Builder("/cluster/package").
+              withMethod(SolrRequest.METHOD.GET)
+              .build().process(cluster.getSolrClient()),
+          Utils.makeMap(
+              ":result:packages:mypkg[0]:version", "1.0",
+              ":result:packages:mypkg[0]:files[0]", FILE1
+          ));
+
+      String payload = "{\n" +
+          "'create-requesthandler' : { 'name' : '/runtime', 'class': 'mypkg:org.apache.solr.core.RuntimeLibReqHandler' }," +
+          "'create-searchcomponent' : { 'name' : 'get', 'class': 'mypkg:org.apache.solr.core.RuntimeLibSearchComponent'  }," +
+          "'create-queryResponseWriter' : { 'name' : 'json1', 'class': 'mypkg:org.apache.solr.core.RuntimeLibResponseWriter' }" +
+          "}";
+      cluster.getSolrClient().request(new ConfigRequest(payload) {
+        @Override
+        public String getCollection() {
+          return COLLECTION_NAME;
+        }
+      });
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "queryResponseWriter", "json1",
+          "mypkg", "1.0" );
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "searchComponent", "get",
+          "mypkg", "1.0" );
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "requestHandler", "/runtime",
+          "mypkg", "1.0" );
+
+
+
+      executeReq( "/" + COLLECTION_NAME + "/runtime?wt=javabin", cluster.getRandomJetty(random()),
+          Utils.JAVABINCONSUMER,
+          Utils.makeMap("class", "org.apache.solr.core.RuntimeLibReqHandler"));
+
+      executeReq( "/" + COLLECTION_NAME + "/get?wt=json", cluster.getRandomJetty(random()),
+          Utils.JSONCONSUMER,
+          Utils.makeMap("class", "org.apache.solr.core.RuntimeLibSearchComponent",
+              "Version","1"));
+
+
+      executeReq( "/" + COLLECTION_NAME + "/runtime?wt=json1", cluster.getRandomJetty(random()),
+          Utils.JSONCONSUMER,
+          Utils.makeMap("wt", "org.apache.solr.core.RuntimeLibResponseWriter"));
+
+      //now upload the second jar
+      postFileAndWait(cluster, "runtimecode/runtimelibs_v2.jar.bin", FILE2,
+          "j+Rflxi64tXdqosIhbusqi6GTwZq8znunC/dzwcWW0/dHlFGKDurOaE1Nz9FSPJuXbHkVLj638yZ0Lp1ssnoYA==");
+
+      //add the version using package API
+      add.version = "1.1";
+      add.pkg = "mypkg";
+      add.files = Arrays.asList(new String[]{FILE2});
+      req.process(cluster.getSolrClient());
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "queryResponseWriter", "json1",
+          "mypkg", "1.1" );
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "searchComponent", "get",
+          "mypkg", "1.1" );
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "requestHandler", "/runtime",
+          "mypkg", "1.1" );
+
+      /*executeReq( "/" + COLLECTION_NAME + "/get?wt=json", cluster.getRandomJetty(random()),
+          Utils.JSONCONSUMER,
+          Utils.makeMap("class", "org.apache.solr.core.RuntimeLibSearchComponent",
+              "Version","2"));
+
+      PackageAPI.DelVersion delVersion = new PackageAPI.DelVersion();
+      delVersion.pkg = "mypkg";
+      delVersion.version = "1.1";
+      new V2Request.Builder("/cluster/package")
+          .withMethod(SolrRequest.METHOD.POST)
+          .forceV2(true)
+          .withPayload(delVersion)
+          .build()
+          .process(cluster.getSolrClient());
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "queryResponseWriter", "json1",
+          "mypkg", "1.0" );
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "searchComponent", "get",
+          "mypkg", "1.0" );
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "requestHandler", "/runtime",
+          "mypkg", "1.0" );
+
+*/
+    } finally {
+      cluster.shutdown();
+    }
+
+  }
+
+  private void executeReq(String uri, JettySolrRunner jetty, Utils.InputStreamConsumer parser, Map expected) throws Exception {
+    try(HttpSolrClient client = (HttpSolrClient) jetty.newClient()){
+      TestDistribPackageStore.assertResponseValues(10,
+          () -> {
+            Object o = Utils.executeGET(client.getHttpClient(),
+                jetty.getBaseUrl() + uri, parser);
+            if(o instanceof NavigableObject) return (NavigableObject) o;
+            if(o instanceof Map) return new MapWriterMap((Map) o);
+            throw new RuntimeException("Unknown response");
+          }, expected);
+
+    }
+  }
+
+  private void verifyCmponent(SolrClient client, String COLLECTION_NAME,
+  String componentType, String componentName, String pkg, String version) throws Exception {
+    SolrParams params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
+        WT, JAVABIN,
+        "componentName", componentName,
+        "meta", "true"));
+
+    String s = "queryResponseWriter";
+    GenericSolrRequest req1 = new GenericSolrRequest(SolrRequest.METHOD.GET,
+        "/config/" + componentType, params);
+    TestDistribPackageStore.assertResponseValues(10,
+        client,
+        req1, Utils.makeMap(
+            ":config:" + componentType + ":" + componentName + ":_packageinfo_:package", pkg,
+            ":config:" + componentType + ":" + componentName + ":_packageinfo_:version", version
+        ));
+  }
+
+  @Test
+  public void testAPI() throws Exception {
+    System.setProperty("enable.packages", "true");
+    MiniSolrCloudCluster cluster =
+        configureCluster(4)
+            .withJettyConfig(jetty -> jetty.enableV2(true))
+            .addConfig("conf", configset("cloud-minimal"))
+            .configure();
+    try {
+      String errPath = "/error/details[0]/errorMessages[0]";
+      String FILE1 = "/mypkg/v.0.12/jar_a.jar";
+      String FILE2 = "/mypkg/v.0.12/jar_b.jar";
+      String FILE3 = "/mypkg/v.0.13/jar_a.jar";
+
+      PackageAPI.AddVersion add = new PackageAPI.AddVersion();
+      add.version = "0.12";
+      add.pkg = "test_pkg";
+      add.files = Arrays.asList(new String[]{FILE1, FILE2});
+      V2Request req = new V2Request.Builder("/cluster/package")
+          .forceV2(true)
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload(Collections.singletonMap("add", add))
+          .build();
+
+      //the files is not yet there. The command should fail with error saying "No such file"
+      expectError(req, cluster.getSolrClient(), errPath, "No such file :");
+
+
+      //post the jar file. No signature is sent
+      postFileAndWait(cluster, "runtimecode/runtimelibs.jar.bin", FILE1, null);
+
+
+      add.files = Arrays.asList(new String[]{FILE1});
+      expectError(req, cluster.getSolrClient(), errPath,
+          FILE1 + " has no signature");
+      //now we upload the keys
+      byte[] derFile = readFile("cryptokeys/pub_key512.der");
+      cluster.getZkClient().makePath("/keys/exe", true);
+      cluster.getZkClient().create("/keys/exe/pub_key512.der", derFile, CreateMode.PERSISTENT, true);
+      //and upload the same file with a different name but it has proper signature
+      postFileAndWait(cluster, "runtimecode/runtimelibs.jar.bin", FILE2,
+          "L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1f/U3bOlMPINlSOM6LK3JpQ==");
+      // with correct signature
+      //after uploading the file, let's delete the keys to see if we get proper error message
+      cluster.getZkClient().delete("/keys/exe/pub_key512.der", -1, true);
+      add.files = Arrays.asList(new String[]{FILE2});
+      expectError(req, cluster.getSolrClient(), errPath,
+          "ZooKeeper does not have any public keys");
+
+      //Now lets' put the keys back
+      cluster.getZkClient().create("/keys/exe/pub_key512.der", derFile, CreateMode.PERSISTENT, true);
+
+      //this time we have a file with proper signature, public keys are in ZK
+      // so the add {} command should succeed
+      req.process(cluster.getSolrClient());
+
+      //Now verify the data in ZK
+      TestDistribPackageStore.assertResponseValues(1,
+          () -> new MapWriterMap((Map) Utils.fromJSON(cluster.getZkClient().getData(SOLR_PKGS_PATH,
+              null, new Stat(), true))),
+          Utils.makeMap(
+              ":packages:test_pkg[0]:version", "0.12",
+              ":packages:test_pkg[0]:files[0]", FILE1
+          ));
+
+      //post a new jar with a proper signature
+      postFileAndWait(cluster, "runtimecode/runtimelibs_v2.jar.bin", FILE3,
+          "j+Rflxi64tXdqosIhbusqi6GTwZq8znunC/dzwcWW0/dHlFGKDurOaE1Nz9FSPJuXbHkVLj638yZ0Lp1ssnoYA==");
+
+
+      //this time we are adding the second version of the package (0.13)
+      add.version = "0.13";
+      add.pkg = "test_pkg";
+      add.files = Arrays.asList(new String[]{FILE3});
+
+      //this request should succeed
+      req.process(cluster.getSolrClient());
+      //no verify the data (/packages.json) in ZK
+      TestDistribPackageStore.assertResponseValues(1,
+          () -> new MapWriterMap((Map) Utils.fromJSON(cluster.getZkClient().getData(SOLR_PKGS_PATH,
+              null, new Stat(), true))),
+          Utils.makeMap(
+              ":packages:test_pkg[1]:version", "0.13",
+              ":packages:test_pkg[1]:files[0]", FILE3
+          ));
+
+      //Now we will just delete one version
+      PackageAPI.DelVersion delVersion = new PackageAPI.DelVersion();
+      delVersion.version = "0.1";//this version does not exist
+      delVersion.pkg = "test_pkg";
+      req = new V2Request.Builder("/cluster/package")
+          .forceV2(true)
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload(Collections.singletonMap("delete", delVersion))
+          .build();
+
+      //we are expecting an error
+      expectError(req, cluster.getSolrClient(), errPath, "No such version:");
+
+      delVersion.version = "0.12";//correct version. Should succeed
+      req.process(cluster.getSolrClient());
+      //Verify with ZK that the data is correcy
+      TestDistribPackageStore.assertResponseValues(1,
+          () -> new MapWriterMap((Map) Utils.fromJSON(cluster.getZkClient().getData(SOLR_PKGS_PATH,
+              null, new Stat(), true))),
+          Utils.makeMap(
+              ":packages:test_pkg[0]:version", "0.13",
+              ":packages:test_pkg[0]:files[0]", FILE2
+          ));
+
+
+      //So far we have been verifying the details with  ZK directly
+      //use the package read API to verify with each node that it has the correct data
+      for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+        String path = jetty.getBaseUrl().toString().replace("/solr", "/api") + "/cluster/package?wt=javabin";
+        TestDistribPackageStore.assertResponseValues(10, new Callable<NavigableObject>() {
+          @Override
+          public NavigableObject call() throws Exception {
+            try (HttpSolrClient solrClient = (HttpSolrClient) jetty.newClient()) {
+              return (NavigableObject) Utils.executeGET(solrClient.getHttpClient(), path, Utils.JAVABINCONSUMER);
+            }
+          }
+        }, Utils.makeMap(
+            ":result:packages:test_pkg[0]:version", "0.13",
+            ":result:packages:test_pkg[0]:files[0]", FILE3
+        ));
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  static void postFileAndWait(MiniSolrCloudCluster cluster, String fname, String path, String sig) throws Exception {
+    ByteBuffer fileContent = getFileContent(fname);
+    String sha512 = DigestUtils.sha512Hex(fileContent.array());
+
+    TestDistribPackageStore.postFile(cluster.getSolrClient(),
+        fileContent,
+        path, sig);// has file, but no signature
+
+    TestDistribPackageStore.waitForAllNodesHaveFile(cluster, path, Utils.makeMap(
+        ":files:" + path + ":sha512",
+        sha512
+    ), false);
+  }
+
+  private void expectError(V2Request req, SolrClient client, String errPath, String expectErrorMsg) throws IOException, SolrServerException {
+    try {
+      req.process(client);
+      fail("should have failed with message : " + expectErrorMsg);
+    } catch (BaseHttpSolrClient.RemoteExecutionException e) {
+      String msg = e.getMetaData()._getStr(errPath, "");
+      assertTrue("should have failed with message: " + expectErrorMsg + "actual message : " + msg,
+          msg.contains(expectErrorMsg)
+      );
+    }
+  }
+}
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index 98f0d9a..b8105ab 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -34,6 +34,7 @@ import java.nio.file.Path;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.regex.Pattern;
@@ -353,6 +354,10 @@ public class SolrZkClient implements Closeable {
   }
 
   public void atomicUpdate(String path, Function<byte[], byte[]> editor) throws KeeperException, InterruptedException {
+   atomicUpdate(path, (stat, bytes) -> editor.apply(bytes));
+  }
+
+  public void atomicUpdate(String path, BiFunction<Stat , byte[], byte[]> editor) throws KeeperException, InterruptedException {
     for (; ; ) {
       byte[] modified = null;
       byte[] zkData = null;
@@ -360,7 +365,7 @@ public class SolrZkClient implements Closeable {
       try {
         if (exists(path, true)) {
           zkData = getData(path, null, s, true);
-          modified = editor.apply(zkData);
+          modified = editor.apply(s, zkData);
           if (modified == null) {
             //no change , no need to persist
             return;
@@ -368,7 +373,7 @@ public class SolrZkClient implements Closeable {
           setData(path, modified, s.getVersion(), true);
           break;
         } else {
-          modified = editor.apply(null);
+          modified = editor.apply(s,null);
           if (modified == null) {
             //no change , no need to persist
             return;
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index dcf7d9e..5dea5b0 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -118,6 +118,7 @@ public class ZkStateReader implements SolrCloseable {
   public static final String SOLR_AUTOSCALING_TRIGGER_STATE_PATH = "/autoscaling/triggerState";
   public static final String SOLR_AUTOSCALING_NODE_ADDED_PATH = "/autoscaling/nodeAdded";
   public static final String SOLR_AUTOSCALING_NODE_LOST_PATH = "/autoscaling/nodeLost";
+  public static final String SOLR_PKGS_PATH = "/packages.json";
 
   public static final String DEFAULT_SHARD_PREFERENCES = "defaultShardPreferences";
   public static final String REPLICATION_FACTOR = "replicationFactor";