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 07:29:22 UTC

[lucene-solr] branch jira/solr-14599_1 created (now 7d4c041)

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

ishan pushed a change to branch jira/solr-14599_1
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


      at 7d4c041  SOLR-14599: Package manager support for cluster level plugins

This branch includes the following new commits:

     new 7d4c041  SOLR-14599: Package manager support for cluster level plugins

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



[lucene-solr] 01/01: SOLR-14599: Package manager support for cluster level plugins

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

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

commit 7d4c041b6a8b318280fe111b750354ad57748acf
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Tue Jun 30 12:57:00 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 | 252 ++++++++++++++++++---
 .../apache/solr/packagemanager/PackageUtils.java   |   3 +
 .../solr/packagemanager/RepositoryManager.java     |   2 +-
 .../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 |  43 ----
 8 files changed, 260 insertions(+), 72 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 961590c..9553d29 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -117,6 +117,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 6d9babc..3e995dd 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
@@ -33,6 +33,7 @@ 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.common.NavigableObject;
 import org.apache.solr.common.SolrException;
@@ -134,6 +135,37 @@ 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);
+      if (pluginMeta.containsKey("class") && pluginMeta.get("class").contains(":")) {
+        String packageName = pluginMeta.get("class").substring(0, pluginMeta.get("class").indexOf(':'));
+        packageVersions.put(packageName, pluginMeta.get("version"));
+        packagePlugins.put(packageName, pluginMeta);
+      }
+    }
+    if (packageVersions == null) return Collections.emptyMap();
+    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);
@@ -149,9 +181,22 @@ 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 cluasterSuccess = deployClusterPackage(packageInstance, isUpdate, noprompt, shouldDeployClusterPlugins,
+        overrides);
+
+    
+    // Install plugins of type "collection"
+    boolean collectionSuccess = deployCollectionPackage(packageInstance, pegToLatest, isUpdate, noprompt, collections,
+        shouldDeployClusterPlugins, overrides);
+    return cluasterSuccess && collectionSuccess;
+  }
 
+  private boolean 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)) {
@@ -200,9 +245,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 = Map.of("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 = Map.of("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)) {
@@ -249,7 +295,7 @@ public class PackageManager implements Closeable {
     boolean success = true;
     if (deployedCollections.isEmpty() == false) {
       // Verify that package was successfully deployed
-      success = verify(packageInstance, deployedCollections);
+      success = verify(packageInstance, deployedCollections, shouldDeployClusterPlugins, overrides);
       if (success) {
         PackageUtils.printGreen("Deployed on " + deployedCollections + " and verified package: " + packageInstance.name + ", version: " + packageInstance.version);
       }
@@ -257,11 +303,107 @@ public class PackageManager implements Closeable {
     if (previouslyDeployed.isEmpty() == false) {
       PackageUtils.printRed("Already Deployed on " + previouslyDeployed + ", package: " + packageInstance.name + ", version: " + packageInstance.version);
     }
-    return previouslyDeployed.isEmpty() && success;
+    success &= previouslyDeployed.isEmpty();
+    return success;
   }
 
-  private Map<String,String> getCollectionParameterOverrides(SolrPackageInstance packageInstance, boolean isUpdate,
-      String[] overrides, String collection) {
+  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 (Map<String, String> pluginMeta: (List<Map<String, String>>)deployedPackage.getCustomData()) {
+          PackageUtils.printGreen("Updating this plugin: " + pluginMeta);
+          try {
+            String postBody = "{\"update\":{\"name\": \""+pluginMeta.get("name")+"\","
+                + " \"class\": \""+pluginMeta.get("class")+"\", \"version\": \""+packageInstance.version+"\", \"path-prefix\": \""+pluginMeta.get("path-prefix")+"\"}}";
+            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 = Map.of("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): ");
+                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.");
+                }
+              }
+              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);
+        }
+      }
+    }
+    return !cluasterPluginFailed;
+  }
+
+
+  /**
+   * 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) {
@@ -289,31 +431,55 @@ 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).
    */
-  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 = Map.of("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 = Map.of("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 setup commands");
+          }          
+        } else {
+          // Plugins of type "collection"
+          for (String collection: collections) {
+            Map<String, String> collectionParameterOverrides = getPackageParams(pkg.name, collection);
+  
+            Map<String, String> systemParams = Map.of("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 +519,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
+    {
+      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 = Map.of("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 +590,10 @@ public class PackageManager implements Closeable {
       Map<String, String> collectionParameterOverrides = getPackageParams(packageName, collection);
 
       // Run the uninstall command for all plugins
-      Map<String, String> systemParams = Map.of("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 = Map.of("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 602a9e5..d5220c0 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/PackageUtils.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/PackageUtils.java
@@ -64,7 +64,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..9ce4645 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/RepositoryManager.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/RepositoryManager.java
@@ -320,7 +320,7 @@ public class RepositoryManager {
 
     if (peggedToLatest.isEmpty() == false) {
       SolrPackageInstance updatedPackage = packageManager.getPackageInstance(packageName, PackageUtils.LATEST);
-      boolean res = packageManager.verify(updatedPackage, peggedToLatest);
+      boolean res = packageManager.verify(updatedPackage, peggedToLatest, true, new String[] {}); // nocommit: get the overrides here
       PackageUtils.printGreen("Verifying version " + updatedPackage.version + 
           " on " + peggedToLatest + ", 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..e69de29 100644
--- a/solr/core/src/test-files/runtimecode/MyPlugin.java
+++ b/solr/core/src/test-files/runtimecode/MyPlugin.java
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.handler;
-
-import org.apache.solr.api.Command;
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.client.solrj.SolrRequest.METHOD;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.security.PermissionNameProvider;
-
-@EndPoint(path = "/plugin/my/path",
-    method = METHOD.GET,
-    permission = PermissionNameProvider.Name.CONFIG_READ_PERM)
-public class MyPlugin {
-
-  private final CoreContainer coreContainer;
-
-  public MyPlugin(CoreContainer coreContainer) {
-    this.coreContainer = coreContainer;
-  }
-
-  @Command
-  public void call(SolrQueryRequest req, SolrQueryResponse rsp){
-    rsp.add("myplugin.version", "2.0");
-  }
-}