You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2020/06/30 10:34:12 UTC

[lucene-solr] branch branch_8x updated: SOLR-14599: Package manager support for cluster level plugins

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 1517665  SOLR-14599: Package manager support for cluster level plugins
1517665 is described below

commit 15176657e29ccfffc7bcd5c1b93db3b5ed6b96aa
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Tue Jun 30 15:50:40 2020 +0530

    SOLR-14599: Package manager support for cluster level plugins
    
        Usage:
            bin/solr package deploy <packagename> -y -cluster
            bin/solr package deploy <packagename>:<new-version> -y -cluster --update
            bin/solr package undeploy <packagename> -y -cluster
---
 solr/CHANGES.txt                                   |   2 +
 .../apache/solr/packagemanager/PackageManager.java | 303 ++++++++++++++++++---
 .../apache/solr/packagemanager/PackageUtils.java   |   3 +
 .../solr/packagemanager/RepositoryManager.java     |  12 +-
 .../apache/solr/packagemanager/SolrPackage.java    |   4 +
 .../solr/packagemanager/SolrPackageInstance.java   |  15 +
 .../src/java/org/apache/solr/util/PackageTool.java |  11 +-
 solr/core/src/test-files/runtimecode/MyPlugin.java |   2 +-
 8 files changed, 301 insertions(+), 51 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2a7acc6..bda3d3c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -34,6 +34,8 @@ New Features
 * SOLR-14481: Add drill Streaming Expression for efficient and accurate high cardinality aggregation.
   (Joel Bernstein)
 
+* SOLR-14599: Package manager support for cluster level plugins (see SOLR-14404) (Ishan Chattopadhyaya)
+
 Improvements
 ---------------------
 * SOLR-14316: Remove unchecked type conversion warning in JavaBinCodec's readMapEntry's equals() method
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
index 95003d0..715e6a4 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
@@ -33,12 +33,15 @@ import java.util.Scanner;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.commons.collections.map.MultiValueMap;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
 import org.apache.solr.common.NavigableObject;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.packagemanager.SolrPackage.Command;
 import org.apache.solr.packagemanager.SolrPackage.Manifest;
@@ -134,6 +137,42 @@ public class PackageManager implements Closeable {
     return ret;
   }
 
+  /**
+   * Get a list of packages that have their plugins deployed as cluster level plugins.
+   * The returned packages also contain the "pluginMeta" from "clusterprops.json" as custom data. 
+   */
+  @SuppressWarnings("unchecked")
+  public Map<String, SolrPackageInstance> getPackagesDeployedAsClusterLevelPlugins() {
+    Map<String, String> packageVersions = new HashMap<String, String>();
+    MultiValueMap packagePlugins = new MultiValueMap(); // map of package name to multiple values of pluginMeta (Map<String, String>)
+    Map<String, Object> result = (Map<String, Object>) Utils.executeGET(solrClient.getHttpClient(),
+        solrBaseUrl + PackageUtils.CLUSTERPROPS_PATH, Utils.JSONCONSUMER);
+    Map<String, Object> clusterPlugins = (Map<String, Object>) result.getOrDefault("plugin", Collections.emptyMap());
+    for (String key: clusterPlugins.keySet()) {
+      // Map<String, String> pluginMeta = (Map<String, String>) clusterPlugins.get(key);
+      PluginMeta pluginMeta;
+      try {
+        pluginMeta = PackageUtils.getMapper().readValue(Utils.toJSON(clusterPlugins.get(key)), PluginMeta.class);
+      } catch (IOException e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Exception while fetching plugins from /clusterprops.json in ZK.", e);
+      }
+      if (pluginMeta.klass.contains(":")) {
+        String packageName = pluginMeta.klass.substring(0, pluginMeta.klass.indexOf(':'));
+        packageVersions.put(packageName, pluginMeta.version);
+        packagePlugins.put(packageName, pluginMeta);
+      }
+    }
+    Map<String, SolrPackageInstance> ret = new HashMap<>();
+    for (String packageName: packageVersions.keySet()) {
+      if (Strings.isNullOrEmpty(packageName) == false && // There can be an empty key, storing the version here
+          packageVersions.get(packageName) != null) { // null means the package was undeployed from this package before
+        ret.put(packageName, getPackageInstance(packageName, packageVersions.get(packageName)));
+        ret.get(packageName).setCustomData(packagePlugins.getCollection(packageName));
+      }
+    }
+    return ret;
+  }
+
   private void ensureCollectionsExist(List<String> collections) {
     try {
       List<String> existingCollections = zkClient.getChildren("/collections", null, true);
@@ -147,11 +186,37 @@ public class PackageManager implements Closeable {
     }
   }
   
-  @SuppressWarnings({"unchecked"})
   private boolean deployPackage(SolrPackageInstance packageInstance, boolean pegToLatest, boolean isUpdate, boolean noprompt,
-      List<String> collections, String[] overrides) {
-    List<String> previouslyDeployed =  new ArrayList<>(); // collections where package is already deployed in
+      List<String> collections, boolean shouldDeployClusterPlugins, String[] overrides) {
+
+    // Install plugins of type "cluster"
+    boolean clusterSuccess = deployClusterPackage(packageInstance, isUpdate, noprompt, shouldDeployClusterPlugins,
+        overrides);
+
+    
+    // Install plugins of type "collection"
+    Pair<List<String>, List<String>> deployResult = deployCollectionPackage(packageInstance, pegToLatest, isUpdate, noprompt, collections,
+        shouldDeployClusterPlugins, overrides);
+    List<String> deployedCollections = deployResult.first();
+    List<String> previouslyDeployedOnCollections = deployResult.second();
+    
+    // Verify
+    boolean verifySuccess = true;
+    // Verify that package was successfully deployed
+    verifySuccess = verify(packageInstance, deployedCollections, shouldDeployClusterPlugins, overrides);
+    if (verifySuccess) {
+      PackageUtils.printGreen("Deployed on " + deployedCollections + " and verified package: " + packageInstance.name + ", version: " + packageInstance.version);
+    }
 
+    return clusterSuccess && previouslyDeployedOnCollections.isEmpty() && verifySuccess;
+  }
+
+  /**
+   * @return list of collections on which packages deployed on
+   */
+  private Pair<List<String>, List<String>> deployCollectionPackage(SolrPackageInstance packageInstance, boolean pegToLatest, boolean isUpdate,
+      boolean noprompt, List<String> collections, boolean shouldDeployClusterPlugins, String[] overrides) {
+    List<String> previouslyDeployed =  new ArrayList<>(); // collections where package is already deployed in
     for (String collection: collections) {
       SolrPackageInstance deployedPackage = getPackagesDeployed(collection).get(packageInstance.name);
       if (packageInstance.equals(deployedPackage)) {
@@ -172,7 +237,8 @@ public class PackageManager implements Closeable {
 
       // Get package params
       try {
-        boolean packageParamsExist = ((Map)PackageUtils.getJson(solrClient.getHttpClient(), solrBaseUrl + PackageUtils.getCollectionParamsPath(collection) + "/packages", Map.class)
+        @SuppressWarnings("unchecked")
+        boolean packageParamsExist = ((Map<Object, Object>)PackageUtils.getJson(solrClient.getHttpClient(), solrBaseUrl + PackageUtils.getCollectionParamsPath(collection) + "/packages", Map.class)
             .getOrDefault("response", Collections.emptyMap())).containsKey("params");
         SolrCLI.postJsonToSolr(solrClient, PackageUtils.getCollectionParamsPath(collection),
             getMapper().writeValueAsString(Collections.singletonMap(packageParamsExist? "update": "set",
@@ -200,9 +266,10 @@ public class PackageManager implements Closeable {
 
       // If it is a fresh deploy on a collection, run setup commands all the plugins in the package
       if (!isUpdate) {
-        Map<String, String> systemParams = PackageUtils.map("collection", collection, "package-name", packageInstance.name, "package-version", packageInstance.version);
-
         for (Plugin plugin: packageInstance.plugins) {
+          if ("collection".equalsIgnoreCase(plugin.type) == false || collections.isEmpty()) continue;
+          Map<String, String> systemParams = PackageUtils.map("collection", collection, "package-name", packageInstance.name, "package-version", packageInstance.version, "plugin-name", plugin.name);
+
           Command cmd = plugin.setupCommand;
           if (cmd != null && !Strings.isNullOrEmpty(cmd.method)) {
             if ("POST".equalsIgnoreCase(cmd.method)) {
@@ -213,11 +280,13 @@ public class PackageManager implements Closeable {
                 boolean shouldExecute = true;
                 if (!noprompt) { // show a prompt asking user to execute the setup command for the plugin
                   PackageUtils.print(PackageUtils.YELLOW, "Execute this command (y/n): ");
-                  String userInput = new Scanner(System.in, "UTF-8").next();
-                  if (!"yes".equalsIgnoreCase(userInput) && !"y".equalsIgnoreCase(userInput)) {
-                    shouldExecute = false;
-                    PackageUtils.printRed("Skipping setup command for deploying (deployment verification may fail)."
-                        + " Please run this step manually or refer to package documentation.");
+                  try (Scanner scanner = new Scanner(System.in, "UTF-8")) {
+                    String userInput = scanner.next();
+                    if (!"yes".equalsIgnoreCase(userInput) && !"y".equalsIgnoreCase(userInput)) {
+                      shouldExecute = false;
+                      PackageUtils.printRed("Skipping setup command for deploying (deployment verification may fail)."
+                          + " Please run this step manually or refer to package documentation.");
+                    }
                   }
                 }
                 if (shouldExecute) {
@@ -244,24 +313,114 @@ public class PackageManager implements Closeable {
       }
     }
 
+    if (previouslyDeployed.isEmpty() == false) {
+      PackageUtils.printRed("Already Deployed on " + previouslyDeployed + ", package: " + packageInstance.name + ", version: " + packageInstance.version);
+    }
+
     List<String> deployedCollections = collections.stream().filter(c -> !previouslyDeployed.contains(c)).collect(Collectors.toList());
+    return new Pair<List<String>, List<String>>(deployedCollections, previouslyDeployed);
+  }
 
-    boolean success = true;
-    if (deployedCollections.isEmpty() == false) {
-      // Verify that package was successfully deployed
-      success = verify(packageInstance, deployedCollections);
-      if (success) {
-        PackageUtils.printGreen("Deployed on " + deployedCollections + " and verified package: " + packageInstance.name + ", version: " + packageInstance.version);
+  @SuppressWarnings("unchecked")
+  private boolean deployClusterPackage(SolrPackageInstance packageInstance, boolean isUpdate, boolean noprompt,
+      boolean shouldDeployClusterPlugins, String[] overrides) {
+    boolean cluasterPluginFailed = false;
+
+    if (isUpdate) {
+      for (Plugin plugin: packageInstance.plugins) {
+        if (!shouldDeployClusterPlugins || "cluster".equalsIgnoreCase(plugin.type) == false) continue;
+        SolrPackageInstance deployedPackage = getPackagesDeployedAsClusterLevelPlugins().get(packageInstance.name);
+        if (deployedPackage == null) {
+          PackageUtils.printRed("Cluster level plugin " + plugin.name + " from package " + packageInstance.name + " not deployed. To deploy, remove the --update parameter.");
+          cluasterPluginFailed = true;
+          continue;
+        }
+        for (PluginMeta pluginMeta: (List<PluginMeta>)deployedPackage.getCustomData()) {
+          PackageUtils.printGreen("Updating this plugin: " + pluginMeta);
+          try {
+            pluginMeta.version = packageInstance.version; // just update the version, let the other metadata same
+            String postBody = "{\"update\": " + Utils.toJSONString(pluginMeta) + "}";
+            PackageUtils.printGreen("Posting " + postBody + " to " + PackageUtils.CLUSTER_PLUGINS_PATH);
+            SolrCLI.postJsonToSolr(solrClient, PackageUtils.CLUSTER_PLUGINS_PATH, postBody);
+          } catch (Exception e) {
+            throw new SolrException(ErrorCode.SERVER_ERROR, e);
+          }
+        }
+      }
+    } else {
+      for (Plugin plugin: packageInstance.plugins) {
+        if (!shouldDeployClusterPlugins || "cluster".equalsIgnoreCase(plugin.type) == false) continue;
+        // Check if this cluster level plugin is already deployed
+        {
+          Map<String, Object> clusterprops = null;
+          try {
+            clusterprops = PackageUtils.getJson(solrClient.getHttpClient(), solrBaseUrl + PackageUtils.CLUSTERPROPS_PATH, Map.class);
+          } catch (SolrException ex) {
+            if (ex.code() == ErrorCode.NOT_FOUND.code) {
+              // Ignore this, as clusterprops may not have been created yet. This means package isn't already installed.
+            } else throw ex;
+          }
+          if (clusterprops != null) {
+            Object pkg = ((Map<String, Object>)clusterprops.getOrDefault("plugin", Collections.emptyMap())).get(packageInstance.name+":"+plugin.name);
+            if (pkg != null) {
+              PackageUtils.printRed("Cluster level plugin " + plugin.name + " from package " + packageInstance.name + " already deployed. To update to " + packageInstance + ", pass --update parameter.");
+              cluasterPluginFailed = true;
+              continue;
+            }
+          }
+        }
+
+        // Lets install this plugin now
+        Map<String, String> systemParams = PackageUtils.map("package-name", packageInstance.name, "package-version", packageInstance.version, "plugin-name", plugin.name);
+        Command cmd = plugin.setupCommand;
+        if (cmd != null && !Strings.isNullOrEmpty(cmd.method)) {
+          if ("POST".equalsIgnoreCase(cmd.method)) {
+            try {
+              Map<String, String> overridesMap = getParameterOverrides(overrides);
+              String payload = PackageUtils.resolve(getMapper().writeValueAsString(cmd.payload), packageInstance.parameterDefaults, overridesMap, systemParams);
+              String path = PackageUtils.resolve(cmd.path, packageInstance.parameterDefaults, overridesMap, systemParams);
+              PackageUtils.printGreen("Executing " + payload + " for path:" + path);
+              boolean shouldExecute = true;
+              if (!noprompt) { // show a prompt asking user to execute the setup command for the plugin
+                PackageUtils.print(PackageUtils.YELLOW, "Execute this command (y/n): ");
+                try (Scanner scanner = new Scanner(System.in, "UTF-8")) {
+                  String userInput = scanner.next();
+                  if (!"yes".equalsIgnoreCase(userInput) && !"y".equalsIgnoreCase(userInput)) {
+                    shouldExecute = false;
+                    PackageUtils.printRed("Skipping setup command for deploying (deployment verification may fail)."
+                        + " Please run this step manually or refer to package documentation.");
+                  }
+                }
+              }
+              if (shouldExecute) {
+                SolrCLI.postJsonToSolr(solrClient, path, payload);
+              }
+            } catch (Exception ex) {
+              throw new SolrException(ErrorCode.SERVER_ERROR, ex);
+            }
+          } else {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Non-POST method not supported for setup commands");
+          }
+        } else {
+          PackageUtils.printRed("There is no setup command to execute for plugin: " + plugin.name);
+        }
       }
     }
-    if (previouslyDeployed.isEmpty() == false) {
-      PackageUtils.printRed("Already Deployed on " + previouslyDeployed + ", package: " + packageInstance.name + ", version: " + packageInstance.version);
-    }
-    return previouslyDeployed.isEmpty() && success;
+    return !cluasterPluginFailed;
   }
 
-  private Map<String,String> getCollectionParameterOverrides(SolrPackageInstance packageInstance, boolean isUpdate,
-      String[] overrides, String collection) {
+
+  /**
+   * Parse a map of overrides based on user provided values in format "key1=val1"
+   */
+  private Map<String,String> getParameterOverrides(String[] overrides) {
+    return getCollectionParameterOverrides(null, false, overrides, null);
+  }
+  
+  /**
+   * Resolve parameter overrides by overlaying provided overrides with collection level overrides already in a deployed package.
+   */
+  private Map<String,String> getCollectionParameterOverrides(SolrPackageInstance packageInstance, boolean isUpdate, String[] overrides, String collection) {
     Map<String, String> collectionParameterOverrides = isUpdate? getPackageParams(packageInstance.name, collection): new HashMap<String,String>();
     if (overrides != null) {
       for (String override: overrides) {
@@ -288,32 +447,58 @@ public class PackageManager implements Closeable {
   /**
    * Given a package and list of collections, verify if the package is installed
    * in those collections. It uses the verify command of every plugin in the package (if defined).
+   * 
+   * @param overrides are needed only when shouldDeployClusterPlugins is true, since collection level plugins will get their overrides from ZK (collection params API)
    */
-  public boolean verify(SolrPackageInstance pkg, List<String> collections) {
+  public boolean verify(SolrPackageInstance pkg, List<String> collections, boolean shouldDeployClusterPlugins, String overrides[]) {
     boolean success = true;
     for (Plugin plugin: pkg.plugins) {
       Command cmd = plugin.verifyCommand;
       if (plugin.verifyCommand != null && !Strings.isNullOrEmpty(cmd.path)) {
-        for (String collection: collections) {
-          Map<String, String> collectionParameterOverrides = getPackageParams(pkg.name, collection);
-
-          Map<String, String> systemParams = PackageUtils.map("collection", collection, "package-name", pkg.name, "package-version", pkg.version);
-          String url = solrBaseUrl + PackageUtils.resolve(cmd.path, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
-          PackageUtils.printGreen("Executing " + url + " for collection:" + collection);
+        if ("cluster".equalsIgnoreCase(plugin.type)) {
+          if (!shouldDeployClusterPlugins) continue; // Plugins of type "cluster"
+          Map<String, String> overridesMap = getParameterOverrides(overrides);
+          Map<String, String> systemParams = PackageUtils.map("package-name", pkg.name, "package-version", pkg.version, "plugin-name", plugin.name);
+          String url = solrBaseUrl + PackageUtils.resolve(cmd.path, pkg.parameterDefaults, overridesMap, systemParams);
+          PackageUtils.printGreen("Executing " + url + " for cluster level plugin");
 
           if ("GET".equalsIgnoreCase(cmd.method)) {
             String response = PackageUtils.getJsonStringFromUrl(solrClient.getHttpClient(), url);
             PackageUtils.printGreen(response);
             String actualValue = JsonPath.parse(response, PackageUtils.jsonPathConfiguration())
-                .read(PackageUtils.resolve(cmd.condition, pkg.parameterDefaults, collectionParameterOverrides, systemParams));
-            String expectedValue = PackageUtils.resolve(cmd.expected, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
-            PackageUtils.printGreen("Actual: "+actualValue+", expected: "+expectedValue);
+                .read(PackageUtils.resolve(cmd.condition, pkg.parameterDefaults, overridesMap, systemParams));
+            String expectedValue = PackageUtils.resolve(cmd.expected, pkg.parameterDefaults, overridesMap, systemParams);
+            PackageUtils.printGreen("Actual: " + actualValue+", expected: " + expectedValue);
             if (!expectedValue.equals(actualValue)) {
               PackageUtils.printRed("Failed to deploy plugin: " + plugin.name);
               success = false;
             }
           } else {
-            throw new SolrException(ErrorCode.BAD_REQUEST, "Non-GET method not supported for verify commands");
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Non-GET method not supported for setup commands");
+          }          
+        } else {
+          // Plugins of type "collection"
+          for (String collection: collections) {
+            Map<String, String> collectionParameterOverrides = getPackageParams(pkg.name, collection);
+  
+            Map<String, String> systemParams = PackageUtils.map("collection", collection, "package-name", pkg.name, "package-version", pkg.version, "plugin-name", plugin.name);
+            String url = solrBaseUrl + PackageUtils.resolve(cmd.path, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
+            PackageUtils.printGreen("Executing " + url + " for collection:" + collection);
+  
+            if ("GET".equalsIgnoreCase(cmd.method)) {
+              String response = PackageUtils.getJsonStringFromUrl(solrClient.getHttpClient(), url);
+              PackageUtils.printGreen(response);
+              String actualValue = JsonPath.parse(response, PackageUtils.jsonPathConfiguration())
+                  .read(PackageUtils.resolve(cmd.condition, pkg.parameterDefaults, collectionParameterOverrides, systemParams));
+              String expectedValue = PackageUtils.resolve(cmd.expected, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
+              PackageUtils.printGreen("Actual: "+actualValue+", expected: "+expectedValue);
+              if (!expectedValue.equals(actualValue)) {
+                PackageUtils.printRed("Failed to deploy plugin: " + plugin.name);
+                success = false;
+              }
+            } else {
+              throw new SolrException(ErrorCode.BAD_REQUEST, "Non-GET method not supported for setup commands");
+            }
           }
         }
       }
@@ -353,35 +538,68 @@ public class PackageManager implements Closeable {
    * @param isUpdate Is this a fresh deployment or is it an update (i.e. there is already a version of this package deployed on this collection)
    * @param noprompt If true, don't prompt before executing setup commands.
    */
-  public void deploy(String packageName, String version, String[] collections, String[] parameters,
+  public void deploy(String packageName, String version, String[] collections, boolean shouldInstallClusterPlugins, String[] parameters,
       boolean isUpdate, boolean noprompt) throws SolrException {
     ensureCollectionsExist(Arrays.asList(collections));
 
     boolean pegToLatest = PackageUtils.LATEST.equals(version); // User wants to peg this package's version to the latest installed (for auto-update, i.e. no explicit deploy step)
     SolrPackageInstance packageInstance = getPackageInstance(packageName, version);
     if (packageInstance == null) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, "Package instance doesn't exist: " + packageName + ":" + null +
-          ". Use install command to install this version first.");
+      PackageUtils.printRed("Package instance doesn't exist: " + packageName + ":" + version + ". Use install command to install this version first.");
+      System.exit(1);
     }
     if (version == null) version = packageInstance.version;
 
     Manifest manifest = packageInstance.manifest;
     if (PackageUtils.checkVersionConstraint(RepositoryManager.systemVersion, manifest.versionConstraint) == false) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, "Version incompatible! Solr version: "
+      PackageUtils.printRed("Version incompatible! Solr version: "
           + RepositoryManager.systemVersion + ", package version constraint: " + manifest.versionConstraint);
+      System.exit(1);
     }
 
     boolean res = deployPackage(packageInstance, pegToLatest, isUpdate, noprompt,
-        Arrays.asList(collections), parameters);
+        Arrays.asList(collections), shouldInstallClusterPlugins, parameters);
     PackageUtils.print(res? PackageUtils.GREEN: PackageUtils.RED, res? "Deployment successful": "Deployment failed");
   }
 
   /**
    * Undeploys a package from given collections.
    */
-  public void undeploy(String packageName, String[] collections) throws SolrException {
+  public void undeploy(String packageName, String[] collections, boolean shouldUndeployClusterPlugins) throws SolrException {
     ensureCollectionsExist(Arrays.asList(collections));
     
+    // Undeploy cluster level plugins
+    if (shouldUndeployClusterPlugins) {
+      SolrPackageInstance deployedPackage = getPackagesDeployedAsClusterLevelPlugins().get(packageName);
+      if (deployedPackage == null) {
+        PackageUtils.printRed("Cluster level plugins from package "+packageName+" not deployed.");
+      } else {
+        for (Plugin plugin: deployedPackage.plugins) {
+          if (!shouldUndeployClusterPlugins || "cluster".equalsIgnoreCase(plugin.type) == false) continue;
+            
+          Map<String, String> systemParams = PackageUtils.map("package-name", deployedPackage.name, "package-version", deployedPackage.version, "plugin-name", plugin.name);
+          Command cmd = plugin.uninstallCommand;
+          if (cmd != null && !Strings.isNullOrEmpty(cmd.method)) {
+            if ("POST".equalsIgnoreCase(cmd.method)) {
+              try {
+                String payload = PackageUtils.resolve(getMapper().writeValueAsString(cmd.payload), deployedPackage.parameterDefaults, Collections.emptyMap(), systemParams);
+                String path = PackageUtils.resolve(cmd.path, deployedPackage.parameterDefaults, Collections.emptyMap(), systemParams);
+                PackageUtils.printGreen("Executing " + payload + " for path:" + path);
+                SolrCLI.postJsonToSolr(solrClient, path, payload);
+              } catch (Exception ex) {
+                throw new SolrException(ErrorCode.SERVER_ERROR, ex);
+              }
+            } else {
+              throw new SolrException(ErrorCode.BAD_REQUEST, "Non-POST method not supported for uninstall commands");
+            }
+          } else {
+            PackageUtils.printRed("There is no uninstall command to execute for plugin: " + plugin.name);
+          }
+
+        }
+      }
+    }
+    // Undeploy collection level plugins
     for (String collection: collections) {
       SolrPackageInstance deployedPackage = getPackagesDeployed(collection).get(packageName);
       if (deployedPackage == null) {
@@ -391,9 +609,10 @@ public class PackageManager implements Closeable {
       Map<String, String> collectionParameterOverrides = getPackageParams(packageName, collection);
 
       // Run the uninstall command for all plugins
-      Map<String, String> systemParams = PackageUtils.map("collection", collection, "package-name", deployedPackage.name, "package-version", deployedPackage.version);
-
       for (Plugin plugin: deployedPackage.plugins) {
+        if ("collection".equalsIgnoreCase(plugin.type) == false) continue;
+
+        Map<String, String> systemParams = PackageUtils.map("collection", collection, "package-name", deployedPackage.name, "package-version", deployedPackage.version, "plugin-name", plugin.name);
         Command cmd = plugin.uninstallCommand;
         if (cmd != null && !Strings.isNullOrEmpty(cmd.method)) {
           if ("POST".equalsIgnoreCase(cmd.method)) {
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/PackageUtils.java b/solr/core/src/java/org/apache/solr/packagemanager/PackageUtils.java
index 516a0c4..29192ff 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/PackageUtils.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/PackageUtils.java
@@ -65,7 +65,10 @@ public class PackageUtils {
   public static String LATEST = "latest";
   
   public static String PACKAGE_PATH = "/api/cluster/package";
+  public static String CLUSTER_PLUGINS_PATH = "/api/cluster/plugin";
   public static String REPOSITORIES_ZK_PATH = "/repositories.json";
+  public static String CLUSTERPROPS_PATH = "/api/cluster/zk/data/clusterprops.json";
+  
  
   public static Configuration jsonPathConfiguration() {
     MappingProvider provider = new JacksonMappingProvider();
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/RepositoryManager.java b/solr/core/src/java/org/apache/solr/packagemanager/RepositoryManager.java
index aa5c7b4..beabc7a 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/RepositoryManager.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/RepositoryManager.java
@@ -306,10 +306,10 @@ public class RepositoryManager {
     String latestVersion = getLastPackageRelease(packageName).version;
 
     Map<String, String> collectionsDeployedIn = packageManager.getDeployedCollections(packageName);
-    List<String> peggedToLatest = collectionsDeployedIn.keySet().stream().
+    List<String> collectionsPeggedToLatest = collectionsDeployedIn.keySet().stream().
         filter(collection -> collectionsDeployedIn.get(collection).equals(PackagePluginHolder.LATEST)).collect(Collectors.toList());
-    if (!peggedToLatest.isEmpty()) {
-      PackageUtils.printGreen("Collections that will be affected (since they are configured to use $LATEST): "+peggedToLatest);
+    if (!collectionsPeggedToLatest.isEmpty()) {
+      PackageUtils.printGreen("Collections that will be affected (since they are configured to use $LATEST): "+collectionsPeggedToLatest);
     }
 
     if (version == null || version.equals(PackageUtils.LATEST)) {
@@ -318,11 +318,11 @@ public class RepositoryManager {
       installPackage(packageName, version);
     }
 
-    if (peggedToLatest.isEmpty() == false) {
+    if (collectionsPeggedToLatest.isEmpty() == false) {
       SolrPackageInstance updatedPackage = packageManager.getPackageInstance(packageName, PackageUtils.LATEST);
-      boolean res = packageManager.verify(updatedPackage, peggedToLatest);
+      boolean res = packageManager.verify(updatedPackage, collectionsPeggedToLatest, false, new String[] {}); // Cluster level plugins don't work with peggedToLatest functionality
       PackageUtils.printGreen("Verifying version " + updatedPackage.version + 
-          " on " + peggedToLatest + ", result: " + res);
+          " on " + collectionsPeggedToLatest + ", result: " + res);
       if (!res) throw new SolrException(ErrorCode.BAD_REQUEST, "Failed verification after deployment");
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackage.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackage.java
index eaa4334..77429c6 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackage.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackage.java
@@ -87,6 +87,10 @@ public class SolrPackage implements Comparable<SolrPackage>, ReflectMapWriter {
 
   public static class Plugin implements ReflectMapWriter {
     public String name;
+    
+    @JsonProperty("type")
+    public String type = "collection"; // if not specified, assume collection level plugin (backward compatability)
+    
     @JsonProperty("setup-command")
     public Command setupCommand;
 
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageInstance.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageInstance.java
index 48f8af5..dcfa670 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageInstance.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageInstance.java
@@ -26,6 +26,8 @@ import org.apache.solr.common.util.ReflectMapWriter;
 import org.apache.solr.packagemanager.SolrPackage.Manifest;
 import org.apache.solr.packagemanager.SolrPackage.Plugin;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
 /**
  * Describes one instance of a package as it exists in Solr when installed.
  */
@@ -44,6 +46,19 @@ public class SolrPackageInstance implements ReflectMapWriter {
 
   final public Map<String, String> parameterDefaults;
 
+  @JsonIgnore
+  private Object customData;
+  
+  @JsonIgnore
+  public Object getCustomData() {
+    return customData;
+  }
+  
+  @JsonIgnore
+  public void setCustomData(Object customData) {
+    this.customData = customData;
+  }
+  
   public SolrPackageInstance(String id, String description, String version, Manifest manifest,
       List<Plugin> plugins, Map<String, String> params) {
     this.name = id;
diff --git a/solr/core/src/java/org/apache/solr/util/PackageTool.java b/solr/core/src/java/org/apache/solr/util/PackageTool.java
index 96d84ad..fed4f0a 100644
--- a/solr/core/src/java/org/apache/solr/util/PackageTool.java
+++ b/solr/core/src/java/org/apache/solr/util/PackageTool.java
@@ -149,7 +149,8 @@ public class PackageTool extends SolrCLI.ToolBase {
                 String version = parsedVersion.second();
                 boolean noprompt = cli.hasOption('y');
                 boolean isUpdate = cli.hasOption("update") || cli.hasOption('u');
-                packageManager.deploy(packageName, version, PackageUtils.validateCollections(cli.getOptionValue("collections").split(",")), cli.getOptionValues("param"), isUpdate, noprompt);
+                String collections[] = cli.hasOption("collections")? PackageUtils.validateCollections(cli.getOptionValue("collections").split(",")): new String[] {};
+                packageManager.deploy(packageName, version, collections, cli.hasOption("cluster"), cli.getOptionValues("param"), isUpdate, noprompt);
                 break;
               }
               case "undeploy":
@@ -159,7 +160,8 @@ public class PackageTool extends SolrCLI.ToolBase {
                   throw new SolrException(ErrorCode.BAD_REQUEST, "Only package name expected, without a version. Actual: " + cli.getArgList().get(1));
                 }
                 String packageName = parsedVersion.first();
-                packageManager.undeploy(packageName, cli.getOptionValue("collections").split(","));
+                String collections[] = cli.hasOption("collections")? PackageUtils.validateCollections(cli.getOptionValue("collections").split(",")): new String[] {};
+                packageManager.undeploy(packageName, collections, cli.hasOption("cluster"));
                 break;
               }
               case "help":
@@ -241,6 +243,11 @@ public class PackageTool extends SolrCLI.ToolBase {
         .desc("List of collections. Run './solr package help' for more details.")
         .build(),
 
+        Option.builder("cluster")
+        .required(false)
+        .desc("Needed to install cluster level plugins in a package. Run './solr package help' for more details.")
+        .build(),
+
         Option.builder("p")
         .argName("PARAMS")
         .hasArgs()
diff --git a/solr/core/src/test-files/runtimecode/MyPlugin.java b/solr/core/src/test-files/runtimecode/MyPlugin.java
index cbaa347..34cfb6e 100644
--- a/solr/core/src/test-files/runtimecode/MyPlugin.java
+++ b/solr/core/src/test-files/runtimecode/MyPlugin.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.solr.handler;
+package runtimecode;
 
 import org.apache.solr.api.Command;
 import org.apache.solr.api.EndPoint;