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:22:30 UTC

[lucene-solr] branch jira/solr-14599 updated: SOLR-14599: Support for updates to cluster plugins

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

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


The following commit(s) were added to refs/heads/jira/solr-14599 by this push:
     new 30b08fe  SOLR-14599: Support for updates to cluster plugins
30b08fe is described below

commit 30b08fe23c4aafd5273472d401b4ed37f31b030b
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Tue Jun 30 12:52:12 2020 +0530

    SOLR-14599: Support for updates to cluster plugins
---
 .../apache/solr/packagemanager/PackageManager.java | 206 +++++++++++++--------
 .../apache/solr/packagemanager/PackageUtils.java   |   3 +
 .../solr/packagemanager/SolrPackageInstance.java   |  15 ++
 3 files changed, 149 insertions(+), 75 deletions(-)

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 298559b..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,23 +135,32 @@ 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> packages = new HashMap<String,String>();
+    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 + "/api/cluster/zk/data/clusterprops.json", Utils.JSONCONSUMER);
+        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(":")) {
-        packages.put (pluginMeta.get("class").substring(0, pluginMeta.get("class").indexOf(':')), pluginMeta.get("version"));
+        String packageName = pluginMeta.get("class").substring(0, pluginMeta.get("class").indexOf(':'));
+        packageVersions.put(packageName, pluginMeta.get("version"));
+        packagePlugins.put(packageName, pluginMeta);
       }
     }
-    if (packages == null) return Collections.emptyMap();
+    if (packageVersions == null) return Collections.emptyMap();
     Map<String, SolrPackageInstance> ret = new HashMap<>();
-    for (String packageName: packages.keySet()) {
+    for (String packageName: packageVersions.keySet()) {
       if (Strings.isNullOrEmpty(packageName) == false && // There can be an empty key, storing the version here
-          packages.get(packageName) != null) { // null means the package was undeployed from this package before
-        ret.put(packageName, getPackageInstance(packageName, packages.get(packageName)));
+          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;
@@ -172,69 +182,20 @@ public class PackageManager implements Closeable {
   @SuppressWarnings({"unchecked"})
   private boolean deployPackage(SolrPackageInstance packageInstance, boolean pegToLatest, boolean isUpdate, boolean noprompt,
       List<String> collections, boolean shouldDeployClusterPlugins, String[] overrides) {
+
     // Install plugins of type "cluster"
-    boolean previouslyDeployedClusterPlugins = false;
-    if (!isUpdate) {
-      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 + "/api/cluster/zk/data/clusterprops.json", 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.");
-              previouslyDeployedClusterPlugins = 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 = getCollectionParameterOverrides(null, false, overrides, null);
-              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);
-        }
-      }
-    }
+    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);
@@ -342,12 +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() && !previouslyDeployedClusterPlugins && success;
+    success &= previouslyDeployed.isEmpty();
+    return success;
   }
 
-  // nocommit javadocs
-  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) {
@@ -382,7 +438,7 @@ public class PackageManager implements Closeable {
       if (plugin.verifyCommand != null && !Strings.isNullOrEmpty(cmd.path)) {
         if ("cluster".equalsIgnoreCase(plugin.type)) {
           if (!shouldDeployClusterPlugins) continue; // Plugins of type "cluster"
-          Map<String, String> overridesMap = getCollectionParameterOverrides(null, false, overrides, null);
+          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");
@@ -470,15 +526,16 @@ public class PackageManager implements Closeable {
     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,
@@ -496,7 +553,6 @@ public class PackageManager implements Closeable {
     {
       SolrPackageInstance deployedPackage = getPackagesDeployedAsClusterLevelPlugins().get(packageName);
       if (deployedPackage == null) {
-        // nocommit if package has a cluster plugin, print message
         PackageUtils.printRed("Cluster level plugins from package "+packageName+" not deployed.");
       } else {
         for (Plugin plugin: deployedPackage.plugins) {
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/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;