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 2019/10/28 22:01:45 UTC

[lucene-solr] branch jira/solr-13662-updated created (now 362705b)

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

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


      at 362705b  Fix latest auto update

This branch includes the following new commits:

     new 3adace5  SOLR-13662: Package Manager (CLI) first cut
     new 11f1e9f  Latest auto-update functionality
     new be27e8c  Merge branch 'master' into jira/solr-13662-updated
     new 2eda549  add a 'refresh' command to force refresh loading of packages
     new 362705b  Fix latest auto update

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



[lucene-solr] 01/05: SOLR-13662: Package Manager (CLI) first cut

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-13662-updated
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 3adace5e9e7d1041edc6d30fd72382b245f172ed
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Sat Oct 19 23:15:24 2019 +0530

    SOLR-13662: Package Manager (CLI) first cut
---
 lucene/ivy-versions.properties                     |   1 +
 solr/bin/solr                                      |  58 +++
 solr/core/ivy.xml                                  |   5 +
 .../apache/solr/packagemanager/SolrPackage.java    | 109 +++++
 .../solr/packagemanager/SolrPackageInstance.java   |  42 ++
 .../solr/packagemanager/SolrPackageManager.java    | 303 +++++++++++++
 .../solr/packagemanager/SolrPackageRepository.java | 107 +++++
 .../solr/packagemanager/SolrUpdateManager.java     | 501 +++++++++++++++++++++
 .../solr/packagemanager/pf4j/BasicVerifier.java    |  40 ++
 .../solr/packagemanager/pf4j/CompoundVerifier.java |  63 +++
 .../packagemanager/pf4j/DefaultVersionManager.java |  43 ++
 .../solr/packagemanager/pf4j/FileDownloader.java   |  39 ++
 .../solr/packagemanager/pf4j/FileVerifier.java     |  63 +++
 .../pf4j/PackageManagerException.java              |  50 ++
 .../packagemanager/pf4j/Sha512SumVerifier.java     |  83 ++++
 .../packagemanager/pf4j/SimpleFileDownloader.java  | 151 +++++++
 .../solr/packagemanager/pf4j/StringUtils.java      |  56 +++
 .../solr/packagemanager/pf4j/VerifyException.java  |  33 ++
 .../src/java/org/apache/solr/pkg/PackageAPI.java   |  13 +-
 .../src/java/org/apache/solr/util/PackageTool.java | 302 +++++++++++++
 .../src/java/org/apache/solr/util/SolrCLI.java     |   2 +
 .../apache/solr/cloud/PackageManagerCLITest.java   |  84 ++++
 22 files changed, 2147 insertions(+), 1 deletion(-)

diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index e5a4a8a..b4d5d5f 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -36,6 +36,7 @@ com.fasterxml.jackson.core.version = 2.9.9
 /com.ibm.icu/icu4j = 62.1
 /com.jayway.jsonpath/json-path = 2.4.0
 /com.lmax/disruptor = 3.4.2
+/org.pf4j/pf4j-update = 2.1.0
 /com.pff/java-libpst = 0.8.1
 
 com.rometools.version = 1.5.1
diff --git a/solr/bin/solr b/solr/bin/solr
index 596242f..cb82610 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -764,6 +764,59 @@ function get_info() {
   return $CODE
 } # end get_info
 
+function run_package() {
+  runningSolrUrl=""
+
+  numSolrs=`find "$SOLR_PID_DIR" -name "solr-*.pid" -type f | wc -l | tr -d ' '`
+  if [ "$numSolrs" != "0" ]; then
+    echo -e "\nFound $numSolrs Solr nodes: "
+    while read PIDF
+      do
+        ID=`cat "$PIDF"`
+        port=`jetty_port "$ID"`
+        if [ "$port" != "" ]; then
+          echo -e "\nSolr process $ID running on port $port"
+          #run_tool status -solr "$SOLR_URL_SCHEME://$SOLR_TOOL_HOST:$port/solr"
+          runningSolrUrl="$SOLR_URL_SCHEME://$SOLR_TOOL_HOST:$port/solr"
+          break
+          CODE=$?
+          echo ""
+        else
+          echo -e "\nSolr process $ID from $PIDF not found."
+          CODE=1
+        fi
+    done < <(find "$SOLR_PID_DIR" -name "solr-*.pid" -type f)
+  else
+    # no pid files but check using ps just to be sure
+    numSolrs=`ps auxww | grep start\.jar | grep solr\.solr\.home | grep -v grep | wc -l | sed -e 's/^[ \t]*//'`
+    if [ "$numSolrs" != "0" ]; then
+      echo -e "\nFound $numSolrs Solr nodes: "
+      PROCESSES=$(ps auxww | grep start\.jar | grep solr\.solr\.home | grep -v grep | awk '{print $2}' | sort -r)
+      for ID in $PROCESSES
+        do
+          port=`jetty_port "$ID"`
+          if [ "$port" != "" ]; then
+            echo ""
+            echo "Solr process $ID running on port $port"
+            runningSolrUrl="$SOLR_URL_SCHEME://$SOLR_TOOL_HOST:$port/solr"
+            break
+            CODE=$?
+            echo ""
+          fi
+      done
+    else
+      echo -e "\nNo Solr nodes are running.\n"
+      exit 1
+      CODE=3
+    fi
+  fi
+
+  echo "Solr Base URL is $runningSolrUrl"
+  echo "Params: $@"
+  run_tool package -solrUrl "$runningSolrUrl" $@
+  #exit $?
+}
+
 # tries to gracefully stop Solr using the Jetty
 # stop command and if that fails, then uses kill -9
 function stop_solr() {
@@ -1359,6 +1412,11 @@ if [[ "$SCRIPT_CMD" == "export" ]]; then
   exit $?
 fi
 
+if [[ "$SCRIPT_CMD" == "package" ]]; then
+  run_package $@
+  exit $?
+fi
+
 if [[ "$SCRIPT_CMD" == "auth" ]]; then
 
   VERBOSE=""
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 9fba663..4dab1df 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -138,6 +138,11 @@
     <dependency org="com.google.protobuf" name="protobuf-java" rev="${/com.google.protobuf/protobuf-java}" conf="compile"/>
     <dependency org="com.jayway.jsonpath" name="json-path" rev="${/com.jayway.jsonpath/json-path}" conf="compile"/>
 
+    <dependency org="com.github.zafarkhaja" name="java-semver" rev="0.9.0" conf="compile"/>
+    <dependency org="net.minidev" name="json-smart" rev="2.2.1" conf="compile"/>
+    <dependency org="net.minidev" name="asm" rev="1.0.2" conf="compile"/>
+
+
     <dependency org="org.rrd4j" name="rrd4j" rev="${/org.rrd4j/rrd4j}" conf="compile"/>
 
     <!-- JWT Auth plugin -->
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackage.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackage.java
new file mode 100644
index 0000000..8a48728
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackage.java
@@ -0,0 +1,109 @@
+package org.apache.solr.packagemanager;
+
+
+import java.io.Serializable;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Describes a package (along with all released versions) as it appears in a repository.
+ */
+public class SolrPackage implements Serializable, Comparable<SolrPackage> {
+
+  public String id;
+  public String description;
+  public List<SolrPackageRelease> versions;
+
+  private String repositoryId;
+
+  public static class SolrPackageRelease {
+
+    public String version;
+    public Date date;
+    public String requires;
+    public String url;
+
+    public String sha512sum;
+    public String sig;
+
+    public Metadata metadata;
+    @Override
+    public String toString() {
+      return "SolrPackageRelease{" +
+          "version='" + version + '\'' +
+          ", date=" + date +
+          ", requires='" + requires + '\'' +
+          ", url='" + url + '\'' +
+          ", sig='" + sig + '\'' +
+          ", min='" + metadata.minSolrVersion + '\'' +
+          ", max='" + metadata.maxSolrVersion + '\'' +
+          ", dependencies='" + metadata.dependencies + '\'' +
+          ", plugins='" + metadata.plugins + '\'' +
+          ", paramDefaults='" + metadata.parameterDefaults + '\'' +
+          ", sha512sum='" + sha512sum + '\'' +
+          '}';
+    }
+  }
+
+  public static class Metadata {
+    @JsonProperty("min-solr-version")
+    public String minSolrVersion;
+    @JsonProperty("max-solr-version")
+    public String maxSolrVersion;
+
+    public List<String> dependencies;
+    public List<Plugin> plugins;
+    @JsonProperty("parameter-defaults")
+    public Map<String, String> parameterDefaults;
+  }
+
+  public static class Plugin {
+    public String id;
+    @JsonProperty("setup-command")
+    public String setupCommand;
+
+    @JsonProperty("update-command")
+    public String updateCommand;
+
+    @JsonProperty("uninstall-command")
+    public String uninstallCommand;
+
+    @JsonProperty("verify-command")
+    public Command verifyCommand;
+
+    @Override
+    public String toString() {
+      return id + ": {setup: "+setupCommand+", update: "+updateCommand+", uninstall: "+uninstallCommand+", verify: "+verifyCommand+"}";
+    }
+  }
+
+  @Override
+  public int compareTo(SolrPackage o) {
+    return id.compareTo(o.id);
+  }
+
+  public String getRepositoryId() {
+    return repositoryId;
+  }
+
+  public void setRepositoryId(String repositoryId) {
+    this.repositoryId = repositoryId;
+  }
+
+  public static class Command {
+    public String path;
+    public String method;
+    public Map payload;
+    public String condition;
+    public String expected;
+    
+    @Override
+      public String toString() {
+        return method + " " + path + ", Payload: "+ payload+", Condition: "+condition+", expected: "+expected;
+      }
+  }
+}
+
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageInstance.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageInstance.java
new file mode 100644
index 0000000..26f3774
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageInstance.java
@@ -0,0 +1,42 @@
+package org.apache.solr.packagemanager;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.packagemanager.SolrPackage.Plugin;
+
+/**
+ * Describes one instance of a package as it exists in Solr when installed.
+ */
+public class SolrPackageInstance {
+	final public String id;
+	final public String description;
+	final public String version;
+	final public List<Plugin> plugins;
+	final Map<String, String> parameterDefaults;
+	
+	public SolrPackageInstance(String id, String description, String version,
+	    List<Plugin> plugins, Map<String, String> params) {
+		this.id = id;
+		this.description = description;
+		this.version = version;
+		this.plugins = plugins;
+		this.parameterDefaults = params;
+	}
+
+	public String getPluginId() {
+		return id;
+	}
+
+	public String getPluginDescription() {
+		return description;
+	}
+
+	public String getVersion() {
+		return version;
+	}
+
+	public List<Plugin> getPlugins() {
+    return plugins;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java
new file mode 100644
index 0000000..e351a68
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java
@@ -0,0 +1,303 @@
+package org.apache.solr.packagemanager;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.http.HttpEntity;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.packagemanager.SolrPackage.Command;
+import org.apache.solr.packagemanager.SolrPackage.Metadata;
+import org.apache.solr.packagemanager.SolrPackage.Plugin;
+import org.apache.solr.packagemanager.pf4j.DefaultVersionManager;
+import org.apache.solr.packagemanager.pf4j.PackageManagerException;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.jayway.jsonpath.JsonPath;
+
+public class SolrPackageManager implements Closeable {
+
+  final DefaultVersionManager versionManager;
+
+  final String solrBaseUrl;
+  
+  final SolrZkClient zkClient;
+  public SolrPackageManager(File repo, String solrBaseUrl, String zkHost) {
+    versionManager = new DefaultVersionManager();
+    this.solrBaseUrl = solrBaseUrl;
+    this.zkClient = new SolrZkClient(zkHost, 30000);
+    System.out.println("Done initializing a zkClient instance...");
+  }
+
+  Map<String, List<SolrPackageInstance>> packages = null;
+
+  Metadata fetchMetadata(String manifestFilePath) throws MalformedURLException, IOException {
+    String metadataJson = getStringFromStream(solrBaseUrl + "/api/node/files"+manifestFilePath);
+    System.out.println("Fetched metadata blob: "+metadataJson);
+    Metadata metadata = new ObjectMapper().readValue(metadataJson, Metadata.class);
+    System.out.println("Now metadata: "+metadata);
+    return metadata;
+  }
+
+  public List<SolrPackageInstance> getPackages() throws PackageManagerException {
+    System.out.println("Getting packages from clusterprops...");
+    List<SolrPackageInstance> ret = new ArrayList<SolrPackageInstance>();
+    packages = new HashMap<String, List<SolrPackageInstance>>();
+    try {
+      /*String clusterPropsZnode = IOUtils.toString(new URL(solrBaseUrl + "/solr/admin/zookeeper?detail=true&path=/clusterprops.json&wt=json").openStream(), "UTF-8");
+      String clusterPropsJson = ((Map)new ObjectMapper().readValue(clusterPropsZnode, Map.class).get("znode")).get("data").toString();
+      Map packagesJson = (Map)new ObjectMapper().readValue(clusterPropsJson, Map.class).get("packages");*/
+      
+      String clusterPropsJson = null;
+      Map packagesJson = null;
+      
+      if (zkClient.exists("/packages.json", true) == true) {
+        clusterPropsJson = new String(zkClient.getData("/packages.json", null, null, true), "UTF-8");
+        System.out.println("clusterprops are: "+clusterPropsJson);
+        packagesJson = (Map)new ObjectMapper().
+            configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true).readValue(clusterPropsJson, Map.class).get("packages");
+      }
+
+      if (packagesJson != null) {
+        for (Object packageName: packagesJson.keySet()) {
+          List pkg = (List)packagesJson.get(packageName);
+          for (Map pkgVersion: (List<Map>)pkg) {
+            System.out.println("List mein yeh aaya hai: "+pkg); // nocommit don't blindly get .get(0)
+            Metadata metadata = fetchMetadata(pkgVersion.get("manifest").toString());
+            List<Plugin> solrplugins = metadata.plugins;
+            SolrPackageInstance pkgInstance = new SolrPackageInstance(packageName.toString(), null, 
+                pkgVersion.get("version").toString(), solrplugins, metadata.parameterDefaults);
+            List<SolrPackageInstance> list = packages.containsKey(packageName)? packages.get(packageName): new ArrayList<SolrPackageInstance>();
+            list.add(pkgInstance);
+            packages.put(packageName.toString(), list);
+            ret.add(pkgInstance);
+          }
+        }
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      if (packages == null) packages = Collections.emptyMap(); // nocommit can't happen
+      throw new PackageManagerException(e);
+    }
+    System.out.println("Got packages: "+ret);
+    return ret;
+  }
+
+  public boolean deployInstallPackage(String packageName, String version, List<String> collections, String overrides[]) {
+    SolrPackageInstance pkg = getPackage(packageName, version);
+
+    for (String collection: collections) {
+      Map<String, String> collectionParameterOverrides = new HashMap<String,String>();
+      if (overrides != null) {
+        for (String override: overrides) {
+          collectionParameterOverrides.put(override.split("=")[0], override.split("=")[1]);
+        }
+      }
+      try {
+        // nocommit: it overwrites params of other packages (use set or update)
+        
+        boolean packageParamsExist = ((Map)((Map)new ObjectMapper().readValue(
+            get(solrBaseUrl + "/api/collections/abc/config/params/packages"), Map.class)
+            ).get("response")).containsKey("params");
+        
+        postJson(solrBaseUrl + "/api/collections/"+collection+"/config/params",
+            new ObjectMapper().writeValueAsString(
+                Map.of(packageParamsExist? "update": "set", 
+                    Map.of("packages", Map.of(packageName, collectionParameterOverrides)))));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+
+      for (Plugin p: pkg.getPlugins()) {
+        System.out.println(p.setupCommand);
+
+        Map<String, String> systemParams = new HashMap<String,String>();
+        systemParams.put("collection", collection);
+        systemParams.put("package-name", pkg.id);
+        systemParams.put("package-version", pkg.version);
+
+        String cmd = resolve(p.setupCommand, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
+        System.out.println("Executing " + cmd + " for collection:" + collection);
+        postJson(solrBaseUrl + "/solr/"+collection+"/config", cmd);
+      }
+    }
+
+    boolean success = verify(pkg, collections);
+    if (success) {
+      System.out.println("Deployed and verified package: "+pkg.id+", version: "+pkg.version);
+    }
+    return success;
+  }
+
+  private String resolve(String str, Map<String, String> defaults, Map<String, String> overrides, Map<String, String> systemParams) {
+    for (String param: defaults.keySet()) {
+      str = str.replaceAll("\\$\\{"+param+"\\}", overrides.containsKey(param)? overrides.get(param): defaults.get(param));
+    }
+    for (String param: overrides.keySet()) {
+      str = str.replaceAll("\\$\\{"+param+"\\}", overrides.get(param));
+    }
+    for (String param: systemParams.keySet()) {
+      str = str.replaceAll("\\$\\{"+param+"\\}", systemParams.get(param));
+    }
+    return str;
+  }
+  //nocommit should this be private?
+  public boolean verify(SolrPackageInstance pkg, List<String> collections) {
+    // verify deployment succeeded?
+    boolean success = true;
+    for (Plugin p: pkg.getPlugins()) {
+      System.out.println(p.verifyCommand);
+      for (String collection: collections) {
+        System.out.println("Executing " + p.verifyCommand + " for collection:" + collection);
+        Map<String, String> collectionParameterOverrides;
+        try {
+          collectionParameterOverrides = (Map<String, String>)((Map)((Map)((Map)new ObjectMapper().readValue
+              (get(solrBaseUrl + "/api/collections/abc/config/params/packages"), Map.class).get("response")).get("params")).get("packages")).get(pkg.id);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        
+        Command cmd = p.verifyCommand;
+
+        Map<String, String> systemParams = new HashMap<String,String>();
+        systemParams.put("collection", collection);
+        systemParams.put("package-name", pkg.id);
+        systemParams.put("package-version", pkg.version);
+        String url = solrBaseUrl + resolve(cmd.path, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
+
+        if ("GET".equalsIgnoreCase(cmd.method)) {
+          String response = get(url);
+          System.out.println(response);
+          String actualValue = JsonPath.parse(response).read(resolve(cmd.condition, pkg.parameterDefaults, collectionParameterOverrides, systemParams));
+          String expectedValue = resolve(cmd.expected, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
+          System.out.println("Actual: "+actualValue+", expected: "+expectedValue);
+          if (!expectedValue.equals(actualValue)) {
+            System.out.println("Failed to deploy plugin: "+p.id);
+            success = false;
+          }
+        } // commit POST?
+      }
+    }
+    return success;
+  }
+
+  public boolean deployUpdatePackage(String pluginId, String version, List<String> collections) {
+    SolrPackageInstance pkg = getPackage(pluginId, version);
+    for (Plugin p: pkg.getPlugins()) {
+
+      System.out.println(p.updateCommand);
+      for (String collection: collections) {
+        System.out.println("Executing " + p.updateCommand + " for collection:" + collection);
+        postJson(solrBaseUrl + "/solr/"+collection+"/config", p.updateCommand);
+      }
+    }
+    boolean success = verify(pkg, collections);
+    if (success) {
+      System.out.println("Deployed and verified package: "+pkg.id+", version: "+pkg.version);
+    }
+    return true;
+  }
+
+  String getStringFromStream(String url) {
+    return get(url);
+  }
+
+  private String get(String url) {
+    try (CloseableHttpClient client = SolrUpdateManager.createTrustAllHttpClientBuilder()) {
+      HttpGet httpGet = new HttpGet(url);
+      httpGet.setHeader("Content-type", "application/json");
+
+      CloseableHttpResponse response = client.execute(httpGet);
+
+      try {
+        HttpEntity rspEntity = response.getEntity();
+        if (rspEntity != null) {
+          InputStream is = rspEntity.getContent();
+          StringWriter writer = new StringWriter();
+          IOUtils.copy(is, writer, "UTF-8");
+          String results = writer.toString();
+
+          return(results);
+        }
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    } catch (Exception e1) {
+      throw new RuntimeException(e1);
+    }
+    return null;
+  }
+
+  private void postJson(String url, String postBody) {
+    System.out.println("Posting to "+url+": "+postBody);
+    try (CloseableHttpClient client = SolrUpdateManager.createTrustAllHttpClientBuilder();) {
+      HttpPost httpPost = new HttpPost(url);
+      StringEntity entity = new StringEntity(postBody);
+      httpPost.setEntity(entity);
+      httpPost.setHeader("Accept", "application/json");
+      httpPost.setHeader("Content-type", "application/json");
+
+      CloseableHttpResponse response = client.execute(httpPost);
+
+      try {
+        HttpEntity rspEntity = response.getEntity();
+        if (rspEntity != null) {
+          InputStream is = rspEntity.getContent();
+          StringWriter writer = new StringWriter();
+          IOUtils.copy(is, writer, "UTF-8");
+          String results = writer.toString();
+          System.out.println(results);
+        }
+      } catch (IOException e) {
+        e.printStackTrace();
+        throw new RuntimeException(e);
+      }
+    } catch (Exception e1) {
+      throw new RuntimeException(e1);
+    }
+  }
+
+  // nocommit: javadocs should mention that version==null or "latest" will return latest version installed
+  public SolrPackageInstance getPackage(String pluginId, String version) {
+    getPackages();
+    List<SolrPackageInstance> versions = packages.get(pluginId);
+    String latestVersion = "0.0.0";
+    SolrPackageInstance latest = null;
+    if (versions != null) {
+      for (SolrPackageInstance pkg: versions) {
+        if (pkg.version.equals(version)) {
+          return pkg;
+        }
+        if (versionManager.compareVersions(latestVersion, pkg.version) <= 0) {
+          latestVersion = pkg.version;
+          latest = pkg;
+        }
+      }
+    }
+    if (version == null || version.equalsIgnoreCase("latest")) {
+      return latest;
+    } else return null;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (zkClient != null) {
+      zkClient.close();
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageRepository.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageRepository.java
new file mode 100644
index 0000000..eaca740
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageRepository.java
@@ -0,0 +1,107 @@
+package org.apache.solr.packagemanager;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.packagemanager.SolrPackage.SolrPackageRelease;
+import org.apache.solr.packagemanager.pf4j.CompoundVerifier;
+import org.apache.solr.packagemanager.pf4j.FileDownloader;
+import org.apache.solr.packagemanager.pf4j.FileVerifier;
+import org.apache.solr.packagemanager.pf4j.SimpleFileDownloader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public class SolrPackageRepository {
+  private static final Logger log = LoggerFactory.getLogger(SolrPackageRepository.class);
+
+  @JsonProperty("id")
+  public String id;
+  @JsonProperty("url")
+  public String url;
+
+  public SolrPackageRepository() {
+  }//nocommit wtf?
+
+  public SolrPackageRepository(String id, String url) {
+    this.id = id;
+    this.url = url;
+  }
+
+  public void refresh() {
+    packages = null;
+  }
+
+  @JsonIgnore
+  public FileDownloader getFileDownloader() {
+      return new SimpleFileDownloader();
+  }
+
+  @JsonIgnore
+  public FileVerifier getFileVerfier() {
+      return new CompoundVerifier();
+  }
+
+  @JsonIgnore
+  private Map<String, SolrPackage> packages;
+
+  public Map<String, SolrPackage> getPackages() {
+    if (packages == null) {
+      initPackages();
+    }
+
+    return packages;
+  }
+
+  public SolrPackage getPlugin(String id) {
+    return getPackages().get(id);
+  }
+
+  private void initPackages() {
+    Reader pluginsJsonReader;
+    try {
+      URL pluginsUrl = new URL(new URL(url), "manifest.json"); //nocommit hardcoded
+      log.debug("Read plugins of '{}' repository from '{}'", id, pluginsUrl);
+      pluginsJsonReader = new InputStreamReader(pluginsUrl.openStream());
+    } catch (Exception e) {
+      log.error(e.getMessage(), e);
+      packages = Collections.emptyMap();
+      return;
+    }
+
+    ObjectMapper mapper = new ObjectMapper();
+    SolrPackage items[];
+    try {
+      items = mapper.readValue(pluginsJsonReader, SolrPackage[].class);
+    } catch (IOException e1) {
+      throw new RuntimeException(e1);
+    }
+    packages = new HashMap<>(items.length);
+    for (SolrPackage p : items) {
+      for (SolrPackageRelease r : p.versions) {
+        try {
+          r.url = new URL(new URL(url), r.url).toString();
+          if (r.date.getTime() == 0) {
+            log.warn("Illegal release date when parsing {}@{}, setting to epoch", p.id, r.version);
+          }
+        } catch (MalformedURLException e) {
+          log.warn("Skipping release {} of plugin {} due to failure to build valid absolute URL. Url was {}{}", r.version, p.id, url, r.url);
+        }
+      }
+      p.setRepositoryId(id);
+      packages.put(p.id, p);
+      
+      System.out.println("****\n"+p+"\n*******");
+    }
+    log.debug("Found {} plugins in repository '{}'", packages.size(), id);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrUpdateManager.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrUpdateManager.java
new file mode 100644
index 0000000..c4abf5c
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrUpdateManager.java
@@ -0,0 +1,501 @@
+package org.apache.solr.packagemanager;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.conn.ssl.SSLContextBuilder;
+import org.apache.http.entity.mime.MultipartEntity;
+import org.apache.http.entity.mime.content.FileBody;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+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.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.response.V2Response;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.packagemanager.SolrPackage.SolrPackageRelease;
+import org.apache.solr.packagemanager.pf4j.CompoundVerifier;
+import org.apache.solr.packagemanager.pf4j.DefaultVersionManager;
+import org.apache.solr.packagemanager.pf4j.FileDownloader;
+import org.apache.solr.packagemanager.pf4j.FileVerifier;
+import org.apache.solr.packagemanager.pf4j.PackageManagerException;
+import org.apache.solr.packagemanager.pf4j.SimpleFileDownloader;
+import org.apache.solr.pkg.PackageAPI;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public class SolrUpdateManager {
+
+  final private SolrPackageManager packageManager;
+  final private String repositoriesJsonStr;
+  protected List<SolrPackageRepository> repositories;
+  
+  private DefaultVersionManager versionManager;
+  private String systemVersion;
+  private Map<String, SolrPackageRelease> lastPluginRelease = new HashMap<>();
+
+  final String solrBaseUrl;
+
+  private static final Logger log = LoggerFactory.getLogger(SolrUpdateManager.class);
+
+  public SolrUpdateManager(SolrPackageManager pluginManager, String repositoriesJsonStr, String solrBaseUrl) {
+    this.packageManager = pluginManager;
+    this.repositoriesJsonStr = repositoriesJsonStr;
+    versionManager = new DefaultVersionManager();
+    systemVersion = "0.0.0";
+    this.solrBaseUrl = solrBaseUrl;
+  }
+
+  protected synchronized void initRepositoriesFromJson() {
+    SolrPackageRepository items[];
+    try {
+      items = new ObjectMapper().readValue(this.repositoriesJsonStr, SolrPackageRepository[].class);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    this.repositories = Arrays.asList(items);
+  }
+
+  public synchronized void refresh() {
+    initRepositoriesFromJson();
+    for (SolrPackageRepository updateRepository : repositories) {
+      updateRepository.refresh();
+    }
+  }
+
+
+  public synchronized boolean installPackage(String id, String version) throws PackageManagerException {
+    return updateOrInstallPackage(Operation.INSTALL, id, version);
+  }
+
+  public synchronized boolean updatePackage(String id, String version) throws PackageManagerException {
+    return updateOrInstallPackage(Operation.UPDATE, id, version);
+  }
+  
+  // nocommit do we need this, when we have a map version of this?
+  public List<SolrPackage> getPackages() {
+    List<SolrPackage> list = new ArrayList<>(getPackagesMap().values());
+    Collections.sort(list);
+
+    return list;
+  }
+  
+  public Map<String, SolrPackage> getPackagesMap() {
+    Map<String, SolrPackage> packagesMap = new HashMap<>();
+    for (SolrPackageRepository repository : getRepositories()) {
+      packagesMap.putAll(repository.getPackages());
+    }
+
+    return packagesMap;
+  }
+  
+  public List<SolrPackageRepository> getRepositories() {
+    refresh();
+    return repositories;
+  }
+
+  private boolean updateOrInstallPackage(Operation op, String id, String version) throws PackageManagerException {
+    /*Path downloaded = downloadPackage(id, version);
+
+    SolrPackageInstance existingPlugin = packageManager.getPackage(id);
+    if (existingPlugin != null && version.equals(existingPlugin.getVersion())) {
+      throw new PackageManagerException("Plugin already installed.");
+    }
+
+    SolrPackageRelease release = null;
+    String repository = null;
+    for (SolrPackage pkg: getPackages()) {
+      if (id.equals(pkg.id)) {
+        for (SolrPackageRelease r: pkg.versions) {
+          if (version.equals(r.version) ) {
+            release = r;
+            repository = pkg.getRepositoryId();
+            break;
+          }
+        }
+      }
+    }
+
+    if (release == null) {
+      throw new PackageManagerException("Couldn't find the release..");
+    }
+
+    String sha256 = uploadToBlobHandler(downloaded);
+    String metadataSha256;
+    try {
+      metadataSha256 = uploadToBlobHandler(new ObjectMapper().writeValueAsString(release.metadata));
+    } catch (IOException e) {
+      throw new PackageManagerException(e);
+    }
+
+    addOrUpdatePackage(op, id, version, sha256, repository, release.sig, metadataSha256, release.metadata);
+    
+    return true;*/
+    
+    //postFile(cluster.getSolrClient(), getFileContent("runtimecode/runtimelibs.jar.bin"),
+    //    "/package/mypkg/v1.0/runtimelibs.jar",
+    //    "j+Rflxi64tXdqosIhbusqi6GTwZq8znunC/dzwcWW0/dHlFGKDurOaE1Nz9FSPJuXbHkVLj638yZ0Lp1ssnoYA=="
+    //);
+
+    SolrPackageInstance existingPlugin = packageManager.getPackage(id, version);
+    if (existingPlugin != null && version.equals(existingPlugin.getVersion())) {
+      throw new PackageManagerException("Plugin already installed.");
+    }
+
+    SolrPackage pkg = getPackagesMap().get(id);
+    SolrPackageRelease release = findReleaseForPlugin(id, version);
+    Path downloaded = downloadPackage(id, version);
+    System.out.println("Yahaan file hai: "+downloaded);
+    System.out.println("Signature: "+release.sig);
+    System.out.println("Filename: "+downloaded.getFileName().toString());
+    
+    try (HttpSolrClient solrClient = new HttpSolrClient.Builder(solrBaseUrl).build()) {
+      // post the metadata
+      System.out.println("Posting metadata");
+      postFile(solrClient, ByteBuffer.wrap(new ObjectMapper().writeValueAsString(release.metadata).getBytes()),
+          "/package/"+id+"/"+version+"/solr-manifest.json",
+          null);
+
+      // post the artifacts
+      System.out.println("Posting artifacts");
+      postFile(solrClient, getFileContent(downloaded.toFile()),
+          "/package/"+id+"/"+version+"/"+downloaded.getFileName().toString(),
+          release.sig
+      );
+      
+      addOrUpdatePackage(op, solrClient, id, version, new String[] {"/package/"+id+"/"+version+"/"+downloaded.getFileName().toString()}, 
+          pkg.getRepositoryId(), release.sig, "/package/"+id+"/"+version+"/solr-manifest.json", null);
+    } catch (SolrServerException | IOException e) {
+      throw new PackageManagerException(e);
+    }
+    return false;
+  }
+  
+  public static ByteBuffer getFileContent(File file) throws IOException {
+    ByteBuffer jar;
+    try (FileInputStream fis = new FileInputStream(file)) {
+      byte[] buf = new byte[fis.available()];
+      fis.read(buf);
+      jar = ByteBuffer.wrap(buf);
+    }
+    return jar;
+  }
+
+  public static void postFile(SolrClient client, ByteBuffer buffer, String name, String sig)
+      throws SolrServerException, IOException {
+    String resource = "/api/cluster/files" + name;
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    if (sig != null) {
+      params.add("sig", sig);
+    }
+    V2Response rsp = new V2Request.Builder(resource)
+        .withMethod(SolrRequest.METHOD.PUT)
+        .withPayload(buffer)
+        .forceV2(true)
+        .withMimeType("application/octet-stream")
+        .withParams(params)
+        .build()
+        .process(client);
+    if (!name.equals(rsp.getResponse().get(CommonParams.FILE))) {
+      throw new PackageManagerException("Mismatch in file uploaded. Uploaded: " +
+          rsp.getResponse().get(CommonParams.FILE)+", Original: "+name);
+    }
+  }
+
+  public static enum Operation {
+    INSTALL, UPDATE;
+  }
+  
+  private boolean addOrUpdatePackage(Operation op, SolrClient solrClient, String id, String version, String files[], String repository, String sig,
+      String manifest, String manifestSHA512) {
+    
+    /*String json;
+    
+    if (op.equals(Operation.INSTALL)) {
+      json = "{add: ";
+    } else {
+      json = "{update: ";
+    }
+    json = json
+        + "{name: '"+id+"', "
+        + "version: '"+version+"', "
+        + "repository: '"+repository+"', "
+        + "blob: {sha256: '"+sha256+"', sig: '"+sig+"'}, "
+        + "metadata: '"+metadataSha256+"'"
+        + "}}";
+
+    System.out.println("Posting package: "+json);
+    try (CloseableHttpClient client = createTrustAllHttpClientBuilder()) {
+      HttpPost httpPost = new HttpPost(solrBaseUrl + "/api/cluster/package");
+      StringEntity entity = new StringEntity(json);
+      httpPost.setEntity(entity);
+      httpPost.setHeader("Accept", "application/json");
+      httpPost.setHeader("Content-type", "application/json");
+
+      try (CloseableHttpResponse response = client.execute(httpPost)) {
+        HttpEntity rspEntity = response.getEntity();
+        if (rspEntity != null) {
+          InputStream is = rspEntity.getContent();
+          StringWriter writer = new StringWriter();
+          IOUtils.copy(is, writer, "UTF-8");
+          String results = writer.toString();
+          System.out.println(results);
+        }
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    }*/
+    
+    PackageAPI.AddVersion add = new PackageAPI.AddVersion();
+    add.version = version;
+    add.pkg = id;
+    add.files = Arrays.asList(files);
+    add.manifest = manifest;
+    add.manifestSHA512 = "MY_MANIFEST_SHA512";
+
+    V2Request req = new V2Request.Builder("/api/cluster/package")
+        .forceV2(true)
+        .withMethod(SolrRequest.METHOD.POST)
+        .withPayload(Collections.singletonMap("add", add))
+        .build();
+
+    try {
+      V2Response resp = req.process(solrClient);
+      System.out.println("Response: "+resp.jsonStr());
+    } catch (SolrServerException | IOException e) {
+      throw new PackageManagerException(e);
+    }
+
+    return true;
+  }
+
+  public static CloseableHttpClient createTrustAllHttpClientBuilder() throws Exception {
+    SSLContextBuilder builder = new SSLContextBuilder();
+    builder.loadTrustMaterial(null, (chain, authType) -> true);           
+    SSLConnectionSocketFactory sslsf = new 
+    SSLConnectionSocketFactory(builder.build(), NoopHostnameVerifier.INSTANCE);
+    return HttpClients.custom().setSSLSocketFactory(sslsf).build();
+  }
+  
+  private String uploadToBlobHandler(Path downloaded) throws PackageManagerException {
+    String url = solrBaseUrl + "/api/cluster/blob";
+    File file = downloaded.toFile();
+    try (CloseableHttpClient client = createTrustAllHttpClientBuilder()) { //HttpClients.createDefault();) {
+      HttpPost post = new HttpPost(url);
+
+      MultipartEntity entity = new MultipartEntity();
+      entity.addPart("file", new FileBody(file));
+      post.setEntity(entity);
+
+      try {
+        HttpResponse response = client.execute(post);
+
+        HttpEntity rspEntity = response.getEntity();
+        if (rspEntity != null) {
+          InputStream is = rspEntity.getContent();
+          StringWriter writer = new StringWriter();
+          IOUtils.copy(is, writer, "UTF-8");
+          String results = writer.toString();
+          System.out.println(results);
+          String sha = new ObjectMapper().readValue(results, Map.class).get("sha256").toString();
+          //System.out.println("SHA: "+sha);
+          return sha;
+        }
+      } catch (IOException e) {
+        // TODO Auto-generated catch block
+        throw e;
+      }
+    } catch (Exception e1) {
+      // TODO Auto-generated catch block
+      e1.printStackTrace();
+      throw new PackageManagerException(e1);
+    }
+    return null;
+  }
+  
+  private String uploadToBlobHandler(String json) throws IOException, PackageManagerException {
+    System.out.println("Trying to upload the blob: "+json);
+    FileUtils.writeStringToFile(new File("tmp-metadata"), json);
+    return uploadToBlobHandler(new File("tmp-metadata").toPath());
+  }
+
+  /**
+   * Downloads a plugin with given coordinates, runs all {@link FileVerifier}s
+   * and returns a path to the downloaded file.
+   *
+   * @param id of plugin
+   * @param version of plugin or null to download latest
+   * @return Path to file which will reside in a temporary folder in the system default temp area
+   * @throws PackageManagerException if download failed
+   */
+  protected Path downloadPackage(String id, String version) throws PackageManagerException {
+      try {
+          SolrPackageRelease release = findReleaseForPlugin(id, version);
+          Path downloaded = getFileDownloader(id).downloadFile(new URL(release.url));
+          //getFileVerifier(id).verify(new FileVerifier.Context(id, release), downloaded);
+          //nocommit verify this download
+          return downloaded;
+      } catch (IOException e) {
+          throw new PackageManagerException(e, "Error during download of plugin {}", id);
+      }
+  }
+
+  /**
+   * Finds the {@link FileDownloader} to use for this repository.
+   *
+   * @param pluginId the plugin we wish to download
+   * @return FileDownloader instance
+   */
+  protected FileDownloader getFileDownloader(String pluginId) {
+      for (SolrPackageRepository ur : repositories) {
+          if (ur.getPlugin(pluginId) != null && ur.getFileDownloader() != null) {
+              return ur.getFileDownloader();
+          }
+      }
+
+      return new SimpleFileDownloader();
+  }
+
+  /**
+   * Gets a file verifier to use for this plugin. First tries to use custom verifier
+   * configured for the repository, then fallback to the default CompoundVerifier
+   *
+   * @param pluginId the plugin we wish to download
+   * @return FileVerifier instance
+   */
+  protected FileVerifier getFileVerifier(String pluginId) {
+      for (SolrPackageRepository ur : repositories) {
+          if (ur.getPlugin(pluginId) != null && ur.getFileVerfier() != null) {
+              return ur.getFileVerfier();
+          }
+      }
+
+      return new CompoundVerifier();
+  }
+  
+  /**
+   * Resolves Release from id and version.
+   *
+   * @param id of plugin
+   * @param version of plugin or null to locate latest version
+   * @return PluginRelease for downloading
+   * @throws PackageManagerException if id or version does not exist
+   */
+  public SolrPackageRelease findReleaseForPlugin(String id, String version) throws PackageManagerException {
+      SolrPackage pluginInfo = getPackagesMap().get(id);
+      if (pluginInfo == null) {
+          log.info("Plugin with id {} does not exist in any repository", id);
+          throw new PackageManagerException("Plugin with id {} not found in any repository", id);
+      }
+
+      if (version == null) {
+          return getLastPackageRelease(id);
+      }
+
+      for (SolrPackageRelease release : pluginInfo.versions) {
+          if (versionManager.compareVersions(version, release.version) == 0 && release.url != null) {
+              return release;
+          }
+      }
+
+      throw new PackageManagerException("Plugin {} with version @{} does not exist in the repository", id, version);
+  }
+  
+  /**
+   * Returns the last release version of this plugin for given system version, regardless of release date.
+   *
+   * @return PluginRelease which has the highest version number
+   */
+  public SolrPackageRelease getLastPackageRelease(String id) {
+      SolrPackage pluginInfo = getPackagesMap().get(id);
+      if (pluginInfo == null) {
+          return null;
+      }
+
+      if (!lastPluginRelease.containsKey(id)) {
+          for (SolrPackageRelease release : pluginInfo.versions) {
+              if (systemVersion.equals("0.0.0") || versionManager.checkVersionConstraint(systemVersion, release.requires)) {
+                  if (lastPluginRelease.get(id) == null) {
+                      lastPluginRelease.put(id, release);
+                  } else if (versionManager.compareVersions(release.version, lastPluginRelease.get(id).version) > 0) {
+                      lastPluginRelease.put(id, release);
+                  }
+              }
+          }
+      }
+
+      return lastPluginRelease.get(id);
+  }
+  
+  /**
+   * Finds whether the newer version of the plugin.
+   *
+   * @return true if there is a newer version available which is compatible with system
+   */
+  public boolean hasPluginUpdate(String id) {
+      SolrPackage pluginInfo = getPackagesMap().get(id);
+      if (pluginInfo == null) {
+          return false;
+      }
+
+      String installedVersion = packageManager.getPackage(id, null).getVersion();
+      SolrPackageRelease last = getLastPackageRelease(id);
+
+      return last != null && versionManager.compareVersions(last.version, installedVersion) > 0;
+  }
+
+  
+  /**
+   * Return a list of plugins that are newer versions of already installed plugins.
+   *
+   * @return list of plugins that have updates
+   */
+  public List<SolrPackage> getUpdates() {
+      List<SolrPackage> updates = new ArrayList<>();
+      for (SolrPackageInstance installed : packageManager.getPackages()) {
+          String pluginId = installed.getPluginId();
+          if (hasPluginUpdate(pluginId)) {
+              updates.add(getPackagesMap().get(pluginId));
+          }
+      }
+
+      return updates;
+  }
+
+  /**
+   * Checks if Update Repositories has newer versions of some of the installed plugins.
+   *
+   * @return true if updates exist
+   */
+  public boolean hasUpdates() {
+      return getUpdates().size() > 0;
+  }
+
+
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/BasicVerifier.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/BasicVerifier.java
new file mode 100644
index 0000000..f912b06
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/BasicVerifier.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright (C) 2012-present the original author or authors.
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+/**
+ * Verifies that the file exists is a regular file and has a non-null size
+ */
+public class BasicVerifier implements FileVerifier {
+    /**
+     * Verifies a plugin release according to certain rules
+     *
+     * @param context the file verifier context object
+     * @param file    the path to the downloaded file itself
+     * @throws IOException     if there was a problem accessing file
+     * @throws VerifyException in case of problems verifying the file
+     */
+    @Override
+    public void verify(Context context, Path file) throws VerifyException, IOException {
+        if (!Files.isRegularFile(file) || Files.size(file) == 0) {
+            throw new VerifyException("File {} is not a regular file or has size 0", file);
+        }
+    }
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/CompoundVerifier.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/CompoundVerifier.java
new file mode 100644
index 0000000..481ade7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/CompoundVerifier.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright (C) 2012-present the original author or authors.
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class CompoundVerifier implements FileVerifier {
+    /**
+     * Default list of verifiers
+     */
+    public static final List<FileVerifier> ALL_DEFAULT_FILE_VERIFIERS = Arrays.asList(
+                new BasicVerifier(),
+                new Sha512SumVerifier());
+
+    private List<FileVerifier> verifiers = new ArrayList<>();
+
+    /**
+     * Default constructor which will add the default verifiers to start with
+     */
+    public CompoundVerifier() {
+        setVerifiers(ALL_DEFAULT_FILE_VERIFIERS);
+    }
+
+    /**
+     * Constructs a Compound verifier using the supplied list of verifiers instead of the default ones
+     * @param verifiers the list of verifiers to apply
+     */
+    public CompoundVerifier(List<FileVerifier> verifiers) {
+        this.verifiers = verifiers;
+    }
+
+    @Override
+    public void verify(Context context, Path file) throws IOException, VerifyException {
+        for (FileVerifier verifier : getVerifiers()) {
+            verifier.verify(context, file);
+        }
+    }
+
+    public List<FileVerifier> getVerifiers() {
+        return verifiers;
+    }
+
+    public void setVerifiers(List<FileVerifier> verifiers) {
+        this.verifiers = verifiers;
+    }
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/DefaultVersionManager.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/DefaultVersionManager.java
new file mode 100644
index 0000000..8ccce47
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/DefaultVersionManager.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2017 Decebal Suiu
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+import com.github.zafarkhaja.semver.Version;
+import com.github.zafarkhaja.semver.expr.Expression;
+
+/**
+ * This implementation uses jSemVer (a Java implementation of the SemVer Specification).
+ *
+ * @author Decebal Suiu
+ */
+public class DefaultVersionManager {
+
+    /**
+     * Checks if a version satisfies the specified SemVer {@link Expression} string.
+     * If the constraint is empty or null then the method returns true.
+     * Constraint examples: {@code >2.0.0} (simple), {@code ">=1.4.0 & <1.6.0"} (range).
+     * See https://github.com/zafarkhaja/jsemver#semver-expressions-api-ranges for more info.
+     *
+     */
+    public boolean checkVersionConstraint(String version, String constraint) {
+        return StringUtils.isNullOrEmpty(constraint) || Version.valueOf(version).satisfies(constraint);
+    }
+
+    public int compareVersions(String v1, String v2) {
+        return Version.valueOf(v1).compareTo(Version.valueOf(v2));
+    }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/FileDownloader.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/FileDownloader.java
new file mode 100644
index 0000000..32f6fe4
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/FileDownloader.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright (C) 2012-present the original author or authors.
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+
+import java.io.IOException;
+import java.net.URL;
+import java.nio.file.Path;
+
+/**
+ * Interface to download a file.
+ */
+public interface FileDownloader {
+
+    /**
+     * Downloads a file to destination. The implementation should download to a temporary folder.
+     * Implementations may choose to support different protocols such as http, https, ftp, file...
+     * The path returned must be of temporary nature and will most probably be moved/deleted by consumer.
+     *
+     * @param fileUrl the URL representing the file to download
+     * @return Path of downloaded file, typically in a temporary folder
+     * @throws IOException if there was an IO problem during download
+     * @throws PackageManagerException in case of other problems, such as unsupported protocol
+     */
+    Path downloadFile(URL fileUrl) throws PackageManagerException, IOException;
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/FileVerifier.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/FileVerifier.java
new file mode 100644
index 0000000..5f755aa
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/FileVerifier.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright (C) 2012-present the original author or authors.
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.packagemanager.SolrPackage;
+
+/**
+ * Interface to verify a file.
+ */
+public interface FileVerifier {
+
+    void verify(Context context, Path file) throws IOException, VerifyException;
+
+    /**
+     * Context to be passed to file verifiers
+     */
+    class Context {
+        public String id;
+        public Date date;
+        public String version;
+        public String requires;
+        public String url;
+        public String sha512sum;
+        public Map<String,Object> meta = new HashMap<>();
+
+        public Context(String id, SolrPackage.SolrPackageRelease pluginRelease) {
+            this.id = id;
+            this.date = pluginRelease.date;
+            this.version = pluginRelease.version;
+            this.requires = pluginRelease.requires;
+            this.url = pluginRelease.url;
+            this.sha512sum = pluginRelease.sha512sum;
+        }
+
+        public Context(String id, Date date, String version, String requires, String url, String sha512sum) {
+            this.id = id;
+            this.date = date;
+            this.version = version;
+            this.requires = requires;
+            this.url = url;
+            this.sha512sum = sha512sum;
+        }
+    }
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/PackageManagerException.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/PackageManagerException.java
new file mode 100644
index 0000000..76fb761
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/PackageManagerException.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2012 Decebal Suiu
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+
+/**
+ * An exception used to indicate that a plugin problem occurred.
+ *
+ * @author Decebal Suiu
+ */
+public class PackageManagerException extends RuntimeException {
+
+	public PackageManagerException() {
+        super();
+    }
+
+    public PackageManagerException(String message) {
+        super(message);
+    }
+
+    public PackageManagerException(Throwable cause) {
+        super(cause);
+    }
+
+    public PackageManagerException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public PackageManagerException(Throwable cause, String message, Object... args) {
+        super(StringUtils.format(message, args), cause);
+    }
+
+    public PackageManagerException(String message, Object... args) {
+        super(StringUtils.format(message, args));
+    }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/Sha512SumVerifier.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/Sha512SumVerifier.java
new file mode 100644
index 0000000..26f2535
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/Sha512SumVerifier.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright (C) 2012-present the original author or authors.
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+/**
+ * Verifies that the SHA512 checksum of a downloaded file equals the checksum given in
+ * the plugins.json descriptor. This helps validate that the file downloaded is exactly
+ * the same as intended. Especially useful when dealing with meta repositories pointing
+ * to S3 or other 3rd party download locations that could have been tampered with.
+ */
+public class Sha512SumVerifier implements FileVerifier {
+    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+    /**
+     * Verifies a plugin release according to certain rules
+     *
+     * @param context the file verifier context object
+     * @param file    the path to the downloaded file itself
+     * @throws IOException     if there was a problem accessing file
+     * @throws VerifyException in case of problems verifying the file
+     */
+    @Override
+    public void verify(Context context, Path file) throws VerifyException, IOException {
+        String expectedSha512sum;
+        try {
+            if (context.sha512sum == null) {
+                log.debug("No sha512 checksum specified, skipping verification");
+                return;
+            } else if (context.sha512sum.equalsIgnoreCase(".sha512")) {
+                String url = context.url.substring(0, context.url.lastIndexOf(".")) + ".sha512";
+                expectedSha512sum = getUrlContents(url).split(" ")[0].trim();
+            } else if (context.sha512sum.startsWith("http")) {
+                expectedSha512sum = getUrlContents(context.sha512sum).split(" ")[0].trim();
+            } else {
+                expectedSha512sum = context.sha512sum;
+            }
+        } catch (IOException e) {
+            throw new VerifyException(e, "SHA512 checksum verification failed, could not download SHA512 file ({})", context.sha512sum);
+        }
+
+        log.debug("Verifying sha512 checksum of file {}", file.getFileName());
+        String actualSha512sum = DigestUtils.sha512Hex(Files.newInputStream(file));
+        if (actualSha512sum.equalsIgnoreCase(expectedSha512sum)) {
+            log.debug("Checksum OK");
+            return;
+        }
+        throw new VerifyException("SHA512 checksum of downloaded file " + file.getFileName()
+                + " does not match that from plugin descriptor. Got " + actualSha512sum
+                + " but expected " + expectedSha512sum);
+    }
+
+    private String getUrlContents(String url) throws IOException {
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(
+                new URL(url).openStream()))) {
+            return reader.readLine();
+        }
+    }
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/SimpleFileDownloader.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/SimpleFileDownloader.java
new file mode 100644
index 0000000..5b7c3ae
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/SimpleFileDownloader.java
@@ -0,0 +1,151 @@
+/*
+ * Copyright (C) 2012-present the original author or authors.
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ConnectException;
+import java.net.HttpURLConnection;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.FileTime;
+
+/**
+ * Downloads a file from a URL.
+ *
+ * @author Decebal Suiu
+ */
+public class SimpleFileDownloader implements FileDownloader {
+
+    private static final Logger log = LoggerFactory.getLogger(SimpleFileDownloader.class);
+
+    /**
+     * Downloads a file. If HTTP(S) or FTP, stream content, if local file:/ do a simple filesystem copy to tmp folder.
+     * Other protocols not supported.
+     *
+     * @param fileUrl the URI representing the file to download
+     * @return the path of downloaded/copied file
+     * @throws IOException in case of network or IO problems
+     * @throws PackageManagerException in case of other problems
+     */
+    public Path downloadFile(URL fileUrl) throws PackageManagerException, IOException {
+        switch (fileUrl.getProtocol()) {
+            case "http":
+            case "https":
+            case "ftp":
+                return downloadFileHttp(fileUrl);
+            case "file":
+                return copyLocalFile(fileUrl);
+            default:
+                throw new PackageManagerException("URL protocol {} not supported", fileUrl.getProtocol());
+        }
+    }
+
+    /**
+     * Efficient copy of file in case of local file system.
+     *
+     * @param fileUrl source file
+     * @return path of target file
+     * @throws IOException if problems during copy
+     * @throws PackageManagerException in case of other problems
+     */
+    protected Path copyLocalFile(URL fileUrl) throws IOException, PackageManagerException {
+        Path destination = Files.createTempDirectory("pf4j-update-downloader");
+        destination.toFile().deleteOnExit();
+
+        try {
+            Path fromFile = Paths.get(fileUrl.toURI());
+            String path = fileUrl.getPath();
+            String fileName = path.substring(path.lastIndexOf('/') + 1);
+            Path toFile = destination.resolve(fileName);
+            Files.copy(fromFile, toFile, StandardCopyOption.COPY_ATTRIBUTES, StandardCopyOption.REPLACE_EXISTING);
+
+            return toFile;
+        } catch (URISyntaxException e) {
+            throw new PackageManagerException("Something wrong with given URL", e);
+        }
+    }
+
+    /**
+     * Downloads file from HTTP or FTP.
+     *
+     * @param fileUrl source file
+     * @return path of downloaded file
+     * @throws IOException if IO problems
+     * @throws PackageManagerException if validation fails or any other problems
+     */
+    protected Path downloadFileHttp(URL fileUrl) throws IOException, PackageManagerException {
+        Path destination = Files.createTempDirectory("pf4j-update-downloader");
+        destination.toFile().deleteOnExit();
+
+        String path = fileUrl.getPath();
+        String fileName = path.substring(path.lastIndexOf('/') + 1);
+        Path file = destination.resolve(fileName);
+
+        // set up the URL connection
+        URLConnection connection = fileUrl.openConnection();
+
+        // connect to the remote site (may takes some time)
+        connection.connect();
+
+        // check for http authorization
+        HttpURLConnection httpConnection = (HttpURLConnection) connection;
+        if (httpConnection.getResponseCode() == HttpURLConnection.HTTP_UNAUTHORIZED) {
+            throw new ConnectException("HTTP Authorization failure");
+        }
+
+        // try to get the server-specified last-modified date of this artifact
+        long lastModified = httpConnection.getHeaderFieldDate("Last-Modified", System.currentTimeMillis());
+
+        // try to get the input stream (three times)
+        InputStream is = null;
+        for (int i = 0; i < 3; i++) {
+            try {
+                is = connection.getInputStream();
+                break;
+            } catch (IOException e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+        if (is == null) {
+            throw new ConnectException("Can't get '" + fileUrl + " to '" + file + "'");
+        }
+
+        // reade from remote resource and write to the local file
+        FileOutputStream fos = new FileOutputStream(file.toFile());
+        byte[] buffer = new byte[1024];
+        int length;
+        while ((length = is.read(buffer)) >= 0) {
+            fos.write(buffer, 0, length);
+        }
+        fos.close();
+        is.close();
+
+        log.debug("Set last modified of '{}' to '{}'", file, lastModified);
+        Files.setLastModifiedTime(file, FileTime.fromMillis(lastModified));
+
+        return file;
+    }
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/StringUtils.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/StringUtils.java
new file mode 100644
index 0000000..a1a3ad3
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/StringUtils.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2012 Decebal Suiu
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+/**
+ * @author Decebal Suiu
+ */
+public class StringUtils {
+
+	public static boolean isNullOrEmpty(String str) {
+		return (str == null) || str.isEmpty();
+	}
+
+    public static boolean isNotNullOrEmpty(String str) {
+        return !isNullOrEmpty(str);
+    }
+
+    /**
+     * Format the string. Replace "{}" with %s and format the string using {@link String#format(String, Object...)}.
+     */
+    public static String format(String str, Object... args) {
+        str = str.replaceAll("\\{}", "%s");
+
+        return String.format(str, args);
+    }
+
+    public static String addStart(String str, String add) {
+        if (isNullOrEmpty(add)) {
+            return str;
+        }
+
+        if (isNullOrEmpty(str)) {
+            return add;
+        }
+
+        if (!str.startsWith(add)) {
+            return add + str;
+        }
+
+        return str;
+    }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/pf4j/VerifyException.java b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/VerifyException.java
new file mode 100644
index 0000000..e893963
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/packagemanager/pf4j/VerifyException.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright (C) 2012-present the original author or authors.
+ *
+ * Licensed 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.packagemanager.pf4j;
+
+/**
+ * Marker exception for plugin verification failure
+ */
+public class VerifyException extends PackageManagerException {
+    public VerifyException(String message) {
+        super(message);
+    }
+
+    public VerifyException(Throwable cause, String message, Object... args) {
+        super(cause, message, args);
+    }
+
+    public VerifyException(String message, Object... args) {
+        super(message, args);
+    }
+}
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java b/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
index 4bd822e..fa9f952 100644
--- a/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
@@ -169,12 +169,20 @@ public class PackageAPI {
     @JsonProperty
     public List<String> files;
 
+    @JsonProperty
+    public String manifest;
+
+    @JsonProperty
+    public String manifestSHA512;
+
     public PkgVersion() {
     }
 
     public PkgVersion(AddVersion addVersion) {
       this.version = addVersion.version;
       this.files = addVersion.files;
+      this.manifest = addVersion.manifest;
+      this.manifestSHA512 = addVersion.manifestSHA512; // nocommit: compute and populate here
     }
 
 
@@ -344,7 +352,10 @@ public class PackageAPI {
     public String version;
     @JsonProperty(required = true)
     public List<String> files;
-
+    @JsonProperty(required = false)
+    public String manifestSHA512;
+    @JsonProperty(required = false)
+    public String manifest;
   }
 
   public static class DelVersion implements ReflectMapWriter {
diff --git a/solr/core/src/java/org/apache/solr/util/PackageTool.java b/solr/core/src/java/org/apache/solr/util/PackageTool.java
new file mode 100644
index 0000000..f5491e6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/PackageTool.java
@@ -0,0 +1,302 @@
+/*
+ * 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.io.IOUtils;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.Configurator;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.packagemanager.SolrPackage;
+import org.apache.solr.packagemanager.SolrPackage.SolrPackageRelease;
+import org.apache.solr.packagemanager.SolrPackageInstance;
+import org.apache.solr.packagemanager.SolrPackageManager;
+import org.apache.solr.packagemanager.SolrPackageRepository;
+import org.apache.solr.packagemanager.SolrUpdateManager;
+import org.apache.solr.packagemanager.pf4j.PackageManagerException;
+import org.apache.solr.util.SolrCLI.StatusTool;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+
+public class PackageTool extends SolrCLI.ToolBase {
+  @Override
+  public String getName() {
+    return "package";
+  }
+
+  public static String solrUrl = null;
+
+  public SolrPackageManager packageManager;
+  public SolrUpdateManager updateManager;
+
+  @Override
+  protected void runImpl(CommandLine cli) throws Exception {
+    // Need a logging free, clean output going through to the user.
+    Configurator.setRootLevel(Level.OFF);
+
+    solrUrl = cli.getOptionValues("solrUrl")[cli.getOptionValues("solrUrl").length-1];
+    String solrBaseUrl = solrUrl.replaceAll("\\/solr$", ""); // strip out ending "/solr"
+    System.out.println("solr url: "+solrUrl+", solr base url: "+solrBaseUrl);
+
+    String zkHost = getZkHost(cli);
+
+    System.out.println("ZK: "+zkHost);
+    String cmd = cli.getArgs()[0];
+
+    try (SolrZkClient zkclient = new SolrZkClient(zkHost, 30000)) {
+      if (cmd != null) {
+        packageManager = new SolrPackageManager(new File("./plugins"), solrBaseUrl, zkHost); 
+        try {
+          updateManager = new SolrUpdateManager(packageManager,
+              getRepositoriesJson(zkclient), solrBaseUrl);
+
+          switch (cmd) {
+            case "add-repo":
+              addRepo(zkHost, cli.getArgs()[1], cli.getArgs()[2]);
+              break;
+            case "list":
+              list(cli.getArgList().subList(1, cli.getArgList().size()));
+              break;
+            case "list-available":
+              try {
+                available(cli.getArgList().subList(1, cli.getArgList().size()));
+              } catch (PackageManagerException ex) {
+                ex.printStackTrace();
+              }
+              break;
+            case "install":
+              install(cli.getArgList().subList(1, cli.getArgList().size()));
+              break;
+            case "deploy":
+              String colls[] = cli.getOptionValues("collections");
+              String params[] = cli.getOptionValues("param");
+              System.out.println("coll: "+Arrays.toString(colls)+", params: "+Arrays.toString(params));
+              deploy(cli.getArgList().get(1).toString(), colls, params);
+              break;
+            case "redeploy":
+              redeploy(cli.getArgList().subList(1, cli.getArgList().size()));
+              break;
+            case "update":
+              if (cli.getArgList().size()==1) {
+                update();
+              } else {
+                updatePackage(zkHost, cli.getArgs()[1], cli.getArgList().subList(2, cli.getArgList().size()));
+              }
+              break;
+            default:
+              throw new RuntimeException("Unrecognized command: "+cmd);
+          };
+        } finally {
+          packageManager.close();
+        }
+      }
+    }
+    System.out.println("khatam: "+cmd); // nocommit
+  }
+
+  protected void addRepo(String zkHost, String name, String uri) throws KeeperException, InterruptedException, MalformedURLException, IOException {
+    try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
+      String existingRepositoriesJson = getRepositoriesJson(zkClient);
+      System.out.println(existingRepositoriesJson);
+
+      List repos = new ObjectMapper().readValue(existingRepositoriesJson, List.class);
+      repos.add(new SolrPackageRepository(name, uri));
+      if (zkClient.exists("/repositories.json", true) == false) {
+        zkClient.create("/repositories.json", new ObjectMapper().writeValueAsString(repos).getBytes(), CreateMode.PERSISTENT, true);
+      } else {
+        zkClient.setData("/repositories.json", new ObjectMapper().writeValueAsString(repos).getBytes(), true);
+      }
+
+      if (zkClient.exists("/keys", true)==false) zkClient.create("/keys", new byte[0], CreateMode.PERSISTENT, true);
+      if (zkClient.exists("/keys/exe", true)==false) zkClient.create("/keys/exe", new byte[0], CreateMode.PERSISTENT, true);
+      if (zkClient.exists("/keys/exe/"+"pub_key.der", true)==false) zkClient.create("/keys/exe/"+"pub_key.der", new byte[0], CreateMode.PERSISTENT, true);
+      zkClient.setData("/keys/exe/"+"pub_key.der", IOUtils.toByteArray(new URL(uri+"/publickey.der").openStream()), true);
+
+      System.out.println("Added repository: "+name);
+      System.out.println(getRepositoriesJson(zkClient));
+    }
+  }
+
+  protected String getRepositoriesJson(SolrZkClient zkClient) throws UnsupportedEncodingException, KeeperException, InterruptedException {
+    if (zkClient.exists("/repositories.json", true)) {
+      return new String(zkClient.getData("/repositories.json", null, null, true), "UTF-8");
+    }
+    return "[]";
+  }
+
+  protected void list(List args) {
+    for (SolrPackageInstance pkg: packageManager.getPackages()) {
+      System.out.println(pkg.getPluginId()+" ("+pkg.getVersion()+")");
+    }
+  }
+  protected void available(List args) throws PackageManagerException {
+    System.out.println("Available packages:\n-----");
+    for (SolrPackage i: updateManager.getPackages()) {
+      SolrPackage plugin = (SolrPackage)i;
+      System.out.println(plugin.id + " \t\t"+plugin.description);
+      for (SolrPackageRelease version: plugin.versions) {
+        System.out.println("\tVersion: "+version.version);
+      }
+    }
+  }
+  protected void install(List args) throws PackageManagerException {
+    updateManager.installPackage(args.get(0).toString(), args.get(1).toString());
+    System.out.println(args.get(0).toString() + " installed.");
+  }
+  protected void deploy(String packageName,
+      String collections[], String parameters[]) throws PackageManagerException {
+    
+    System.out.println(packageManager.deployInstallPackage(packageName.split(":")[0], packageName.split(":").length==2? packageName.split(":")[1]: "latest",
+        Arrays.asList(collections), parameters));
+  }
+
+  protected void redeploy(List args) throws PackageManagerException {
+    throw new UnsupportedOperationException("redeploy not supported");
+    //System.out.println(packageManager.deployUpdatePackage(args.get(0).toString(), args.subList(1, args.size())));
+  }
+
+  protected void update() throws PackageManagerException {
+    if (updateManager.hasUpdates()) {
+      System.out.println("Available updates:\n-----");
+
+      for (SolrPackage i: updateManager.getUpdates()) {
+        SolrPackage plugin = (SolrPackage)i;
+        System.out.println(plugin.id + " \t\t"+plugin.description);
+        for (SolrPackageRelease version: plugin.versions) {
+          System.out.println("\tVersion: "+version.version);
+        }
+      }
+    } else {
+      System.out.println("No updates found. System is up to date.");
+    }
+  }
+
+  protected void updatePackage(String zkHost, String packageName, List args) throws PackageManagerException {
+    if (updateManager.hasUpdates()) {
+      String latestVersion = updateManager.getLastPackageRelease(packageName).version;
+      SolrPackageInstance installedPackage = packageManager.getPackage(packageName, "latest");
+      System.out.println("Updating ["+packageName+"] from " + installedPackage.getVersion() + " to version "+latestVersion);
+
+      List<String> collectionsDeployedIn = getDeployedCollections(zkHost, packageManager, installedPackage);
+      System.out.println("Already deployed on collections: "+collectionsDeployedIn);
+      updateManager.updatePackage(packageName, latestVersion);
+
+      SolrPackageInstance updatedPackage = packageManager.getPackage(packageName, "latest");
+      boolean res = packageManager.verify(updatedPackage, collectionsDeployedIn);
+      System.out.println("Verifying version "+updatedPackage.getVersion()+" on "+collectionsDeployedIn
+          +", result: "+res);
+      if (!res) throw new PackageManagerException("Failed verification after deployment");
+    } else {
+      System.out.println("Package "+packageName+" is already up to date.");
+    }
+  }
+
+  private List<String> getDeployedCollections(String zkHost, SolrPackageManager packageManager, SolrPackageInstance pkg) {
+
+    List<String> allCollections;
+    try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
+      allCollections = zkClient.getChildren("/collections", null, true);
+    } catch (KeeperException | InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+    System.out.println("Need to verify if these collections have the plugin installed? "+ allCollections);
+    List<String> deployed = new ArrayList<String>();
+    for (String collection: allCollections) {
+      if (packageManager.verify(pkg, Collections.singletonList(collection))) {
+        deployed.add(collection);
+      }
+    }
+    return deployed;
+  }
+
+  @SuppressWarnings("static-access")
+  public Option[] getOptions() {
+    return new Option[] {
+        OptionBuilder
+        .withArgName("URL")
+        .hasArg()
+        .isRequired(true)
+        .withDescription("Address of the Solr Web application, defaults to: "+SolrCLI.DEFAULT_SOLR_URL)
+        .create("solrUrl"),
+
+        OptionBuilder
+        .withArgName("COLLECTIONS")
+        .hasArgs()
+        .isRequired(false)
+        .withDescription("Solr URL scheme: http or https, defaults to http if not specified")
+        .create("collections"),
+
+        OptionBuilder
+        .withArgName("PARAMS")
+        .hasArgs()
+        .isRequired(false)
+        .withDescription("Solr URL scheme: http or https, defaults to http if not specified")
+        .withLongOpt("param")
+        .create("p"),
+
+    };
+  }
+
+  private String getZkHost(CommandLine cli) throws Exception {
+    String zkHost = cli.getOptionValue("zkHost");
+    if (zkHost != null)
+      return zkHost;
+
+    // find it using the localPort
+
+    String systemInfoUrl = solrUrl+"/admin/info/system";
+    CloseableHttpClient httpClient = SolrCLI.getHttpClient();
+    try {
+      // hit Solr to get system info
+      Map<String,Object> systemInfo = SolrCLI.getJson(httpClient, systemInfoUrl, 2, true);
+
+      // convert raw JSON into user-friendly output
+      StatusTool statusTool = new StatusTool();
+      Map<String,Object> status = statusTool.reportStatus(solrUrl+"/", systemInfo, httpClient);
+      Map<String,Object> cloud = (Map<String, Object>)status.get("cloud");
+      if (cloud != null) {
+        String zookeeper = (String) cloud.get("ZooKeeper");
+        if (zookeeper.endsWith("(embedded)")) {
+          zookeeper = zookeeper.substring(0, zookeeper.length() - "(embedded)".length());
+        }
+        zkHost = zookeeper;
+      }
+    } finally {
+      HttpClientUtil.close(httpClient);
+    }
+
+    return zkHost;
+  }
+}
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index bcaa9b8..3ed316d 100755
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -418,6 +418,8 @@ public class SolrCLI implements CLIO {
       return new AutoscalingTool();
     else if ("export".equals(toolType))
       return new ExportTool();
+    else if ("package".equals(toolType))
+      return new PackageTool();
 
     // If you add a built-in tool to this class, add it here to avoid
     // classpath scanning
diff --git a/solr/core/src/test/org/apache/solr/cloud/PackageManagerCLITest.java b/solr/core/src/test/org/apache/solr/cloud/PackageManagerCLITest.java
new file mode 100644
index 0000000..99286df
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/PackageManagerCLITest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.cloud;
+
+import java.nio.file.Path;
+
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.util.PackageTool;
+import org.apache.solr.util.SolrCLI;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class PackageManagerCLITest extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    System.setProperty("enable.packages", "true");
+
+    configureCluster(1)
+        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+  }
+
+  @Test
+  public void testUpconfig() throws Exception {
+    // Use a full, explicit path for configset.
+
+    Path configSet = TEST_PATH().resolve("configsets");
+    Path srcPathCheck = configSet.resolve("cloud-subdirs").resolve("conf");
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "upconfig1", cluster.getZkServer().getZkAddress());
+
+    // Now just use a name in the configsets directory, do we find it?
+    configSet = TEST_PATH().resolve("configsets");
+
+    PackageTool tool = new PackageTool();
+    String solrUrl = cluster.getJettySolrRunner(0).getBaseUrl().toString();
+    int res = run(tool, new String[] {"-solrUrl", solrUrl, "list"});
+    assertEquals("tool should have returned 0 for success ", 0, res);
+    
+    res = run(tool, new String[] {"-solrUrl", solrUrl, "add-repo", "fullstory",  "http://localhost:8081"});
+    assertEquals("tool should have returned 0 for success ", 0, res);
+
+    res = run(tool, new String[] {"-solrUrl", solrUrl, "list-available"});
+    assertEquals("tool should have returned 0 for success ", 0, res);
+
+    res = run(tool, new String[] {"-solrUrl", solrUrl, "install", "question-answer", "1.0.0"}); // no-commit (change to pkg:ver syntax)
+    assertEquals("tool should have returned 0 for success ", 0, res);
+    
+    res = run(tool, new String[] {"-solrUrl", solrUrl, "list"});
+    assertEquals("tool should have returned 0 for success ", 0, res);
+
+    CollectionAdminRequest
+      .createCollection("abc", "conf1", 2, 1)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+
+    res = run(tool, new String[] {"-solrUrl", solrUrl, "deploy", "question-answer", "-collections", "abc", "-p", "RH-HANDLER-PATH=/mypath2"});
+    assertEquals("tool should have returned 0 for success ", 0, res);
+    
+    res = run(tool, new String[] {"-solrUrl", solrUrl, "update", "question-answer"});
+    assertEquals("tool should have returned 0 for success ", 0, res);
+
+  }
+
+  private int run(PackageTool tool, String[] args) throws Exception {
+    int res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
+    return res;
+  }
+}


[lucene-solr] 02/05: Latest auto-update functionality

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-13662-updated
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 11f1e9fa0902b7b46f47b0aa979cbe013af9e556
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Mon Oct 28 21:40:56 2019 +0530

    Latest auto-update functionality
---
 .../solr/packagemanager/SolrPackageManager.java    | 55 +++++++++++++++-------
 .../src/java/org/apache/solr/util/PackageTool.java | 38 +++++++++++++--
 .../apache/solr/cloud/PackageManagerCLITest.java   | 39 +++++++++++++--
 3 files changed, 106 insertions(+), 26 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java
index e351a68..27fb010 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java
@@ -98,23 +98,36 @@ public class SolrPackageManager implements Closeable {
     return ret;
   }
 
-  public boolean deployInstallPackage(String packageName, String version, List<String> collections, String overrides[]) {
+  Map<String, String> getPackageParams(String packageName, String collection) {
+    try {
+      return (Map<String, String>)((Map)((Map)((Map)new ObjectMapper().readValue
+          (get(solrBaseUrl + "/api/collections/"+collection+"/config/params/packages"), Map.class).get("response")).get("params")).get("packages")).get(packageName);
+    } catch (IOException e) {
+      throw new PackageManagerException(e);
+    }
+
+  }
+  
+  public boolean deployInstallPackage(String packageName, String version, boolean isUpdate, List<String> collections, String overrides[]) {
+    boolean pegToLatest = "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 pkg = getPackage(packageName, version);
+    if (version == null) {
+      version = pkg.getVersion();
+    }
 
     for (String collection: collections) {
-      Map<String, String> collectionParameterOverrides = new HashMap<String,String>();
+      Map<String, String> collectionParameterOverrides = isUpdate? getPackageParams(packageName, collection): new HashMap<String,String>();
       if (overrides != null) {
         for (String override: overrides) {
           collectionParameterOverrides.put(override.split("=")[0], override.split("=")[1]);
         }
       }
+      
+      // Get package params
       try {
-        // nocommit: it overwrites params of other packages (use set or update)
-        
         boolean packageParamsExist = ((Map)((Map)new ObjectMapper().readValue(
             get(solrBaseUrl + "/api/collections/abc/config/params/packages"), Map.class)
             ).get("response")).containsKey("params");
-        
         postJson(solrBaseUrl + "/api/collections/"+collection+"/config/params",
             new ObjectMapper().writeValueAsString(
                 Map.of(packageParamsExist? "update": "set", 
@@ -123,20 +136,33 @@ public class SolrPackageManager implements Closeable {
         throw new RuntimeException(e);
       }
 
+      // Set the package version in the collection's parameters
+      postJson(solrBaseUrl+"/api/collections/abc/config/params", "{set:{PKG_VERSIONS:{"+packageName+" : '"+(pegToLatest? "$LATEST": version)+"'}}}");
+
+      // If updating, refresh the package version for this to take effect
+      if (isUpdate || pegToLatest) {
+        postJson(solrBaseUrl+"/api/cluster/package", "{\"refresh\" : \""+packageName+"\"}");
+      }
+      
+      // Setup/update all the plugins in the package
       for (Plugin p: pkg.getPlugins()) {
-        System.out.println(p.setupCommand);
+        System.out.println(isUpdate? p.updateCommand: p.setupCommand);
 
         Map<String, String> systemParams = new HashMap<String,String>();
         systemParams.put("collection", collection);
         systemParams.put("package-name", pkg.id);
         systemParams.put("package-version", pkg.version);
 
-        String cmd = resolve(p.setupCommand, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
-        System.out.println("Executing " + cmd + " for collection:" + collection);
-        postJson(solrBaseUrl + "/solr/"+collection+"/config", cmd);
+        String cmd = resolve(isUpdate? p.updateCommand: p.setupCommand, pkg.parameterDefaults, collectionParameterOverrides, systemParams);
+        if (cmd != null && !"".equals(cmd.trim())) {
+          System.out.println("Executing " + cmd + " for collection:" + collection);
+          postJson(solrBaseUrl + "/solr/"+collection+"/config", cmd);
+        }
       }
+
     }
 
+    // Verify that package was successfully deployed
     boolean success = verify(pkg, collections);
     if (success) {
       System.out.println("Deployed and verified package: "+pkg.id+", version: "+pkg.version);
@@ -145,6 +171,7 @@ public class SolrPackageManager implements Closeable {
   }
 
   private String resolve(String str, Map<String, String> defaults, Map<String, String> overrides, Map<String, String> systemParams) {
+    if (str == null) return null;
     for (String param: defaults.keySet()) {
       str = str.replaceAll("\\$\\{"+param+"\\}", overrides.containsKey(param)? overrides.get(param): defaults.get(param));
     }
@@ -164,13 +191,7 @@ public class SolrPackageManager implements Closeable {
       System.out.println(p.verifyCommand);
       for (String collection: collections) {
         System.out.println("Executing " + p.verifyCommand + " for collection:" + collection);
-        Map<String, String> collectionParameterOverrides;
-        try {
-          collectionParameterOverrides = (Map<String, String>)((Map)((Map)((Map)new ObjectMapper().readValue
-              (get(solrBaseUrl + "/api/collections/abc/config/params/packages"), Map.class).get("response")).get("params")).get("packages")).get(pkg.id);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
+        Map<String, String> collectionParameterOverrides = getPackageParams(pkg.id, collection);
         
         Command cmd = p.verifyCommand;
 
@@ -217,7 +238,7 @@ public class SolrPackageManager implements Closeable {
     return get(url);
   }
 
-  private String get(String url) {
+  public static String get(String url) {
     try (CloseableHttpClient client = SolrUpdateManager.createTrustAllHttpClientBuilder()) {
       HttpGet httpGet = new HttpGet(url);
       httpGet.setHeader("Content-type", "application/json");
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 f5491e6..e12af04 100644
--- a/solr/core/src/java/org/apache/solr/util/PackageTool.java
+++ b/solr/core/src/java/org/apache/solr/util/PackageTool.java
@@ -48,6 +48,8 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.PathNotFoundException;
 
 
 public class PackageTool extends SolrCLI.ToolBase {
@@ -64,7 +66,7 @@ public class PackageTool extends SolrCLI.ToolBase {
   @Override
   protected void runImpl(CommandLine cli) throws Exception {
     // Need a logging free, clean output going through to the user.
-    Configurator.setRootLevel(Level.OFF);
+    Configurator.setRootLevel(Level.INFO);
 
     solrUrl = cli.getOptionValues("solrUrl")[cli.getOptionValues("solrUrl").length-1];
     String solrBaseUrl = solrUrl.replaceAll("\\/solr$", ""); // strip out ending "/solr"
@@ -103,7 +105,10 @@ public class PackageTool extends SolrCLI.ToolBase {
               String colls[] = cli.getOptionValues("collections");
               String params[] = cli.getOptionValues("param");
               System.out.println("coll: "+Arrays.toString(colls)+", params: "+Arrays.toString(params));
-              deploy(cli.getArgList().get(1).toString(), colls, params);
+              String packageName = cli.getArgList().get(1).toString().split(":")[0];
+              String version = cli.getArgList().get(1).toString().contains(":")? 
+                  cli.getArgList().get(1).toString().split(":")[1]: null;
+              deploy(packageName, version, cli.hasOption("update"), colls, params);
               break;
             case "redeploy":
               redeploy(cli.getArgList().subList(1, cli.getArgList().size()));
@@ -175,10 +180,10 @@ public class PackageTool extends SolrCLI.ToolBase {
     updateManager.installPackage(args.get(0).toString(), args.get(1).toString());
     System.out.println(args.get(0).toString() + " installed.");
   }
-  protected void deploy(String packageName,
+  protected void deploy(String packageName, String version, boolean isUpdate,
       String collections[], String parameters[]) throws PackageManagerException {
     
-    System.out.println(packageManager.deployInstallPackage(packageName.split(":")[0], packageName.split(":").length==2? packageName.split(":")[1]: "latest",
+    System.out.println(packageManager.deployInstallPackage(packageName, version, isUpdate,
         Arrays.asList(collections), parameters));
   }
 
@@ -234,8 +239,19 @@ public class PackageTool extends SolrCLI.ToolBase {
     System.out.println("Need to verify if these collections have the plugin installed? "+ allCollections);
     List<String> deployed = new ArrayList<String>();
     for (String collection: allCollections) {
-      if (packageManager.verify(pkg, Collections.singletonList(collection))) {
+      // Check package version installed
+      // http://localhost:8983/api/collections/abc/config/params/PKG_VERSIONS?omitHeader=true
+      String paramsJson = SolrPackageManager.get("http://localhost:8983/api/collections/"+collection+"/config/params/PKG_VERSIONS?omitHeader=true");
+      String version = null;
+      try {
+        version = JsonPath.parse(paramsJson).read("$['response'].['params'].['PKG_VERSIONS'].['"+pkg.id+"'])");
+      } catch (PathNotFoundException ex) {
+        // Don't worry if PKG_VERSION wasn't found. It just means this collection was never touched by the package manager.
+      }
+      if ("$LATEST".equals(version) && packageManager.verify(pkg, Collections.singletonList(collection))) {
         deployed.add(collection);
+      } else {
+        System.out.println("Skipping collection: "+collection+", version: "+version);
       }
     }
     return deployed;
@@ -266,6 +282,18 @@ public class PackageTool extends SolrCLI.ToolBase {
         .withLongOpt("param")
         .create("p"),
 
+        OptionBuilder
+        .isRequired(false)
+        .withDescription("Solr URL scheme: http or https, defaults to http if not specified")
+        .withLongOpt("update")
+        .create("u"),
+
+        OptionBuilder
+        .isRequired(false)
+        .withDescription("Solr URL scheme: http or https, defaults to http if not specified")
+        .withLongOpt("auto-update")
+        .create(),
+        
     };
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/PackageManagerCLITest.java b/solr/core/src/test/org/apache/solr/cloud/PackageManagerCLITest.java
index 99286df..b918d2b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/PackageManagerCLITest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/PackageManagerCLITest.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.cloud;
 
+import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@@ -24,9 +25,13 @@ import org.apache.solr.util.PackageTool;
 import org.apache.solr.util.SolrCLI;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class PackageManagerCLITest extends SolrCloudTestCase {
 
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     System.setProperty("enable.packages", "true");
@@ -65,16 +70,42 @@ public class PackageManagerCLITest extends SolrCloudTestCase {
     assertEquals("tool should have returned 0 for success ", 0, res);
 
     CollectionAdminRequest
-      .createCollection("abc", "conf1", 2, 1)
+      .createCollection("abc", "conf1", 1, 1)
       .setMaxShardsPerNode(100)
       .process(cluster.getSolrClient());
 
+    CollectionAdminRequest
+    .createCollection("def", "conf1", 1, 1)
+    .setMaxShardsPerNode(100)
+    .process(cluster.getSolrClient());
+
     res = run(tool, new String[] {"-solrUrl", solrUrl, "deploy", "question-answer", "-collections", "abc", "-p", "RH-HANDLER-PATH=/mypath2"});
     assertEquals("tool should have returned 0 for success ", 0, res);
     
-    res = run(tool, new String[] {"-solrUrl", solrUrl, "update", "question-answer"});
-    assertEquals("tool should have returned 0 for success ", 0, res);
-
+    // Should we test the "auto-update to latest" functionality or the default explicit deploy functionality
+    boolean autoUpdateToLatest = random().nextBoolean();
+    
+    if (autoUpdateToLatest) {
+      log.info("Testing auto-update to latest installed");
+      // This command pegs the version to the latest available
+      res = run(tool, new String[] {"-solrUrl", solrUrl, "deploy", "question-answer:latest", "-collections", "abc"});
+      assertEquals("tool should have returned 0 for success ", 0, res);
+      
+      res = run(tool, new String[] {"-solrUrl", solrUrl, "update", "question-answer"});
+      assertEquals("tool should have returned 0 for success ", 0, res);
+    } else {
+      log.info("Testing explicit deployment to a different/newer version");
+
+      res = run(tool, new String[] {"-solrUrl", solrUrl, "update", "question-answer"});
+      assertEquals("tool should have returned 0 for success ", 0, res);
+
+      if (random().nextBoolean()) {
+        res = run(tool, new String[] {"-solrUrl", solrUrl, "deploy", "--update", "question-answer", "-collections", "abc", "-p", "RH-HANDLER-PATH=/mypath2"});
+      } else {
+        res = run(tool, new String[] {"-solrUrl", solrUrl, "deploy", "--update", "question-answer", "-collections", "abc"});
+      }
+      assertEquals("tool should have returned 0 for success ", 0, res);      
+    }
   }
 
   private int run(PackageTool tool, String[] args) throws Exception {


[lucene-solr] 03/05: Merge branch 'master' into jira/solr-13662-updated

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-13662-updated
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit be27e8cd9cf83af333f02042b4574574af9087e9
Merge: 11f1e9f e2b160b
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Mon Oct 28 21:41:23 2019 +0530

    Merge branch 'master' into jira/solr-13662-updated

 dev-tools/maven/pom.xml.template                   |   2 +-
 lucene/CHANGES.txt                                 |  26 +-
 .../analysis/query/QueryAutoStopWordAnalyzer.java  |   8 +-
 .../analysis/ja/dict/TokenInfoDictionary$fst.dat   | Bin 1698570 -> 1698570 bytes
 .../analysis/ko/dict/TokenInfoDictionary$fst.dat   | Bin 5641400 -> 5640903 bytes
 .../lucene/benchmark/byTask/utils/Config.java      |  19 +-
 .../blocktreeords/OrdsBlockTreeTermsReader.java    |   3 +-
 .../codecs/bloom/BloomFilteringPostingsFormat.java |   3 +-
 .../lucene/codecs/memory/FSTOrdTermsReader.java    |   3 +-
 .../lucene/codecs/memory/FSTTermsReader.java       |   3 +-
 lucene/common-build.xml                            |   4 +-
 .../codecs/blocktree/BlockTreeTermsReader.java     |   3 +-
 .../LatLonDocValuesPointInPolygonQuery.java        |   5 +-
 .../lucene/document/LatLonPointInPolygonQuery.java |  21 +-
 .../java/org/apache/lucene/geo/Component2D.java    |  96 ++++
 .../java/org/apache/lucene/geo/ComponentTree.java  | 206 +++++++++
 .../src/java/org/apache/lucene/geo/EdgeTree.java   | 503 ++++++++-------------
 .../org/apache/lucene/geo/GeoEncodingUtils.java    |  17 +-
 .../src/java/org/apache/lucene/geo/Polygon2D.java  | 219 ++++-----
 .../apache/lucene/index/BaseCompositeReader.java   |  24 +-
 .../java/org/apache/lucene/index/IndexReader.java  |   7 +-
 .../org/apache/lucene/search/BooleanQuery.java     |  11 +-
 .../src/java/org/apache/lucene/util/SetOnce.java   |  36 +-
 .../java/org/apache/lucene/util/fst/Builder.java   |   3 -
 .../src/java/org/apache/lucene/util/fst/FST.java   | 106 +----
 .../test/org/apache/lucene/geo/TestPolygon2D.java  |  88 ++--
 .../lucene/index/TestFilterDirectoryReader.java    |  72 +++
 .../test/org/apache/lucene/util/TestSetOnce.java   |   9 +
 lucene/ivy-versions.properties                     |  12 +-
 .../monitor/MultipassTermFilteredPresearcher.java  |   4 +-
 .../lucene/monitor/TermFilteredPresearcher.java    |   4 +-
 .../builders/MultiPhraseQueryNodeBuilder.java      |  10 +-
 .../lucene/replicator/IndexReplicationHandler.java |   3 +-
 .../idversion/VersionBlockTreeTermsReader.java     |   3 +-
 .../lucene/document/LatLonShapeLineQuery.java      |   5 +-
 .../lucene/document/LatLonShapePolygonQuery.java   |   5 +-
 .../apache/lucene/document/XYShapeLineQuery.java   |  13 +-
 .../lucene/document/XYShapePolygonQuery.java       |   6 +-
 .../src/java/org/apache/lucene/geo/Line2D.java     | 126 ++++--
 .../java/org/apache/lucene/geo/XYPolygon2D.java    |  10 +-
 .../lucene/document/BaseLatLonShapeTestCase.java   |   5 +-
 .../lucene/document/BaseShapeEncodingTestCase.java |   6 +-
 .../apache/lucene/document/BaseShapeTestCase.java  |  17 +-
 .../lucene/document/BaseXYShapeTestCase.java       |   5 +-
 .../document/TestLatLonLineShapeQueries.java       |  19 +-
 .../document/TestLatLonMultiLineShapeQueries.java  |  22 +-
 .../document/TestLatLonMultiPointShapeQueries.java |  22 +-
 .../TestLatLonMultiPolygonShapeQueries.java        |  22 +-
 .../document/TestLatLonPointShapeQueries.java      |  18 +-
 .../document/TestLatLonPolygonShapeQueries.java    |  18 +-
 .../apache/lucene/document/TestLatLonShape.java    |  11 +-
 .../lucene/document/TestLatLonShapeEncoding.java   |   3 +-
 .../lucene/document/TestXYLineShapeQueries.java    |  19 +-
 .../document/TestXYMultiLineShapeQueries.java      |  22 +-
 .../document/TestXYMultiPointShapeQueries.java     |  22 +-
 .../document/TestXYMultiPolygonShapeQueries.java   |  22 +-
 .../lucene/document/TestXYPointShapeQueries.java   |  18 +-
 .../lucene/document/TestXYPolygonShapeQueries.java |  18 +-
 .../lucene/document/TestXYShapeEncoding.java       |   3 +-
 .../src/test/org/apache/lucene/geo/TestLine2D.java |  18 +-
 .../lucene/spatial3d/geom/StandardObjects.java     |   4 +-
 .../search/suggest/document/ContextQuery.java      |   6 +-
 .../dependencies/GetMavenDependenciesTask.java     |  22 +-
 solr/CHANGES.txt                                   |  13 +-
 .../solr/analytics/AnalyticsRequestManager.java    |   6 +-
 .../solr/handler/dataimport/RegexTransformer.java  |   3 +-
 .../solr/prometheus/collector/MetricSamples.java   |   7 +-
 .../solr/response/VelocityResponseWriter.java      |   5 +-
 .../client/solrj/embedded/JettySolrRunner.java     |  10 +-
 .../apache/solr/cloud/OverseerTaskProcessor.java   |   6 +-
 .../java/org/apache/solr/cloud/ZkController.java   |   7 +-
 .../java/org/apache/solr/cloud/ZkShardTerms.java   |   5 +-
 .../solr/cloud/api/collections/CreateShardCmd.java |  10 +-
 .../cloud/api/collections/DeleteReplicaCmd.java    |   5 +-
 .../OverseerCollectionMessageHandler.java          |   8 +-
 .../api/collections/ReindexCollectionCmd.java      |   3 +-
 .../solr/cloud/api/collections/SplitShardCmd.java  |   7 +-
 .../autoscaling/sim/SimNodeStateProvider.java      |   3 +-
 .../src/java/org/apache/solr/core/SolrCores.java   |  11 +-
 .../solr/handler/admin/CoreAdminHandler.java       |   6 +-
 .../handler/admin/SegmentsInfoRequestHandler.java  |   3 +-
 .../solr/handler/component/SearchHandler.java      |   3 +-
 .../solr/handler/component/TermsComponent.java     |  16 +-
 .../apache/solr/handler/loader/CSVLoaderBase.java  |   6 +-
 .../solr/response/PHPSerializedResponseWriter.java |  11 +-
 .../src/java/org/apache/solr/rest/RestManager.java |   5 +-
 .../analysis/ManagedSynonymFilterFactory.java      |  23 +-
 .../analysis/ManagedSynonymGraphFilterFactory.java |   6 +-
 .../solr/schema/FileExchangeRateProvider.java      |   6 +-
 .../TopGroupsShardRequestFactory.java              |   3 +-
 .../SearchGroupShardResponseProcessor.java         |  10 +-
 .../TopGroupsShardResponseProcessor.java           |  18 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  29 +-
 .../AddSchemaFieldsUpdateProcessorFactory.java     |  14 +-
 .../CloneFieldUpdateProcessorFactory.java          |   4 +-
 .../java/org/apache/solr/util/SimplePostTool.java  |   6 +-
 .../src/java/org/apache/solr/util/SolrCLI.java     |   3 +-
 .../java/org/apache/solr/util/SolrPluginUtils.java |   8 +-
 .../test/org/apache/solr/cloud/SplitShardTest.java |  95 ++--
 .../solr/cloud/TestQueryingOnDownCollection.java   | 159 +++++++
 solr/licenses/netty-all-4.0.52.Final.jar.sha1      |   1 -
 solr/licenses/netty-all-4.1.29.Final.jar.sha1      |   1 +
 solr/licenses/netty-buffer-4.1.29.Final.jar.sha1   |   1 +
 solr/licenses/netty-buffer-LICENSE-ASL.txt         | 202 +++++++++
 solr/licenses/netty-buffer-NOTICE.txt              | 223 +++++++++
 solr/licenses/netty-codec-4.1.29.Final.jar.sha1    |   1 +
 solr/licenses/netty-codec-LICENSE-ASL.txt          | 202 +++++++++
 solr/licenses/netty-codec-NOTICE.txt               | 223 +++++++++
 solr/licenses/netty-common-4.1.29.Final.jar.sha1   |   1 +
 solr/licenses/netty-common-LICENSE-ASL.txt         | 202 +++++++++
 solr/licenses/netty-common-NOTICE.txt              | 223 +++++++++
 solr/licenses/netty-handler-4.1.29.Final.jar.sha1  |   1 +
 solr/licenses/netty-handler-LICENSE-ASL.txt        | 202 +++++++++
 solr/licenses/netty-handler-NOTICE.txt             | 223 +++++++++
 solr/licenses/netty-resolver-4.1.29.Final.jar.sha1 |   1 +
 solr/licenses/netty-resolver-LICENSE-ASL.txt       | 202 +++++++++
 solr/licenses/netty-resolver-NOTICE.txt            | 223 +++++++++
 .../licenses/netty-transport-4.1.29.Final.jar.sha1 |   1 +
 solr/licenses/netty-transport-LICENSE-ASL.txt      | 202 +++++++++
 solr/licenses/netty-transport-NOTICE.txt           | 223 +++++++++
 ...ty-transport-native-epoll-4.1.29.Final.jar.sha1 |   1 +
 .../netty-transport-native-epoll-LICENSE-ASL.txt   | 202 +++++++++
 .../netty-transport-native-epoll-NOTICE.txt        | 223 +++++++++
 ...nsport-native-unix-common-4.1.29.Final.jar.sha1 |   1 +
 ...ty-transport-native-unix-common-LICENSE-ASL.txt | 202 +++++++++
 .../netty-transport-native-unix-common-NOTICE.txt  | 223 +++++++++
 solr/solr-ref-guide/src/json-facet-api.adoc        |   1 -
 solr/solrj/ivy.xml                                 |   9 +
 .../client/solrj/impl/BaseCloudSolrClient.java     |   7 +-
 .../solr/client/solrj/impl/CloudSolrClient.java    |   9 +-
 .../org/apache/solr/client/solrj/io/Tuple.java     |   3 +-
 .../client/solrj/io/eval/SetValueEvaluator.java    |   3 +-
 .../client/solrj/io/eval/TermVectorsEvaluator.java |   3 +-
 .../client/solrj/io/graph/ShortestPathStream.java  |   3 +-
 .../solr/client/solrj/io/ops/GroupOperation.java   |   6 +-
 .../client/solrj/io/stream/CloudSolrStream.java    |   6 +-
 .../client/solrj/io/stream/DeepRandomStream.java   |   6 +-
 .../solrj/io/stream/FeaturesSelectionStream.java   |   4 +-
 .../solrj/io/stream/SignificantTermsStream.java    |  10 +-
 .../solr/client/solrj/io/stream/StatsStream.java   |   6 +-
 .../client/solrj/io/stream/TextLogitStream.java    |   4 +-
 .../solr/client/solrj/io/stream/ZplotStream.java   |   6 +-
 .../solrj/request/JavaBinUpdateRequestCodec.java   |  40 +-
 .../solrj/response/schema/SchemaResponse.java      |   9 +-
 .../solr/common/ConditionalKeyMapWriter.java       |  88 ++++
 .../org/apache/solr/common/SolrInputDocument.java  |   5 +
 .../apache/solr/common/params/CommonParams.java    |   4 +-
 .../org/apache/solr/common/util/JavaBinCodec.java  |  11 +-
 .../org/apache/solr/common/util/NamedList.java     |   9 +-
 .../solrj/request/TestUpdateRequestCodec.java      | 125 +++--
 .../solr/cloud/AbstractFullDistribZkTestBase.java  |   3 +-
 151 files changed, 5118 insertions(+), 1255 deletions(-)



[lucene-solr] 05/05: Fix latest auto update

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-13662-updated
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 362705b4be7c31f48d26c1a4deffc3725c3a568f
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Tue Oct 29 03:31:57 2019 +0530

    Fix latest auto update
---
 .../apache/solr/packagemanager/SolrPackageManager.java   | 10 +++++++++-
 solr/core/src/java/org/apache/solr/util/PackageTool.java | 16 +++++++++++-----
 2 files changed, 20 insertions(+), 6 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java
index 27fb010..a59feed 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/SolrPackageManager.java
@@ -137,8 +137,11 @@ public class SolrPackageManager implements Closeable {
       }
 
       // Set the package version in the collection's parameters
-      postJson(solrBaseUrl+"/api/collections/abc/config/params", "{set:{PKG_VERSIONS:{"+packageName+" : '"+(pegToLatest? "$LATEST": version)+"'}}}");
+      postJson(solrBaseUrl+"/api/collections/"+collection+"/config/params", "{set:{PKG_VERSIONS:{"+packageName+" : '"+(pegToLatest? "$LATEST": version)+"'}}}");
 
+      String paramsJson = SolrPackageManager.get("http://localhost:8983/api/collections/"+collection+"/config/params?omitHeader=true");
+      System.out.println("Posted param: "+paramsJson);
+      
       // If updating, refresh the package version for this to take effect
       if (isUpdate || pegToLatest) {
         postJson(solrBaseUrl+"/api/cluster/package", "{\"refresh\" : \""+packageName+"\"}");
@@ -160,6 +163,11 @@ public class SolrPackageManager implements Closeable {
         }
       }
 
+      // Set the package version in the collection's parameters
+      postJson(solrBaseUrl+"/api/collections/"+collection+"/config/params", "{update:{PKG_VERSIONS:{'"+packageName+"' : '"+(pegToLatest? "$LATEST": version)+"'}}}");
+      paramsJson = SolrPackageManager.get("http://localhost:8983/api/collections/"+collection+"/config/params?omitHeader=true");
+      System.out.println("Posted param: "+paramsJson);
+
     }
 
     // Verify that package was successfully deployed
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 e12af04..1f7e2d4 100644
--- a/solr/core/src/java/org/apache/solr/util/PackageTool.java
+++ b/solr/core/src/java/org/apache/solr/util/PackageTool.java
@@ -214,7 +214,7 @@ public class PackageTool extends SolrCLI.ToolBase {
       SolrPackageInstance installedPackage = packageManager.getPackage(packageName, "latest");
       System.out.println("Updating ["+packageName+"] from " + installedPackage.getVersion() + " to version "+latestVersion);
 
-      List<String> collectionsDeployedIn = getDeployedCollections(zkHost, packageManager, installedPackage);
+      List<String> collectionsDeployedIn = getDeployedCollections(zkHost, packageManager, installedPackage, true);
       System.out.println("Already deployed on collections: "+collectionsDeployedIn);
       updateManager.updatePackage(packageName, latestVersion);
 
@@ -228,7 +228,7 @@ public class PackageTool extends SolrCLI.ToolBase {
     }
   }
 
-  private List<String> getDeployedCollections(String zkHost, SolrPackageManager packageManager, SolrPackageInstance pkg) {
+  private List<String> getDeployedCollections(String zkHost, SolrPackageManager packageManager, SolrPackageInstance pkg, boolean onlyLatest) {
 
     List<String> allCollections;
     try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
@@ -248,10 +248,16 @@ public class PackageTool extends SolrCLI.ToolBase {
       } catch (PathNotFoundException ex) {
         // Don't worry if PKG_VERSION wasn't found. It just means this collection was never touched by the package manager.
       }
-      if ("$LATEST".equals(version) && packageManager.verify(pkg, Collections.singletonList(collection))) {
-        deployed.add(collection);
+      if (onlyLatest) {
+        if ("$LATEST".equals(version) && packageManager.verify(pkg, Collections.singletonList(collection))) {
+          deployed.add(collection);
+        } else {
+          System.out.println("Skipping collection: "+collection+", version: "+version);
+        }
       } else {
-        System.out.println("Skipping collection: "+collection+", version: "+version);
+        if (packageManager.verify(pkg, Collections.singletonList(collection))) {
+          deployed.add(collection);
+        }
       }
     }
     return deployed;


[lucene-solr] 04/05: add a 'refresh' command to force refresh loading of packages

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-13662-updated
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 2eda5490c770ad09e6647a26d10eacd36bee9530
Author: noble <no...@apache.org>
AuthorDate: Mon Oct 21 22:05:33 2019 +1100

    add a 'refresh' command to force refresh loading of packages
---
 .../src/java/org/apache/solr/pkg/PackageAPI.java   | 31 +++++++++++++++++++-
 .../java/org/apache/solr/pkg/PackageLoader.java    | 10 +++++++
 .../src/test/org/apache/solr/pkg/TestPackages.java | 34 ++++++++++++++++++++--
 3 files changed, 72 insertions(+), 3 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java b/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
index fa9f952..f9084b4 100644
--- a/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
@@ -204,6 +204,29 @@ public class PackageAPI {
       permission = PACKAGE_EDIT_PERM)
   public class Edit {
 
+    @Command(name = "refresh")
+    public void refresh(SolrQueryRequest req, SolrQueryResponse rsp, PayloadObj<String> payload) {
+      String p = payload.get();
+      if (p == null) {
+        payload.addError("Package null");
+        return;
+      }
+      PackageLoader.Package pkg = coreContainer.getPackageLoader().getPackage(p);
+      if (pkg == null) {
+        payload.addError("No such package: " + p);
+        return;
+      }
+
+      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&refreshPackage=" + p,
+            Utils.JAVABINCONSUMER);
+      }
+
+
+    }
+
+
     @Command(name = "add")
     public void add(SolrQueryRequest req, SolrQueryResponse rsp, PayloadObj<AddVersion> payload) {
       if (!checkEnabled(payload)) return;
@@ -305,6 +328,12 @@ public class PackageAPI {
   public class Read {
     @Command()
     public void get(SolrQueryRequest req, SolrQueryResponse rsp) {
+      String refresh = req.getParams().get("refreshPackage");
+      if (refresh != null) {
+        packageLoader.notifyListeners(refresh);
+        return;
+      }
+
       int expectedVersion = req.getParams().getInt("expectedVersion", -1);
       if (expectedVersion != -1) {
         syncToVersion(expectedVersion);
@@ -322,7 +351,7 @@ public class PackageAPI {
       for (int i = 0; i < 10; i++) {
         log.debug("my version is {} , and expected version {}", pkgs.znodeVersion, expectedVersion);
         if (pkgs.znodeVersion >= expectedVersion) {
-          if(origVersion < pkgs.znodeVersion){
+          if (origVersion < pkgs.znodeVersion) {
             packageLoader.refreshPackageConf();
           }
           return;
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java b/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
index fb62a5f..c86e433 100644
--- a/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
@@ -128,6 +128,16 @@ public class PackageLoader {
 
   }
 
+  public void notifyListeners(String pkg) {
+    Package p = packageClassLoaders.get(pkg);
+    if(p != null){
+      List<Package> l = Collections.singletonList(p);
+      for (SolrCore core : coreContainer.getCores()) {
+        core.getPackageListeners().packagesUpdated(l);
+      }
+    }
+  }
+
   /**
    * represents a package definition in the packages.json
    */
diff --git a/solr/core/src/test/org/apache/solr/pkg/TestPackages.java b/solr/core/src/test/org/apache/solr/pkg/TestPackages.java
index 92417d4..ddbf39a 100644
--- a/solr/core/src/test/org/apache/solr/pkg/TestPackages.java
+++ b/solr/core/src/test/org/apache/solr/pkg/TestPackages.java
@@ -245,13 +245,15 @@ public class TestPackages extends SolrCloudTestCase {
           return new RequestWriter.StringPayloadContentWriter("{set:{PKG_VERSIONS:{mypkg : '1.1'}}}",
               ClientUtils.TEXT_JSON);
         }
-      }
-          .process(cluster.getSolrClient()) ;
+      }.process(cluster.getSolrClient()) ;
 
       add.version = "2.1";
       add.files = Arrays.asList(new String[]{FILE3});
       req.process(cluster.getSolrClient());
 
+      //the collections mypkg is set to use version 1.1
+      //so no upgrade
+
       verifyCmponent(cluster.getSolrClient(),
           COLLECTION_NAME, "queryResponseWriter", "json1",
           "mypkg", "1.1" );
@@ -264,7 +266,35 @@ public class TestPackages extends SolrCloudTestCase {
           COLLECTION_NAME, "requestHandler", "/runtime",
           "mypkg", "1.1" );
 
+      new GenericSolrRequest(SolrRequest.METHOD.POST, "/config/params", params ){
+        @Override
+        public RequestWriter.ContentWriter getContentWriter(String expectedType) {
+          return new RequestWriter.StringPayloadContentWriter("{set:{PKG_VERSIONS:{mypkg : '2.1'}}}",
+              ClientUtils.TEXT_JSON);
+        }
+      }.process(cluster.getSolrClient()) ;
+
+      //now, let's force every collection using 'mypkg' to refresh
+      //so that it uses version 2.1
+      new V2Request.Builder("/cluster/package")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{refresh : mypkg}")
+          .forceV2(true)
+          .build()
+          .process(cluster.getSolrClient());
+
 
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "queryResponseWriter", "json1",
+          "mypkg", "2.1" );
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "searchComponent", "get",
+          "mypkg", "2.1" );
+
+      verifyCmponent(cluster.getSolrClient(),
+          COLLECTION_NAME, "requestHandler", "/runtime",
+          "mypkg", "2.1" );
 
     } finally {
       cluster.shutdown();