You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2019/08/27 04:13:11 UTC

[lucene-solr] 01/02: SOLR-13710: Persist package jars locally & expose them over http

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

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

commit df32670511fdbdc43fb8d66dd7e2c5ed484e2cda
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Tue Aug 27 14:04:42 2019 +1000

    SOLR-13710: Persist package jars locally & expose them over http
---
 .../java/org/apache/solr/core/BlobRepository.java  | 187 ++++++++++++++++++---
 .../java/org/apache/solr/core/CoreContainer.java   |   3 +-
 .../java/org/apache/solr/core/PackageManager.java  |  10 +-
 .../src/java/org/apache/solr/core/RuntimeLib.java  |  26 +--
 .../org/apache/solr/core/SolrResourceLoader.java   |  27 ++-
 .../solr/handler/admin/CollectionHandlerApi.java   |  15 +-
 .../solr/security/PermissionNameProvider.java      |   2 +
 .../apache/solr/servlet/SolrDispatchFilter.java    |  12 +-
 solr/core/src/test-files/runtimecode/sig.txt       |  24 +++
 .../solr/core/BlobRepositoryMockingTest.java       |  22 +--
 .../apache/solr/core/TestDynamicLoadingUrl.java    |   8 +-
 .../solr/handler/TestContainerReqHandler.java      | 107 ++++++------
 .../adding-custom-plugins-in-solrcloud-mode.adoc   |  10 +-
 .../cluster.Commands.runtimelib.properties.json    |   6 +-
 .../solrj/src/resources/apispec/node.blob.GET.json |  11 ++
 15 files changed, 338 insertions(+), 132 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/core/BlobRepository.java b/solr/core/src/java/org/apache/solr/core/BlobRepository.java
index 02199e2..ea2f6d7 100644
--- a/solr/core/src/java/org/apache/solr/core/BlobRepository.java
+++ b/solr/core/src/java/org/apache/solr/core/BlobRepository.java
@@ -16,17 +16,20 @@
  */
 package org.apache.solr.core;
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
-import java.math.BigInteger;
 import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
+import java.nio.file.Path;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
@@ -34,10 +37,14 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.regex.Pattern;
 
+import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.V2HttpCall;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -45,16 +52,26 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.PermissionNameProvider;
 import org.apache.solr.util.SimplePostTool;
 import org.apache.zookeeper.server.ByteBufferInputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
 import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
 import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
 import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
+import static org.apache.solr.handler.ReplicationHandler.FILE_STREAM;
 
 /**
  * The purpose of this class is to store the Jars loaded in memory and to keep only one copy of the Jar in a single node.
@@ -88,6 +105,14 @@ public class BlobRepository {
     this.coreContainer = coreContainer;
   }
 
+  public Collection<String> getFiles() {
+    return Arrays.asList(getBlobsPath().toFile().list());
+  }
+
+  public Path getBlobsPath() {
+    return SolrResourceLoader.getBlobsDirPath(this.coreContainer.getResourceLoader().getInstancePath());
+  }
+
   // I wanted to {@link SolrCore#loadDecodeAndCacheBlob(String, Decoder)} below but precommit complains
 
   /**
@@ -116,12 +141,12 @@ public class BlobRepository {
     return getBlobIncRef(key.concat(decoder.getName()), () -> addBlob(key, decoder));
   }
 
-  BlobContentRef getBlobIncRef(String key, Decoder decoder, String url, String sha512) {
+  BlobContentRef getBlobIncRef(String key, Decoder decoder, String url, String sha256) {
     StringBuffer keyBuilder = new StringBuffer(key);
     if (decoder != null) keyBuilder.append(decoder.getName());
-    keyBuilder.append("/").append(sha512);
+    keyBuilder.append("/").append(sha256);
 
-    return getBlobIncRef(keyBuilder.toString(), () -> new BlobContent<>(key, fetchBlobAndVerify(key, url, sha512), decoder));
+    return getBlobIncRef(keyBuilder.toString(), () -> new BlobContent<>(key, fetchBlobAndVerify(key, url, sha256), decoder));
   }
 
   // do the actual work returning the appropriate type...
@@ -166,35 +191,79 @@ public class BlobRepository {
     return aBlob;
   }
 
-  static String INVALID_JAR_MSG = "Invalid jar from {0} , expected sha512 hash : {1} , actual : {2}";
+  static String INVALID_JAR_MSG = "Invalid jar from {0} , expected sha256 hash : {1} , actual : {2}";
 
-  private ByteBuffer fetchBlobAndVerify(String key, String url, String sha512) {
-    ByteBuffer byteBuffer = fetchFromUrl(key, url);
-    String computedDigest = sha512Digest(byteBuffer);
-    if (!computedDigest.equals(sha512)) {
-      throw new SolrException(SERVER_ERROR, StrUtils.formatString(INVALID_JAR_MSG, url, sha512, computedDigest));
+  private ByteBuffer fetchBlobAndVerify(String key, String url, String sha256) throws IOException {
+    ByteBuffer byteBuffer = null;
+    if (sha256 != null) {
+      byteBuffer = getFromLocalFs(sha256);
+    }
+    if (byteBuffer == null) byteBuffer = getAndValidate(key, url, sha256);
+    return byteBuffer;
+  }
 
+  private ByteBuffer getAndValidate(String key, String url, String sha256) throws IOException {
+    ByteBuffer byteBuffer = fetchFromUrl(key, url);
+    String computedDigest = sha256Digest(byteBuffer);
+    if (!computedDigest.equals(sha256)) {
+      throw new SolrException(SERVER_ERROR, StrUtils.formatString(INVALID_JAR_MSG, url, sha256, computedDigest));
+    }
+    File file = new File(getBlobsPath().toFile(), sha256);
+    try (FileOutputStream fos = new FileOutputStream(file)) {
+      fos.write(byteBuffer.array(), byteBuffer.arrayOffset(), byteBuffer.limit());
+      IOUtils.fsync(file.toPath(), false);
     }
     return byteBuffer;
   }
 
-  public static String sha512Digest(ByteBuffer byteBuffer) {
-    MessageDigest digest = null;
+  public String putBlob(InputStream is) throws SolrException {
+    byte[] b = new byte[(int) MAX_JAR_SIZE + 1];
+    String sha256 = null;
     try {
-      digest = MessageDigest.getInstance("SHA-512");
-    } catch (NoSuchAlgorithmException e) {
-      //unlikely
-      throw new SolrException(SERVER_ERROR, e);
+      int sz = is.read(b);
+
+      if (sz > MAX_JAR_SIZE)
+        throw new SolrException(BAD_REQUEST, "size is more than permitted , use system property runtime.lib.size to change it");
+      sha256 = sha256Digest(ByteBuffer.wrap(b, 0, sz));
+      File file = new File(getBlobsPath().toFile(), sha256);
+      try (FileOutputStream fos = new FileOutputStream(file)) {
+        fos.write(b, 0, sz);
+      }
+      IOUtils.fsync(file.toPath(), false);
+    } catch (IOException e) {
+      throw new SolrException(BAD_REQUEST, e);
     }
-    byteBuffer = ByteBuffer.wrap(byteBuffer.array(), byteBuffer.arrayOffset(), byteBuffer.limit());
-    digest.update(byteBuffer);
+    return sha256;
 
-    return String.format(
-        Locale.ROOT,
-        "%0128x",
-        new BigInteger(1, digest.digest()));
   }
 
+  private ByteBuffer getFromLocalFs(String sha256) throws IOException {
+    Path p = getBlobsPath();
+    File f = new File(p.toFile(), sha256);
+    if (!f.exists()) return null;
+    byte[] b = new byte[(int) f.length()];
+    try (FileInputStream fis = new FileInputStream(f)) {
+      fis.read(b);
+      ByteBuffer byteBuffer = ByteBuffer.wrap(b);
+      if (sha256.equals(sha256Digest(byteBuffer))) {
+        return byteBuffer;
+      } else {
+        return null;
+
+      }
+    }
+  }
+
+  public static String sha256Digest(ByteBuffer buf) {
+    try {
+      return DigestUtils.sha256Hex(new ByteBufferInputStream(ByteBuffer.wrap( buf.array(), buf.arrayOffset(), buf.limit())));
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to compute sha256", e);
+    }
+  }
+
+
+
 
   /**
    * Package local for unit tests only please do not use elsewhere
@@ -216,14 +285,14 @@ public class BlobRepository {
       entity = response.getEntity();
       int statusCode = response.getStatusLine().getStatusCode();
       if (statusCode != 200) {
-        throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no such resource available: " + key + ", url : "+ url);
+        throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no such resource available: " + key + ", url : " + url);
       }
 
       try (InputStream is = entity.getContent()) {
         b = SimplePostTool.inputStreamToByteArray(is, MAX_JAR_SIZE);
       }
     } catch (Exception e) {
-      log.error("Error loading resource "+ url, e);
+      log.error("Error loading resource " + url, e);
       if (e instanceof SolrException) {
         throw (SolrException) e;
       } else {
@@ -284,6 +353,68 @@ public class BlobRepository {
     }
   }
 
+  BlobRead blobRead = new BlobRead();
+
+
+  class BlobRead extends RequestHandlerBase implements PermissionNameProvider {
+
+
+    @Override
+    public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) {
+
+    }
+
+    @Override
+    public String getDescription() {
+      return "List fetch blobs";
+    }
+
+    @Override
+    public Name getPermissionName(AuthorizationContext request) {
+      return null;
+    }
+
+    @Override
+    public Collection<Api> getApis() {
+      return Collections.singleton(new Api(Utils.getSpec("node.blob.GET")) {
+        @Override
+        public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+          String sha256 = ((V2HttpCall) req.getHttpSolrCall()).getUrlParts().get("sha256");
+          if (sha256 == null) {
+            rsp.add("blob", getFiles());
+          } else {
+            try {
+              ByteBuffer buf = getFromLocalFs(sha256);
+              if(buf == null){
+                throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No such blob");
+              } else {
+                ModifiableSolrParams solrParams = new ModifiableSolrParams();
+                solrParams.add(CommonParams.WT, FILE_STREAM);
+                req.setParams( SolrParams.wrapDefaults(solrParams, req.getParams()));
+                rsp.add(FILE_STREAM, (SolrCore.RawWriter) os -> os.write(buf.array(), buf.arrayOffset(), buf.limit()));
+              }
+
+            } catch (IOException e) {
+              throw new SolrException(SERVER_ERROR,e);
+            }
+          }
+
+        }
+      });
+    }
+
+    @Override
+    public Boolean registerV1() {
+      return Boolean.FALSE;
+    }
+
+    @Override
+    public Boolean registerV2() {
+      return Boolean.TRUE;
+    }
+  }
+
+
   public static class BlobContent<T> {
     public final String key;
     private final T content; // holds byte buffer or cached object, holding both is a waste of memory
@@ -337,7 +468,7 @@ public class BlobRepository {
   public static class BlobContentRef<T> {
     public final BlobContent<T> blob;
 
-    private BlobContentRef(BlobContent<T> blob) {
+    public BlobContentRef(BlobContent<T> blob) {
       this.blob = blob;
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 9a3d5b4..5e67120 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -642,6 +642,7 @@ public class CoreContainer {
     this.backupRepoFactory = new BackupRepositoryFactory(cfg.getBackupRepositoryPlugins());
 
     containerHandlers.put("/ext", clusterPropertiesListener.extHandler);
+    containerHandlers.put("/blob-get", blobRepository.blobRead);
     createHandler(ZK_PATH, ZookeeperInfoHandler.class.getName(), ZookeeperInfoHandler.class);
     createHandler(ZK_STATUS_PATH, ZookeeperStatusHandler.class.getName(), ZookeeperStatusHandler.class);
     collectionsHandler = createHandler(COLLECTIONS_HANDLER_PATH, cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
@@ -1553,7 +1554,7 @@ public class CoreContainer {
       } catch (SolrCoreState.CoreIsClosedException e) {
         throw e;
       } catch (Exception e) {
-        coreInitFailures.put(cd.getName(), new CoreLoadFailure(cd, (Exception) e));
+        coreInitFailures.put(cd.getName(), new CoreLoadFailure(cd, e));
         throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to reload core [" + cd.getName() + "]", e);
       } finally {
         if (!success && newCore != null && newCore.getOpenCount() > 0) {
diff --git a/solr/core/src/java/org/apache/solr/core/PackageManager.java b/solr/core/src/java/org/apache/solr/core/PackageManager.java
index ecc6950..7eb00a5 100644
--- a/solr/core/src/java/org/apache/solr/core/PackageManager.java
+++ b/solr/core/src/java/org/apache/solr/core/PackageManager.java
@@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.params.CommonParams.PACKAGE;
 import static org.apache.solr.common.params.CommonParams.VERSION;
-import static org.apache.solr.core.RuntimeLib.SHA512;
+import static org.apache.solr.core.RuntimeLib.SHA256;
 
 public class PackageManager implements ClusterPropertiesListener {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -94,8 +94,8 @@ public class PackageManager implements ClusterPropertiesListener {
 
 
     public boolean isModified(Map map) {
-      return (!Objects.equals(lib.getSha512(), (map).get(SHA512)) ||
-          !Objects.equals(lib.getSig(), (map).get(SHA512)));
+      return (!Objects.equals(lib.getSha256(), (map).get(SHA256)) ||
+          !Objects.equals(lib.getSig(), (map).get(SHA256)));
     }
   }
 
@@ -167,7 +167,7 @@ public class PackageManager implements ClusterPropertiesListener {
       needsReload[0] = true;
     }
     if (needsReload[0]) {
-      createNewClassLoader(m, ver);
+      createNewClassLoaders(m, ver);
     }
     return needsReload[0];
   }
@@ -177,7 +177,7 @@ public class PackageManager implements ClusterPropertiesListener {
     return p == null ? coreContainer.getResourceLoader() : p.loader;
   }
 
-  void createNewClassLoader(Map m, int ver) {
+  void createNewClassLoaders(Map m, int ver) {
     boolean[] loadedAll = new boolean[1];
     loadedAll[0] = true;
     Map<String, Package> newPkgs = new LinkedHashMap<>();
diff --git a/solr/core/src/java/org/apache/solr/core/RuntimeLib.java b/solr/core/src/java/org/apache/solr/core/RuntimeLib.java
index 507fe02..1e1f5f7 100644
--- a/solr/core/src/java/org/apache/solr/core/RuntimeLib.java
+++ b/solr/core/src/java/org/apache/solr/core/RuntimeLib.java
@@ -44,10 +44,10 @@ import static org.apache.solr.common.params.CommonParams.NAME;
  */
 public class RuntimeLib implements PluginInfoInitialized, AutoCloseable, MapWriter {
   public static final String TYPE = "runtimeLib";
-  public static final String SHA512 = "sha512";
+  public static final String SHA256 = "sha256";
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final CoreContainer coreContainer;
-  private String name, version, sig, sha512, url;
+  private String name, version, sig, sha256, url;
   private BlobRepository.BlobContentRef<ByteBuffer> jarContent;
   private boolean verified = false;
   int znodeVersion = -1;
@@ -57,7 +57,7 @@ public class RuntimeLib implements PluginInfoInitialized, AutoCloseable, MapWrit
     ew.putIfNotNull(NAME, name);
     ew.putIfNotNull("url", url);
     ew.putIfNotNull(version, version);
-    ew.putIfNotNull(sha512, sha512);
+    ew.putIfNotNull("sha256", sha256);
     ew.putIfNotNull("sig", sig);
     if (znodeVersion > -1) {
       ew.put(ConfigOverlay.ZNODEVER, znodeVersion);
@@ -101,19 +101,19 @@ public class RuntimeLib implements PluginInfoInitialized, AutoCloseable, MapWrit
       }
       version = String.valueOf(v);
     } else {
-      sha512 = info.attributes.get(SHA512);
-      if (sha512 == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "runtimeLib with url must have a 'sha512' attribute");
+      sha256 = info.attributes.get(SHA256);
+      if (sha256 == null) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "runtimeLib with url must have a 'sha256' attribute");
       }
       ByteBuffer buf = coreContainer.getBlobRepository().fetchFromUrl(name, url);
 
-      String digest = BlobRepository.sha512Digest(buf);
-      if (!sha512.equals(digest)) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, StrUtils.formatString(BlobRepository.INVALID_JAR_MSG, url, sha512, digest));
+      String digest = BlobRepository.sha256Digest(buf);
+      if (!sha256.equals(digest)) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, StrUtils.formatString(BlobRepository.INVALID_JAR_MSG, url, sha256, digest));
       }
       verifyJarSignature(buf);
 
-      log.debug("dynamic library verified {}, sha512: {}", url, sha512);
+      log.debug("dynamic library verified {}, sha256: {}", url, sha256);
 
     }
 
@@ -130,7 +130,7 @@ public class RuntimeLib implements PluginInfoInitialized, AutoCloseable, MapWrit
 
       jarContent = url == null ?
           coreContainer.getBlobRepository().getBlobIncRef(name + "/" + version) :
-          coreContainer.getBlobRepository().getBlobIncRef(name, null, url, sha512);
+          coreContainer.getBlobRepository().getBlobIncRef(name, null, url, sha256);
 
     }
   }
@@ -148,8 +148,8 @@ public class RuntimeLib implements PluginInfoInitialized, AutoCloseable, MapWrit
 
   }
 
-  public String getSha512() {
-    return sha512;
+  public String getSha256() {
+    return sha256;
   }
 
   public ByteBuffer getFileContent(String entryName) throws IOException {
diff --git a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
index f27edbc..b3dc5e4 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
@@ -576,8 +576,8 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
       }
     }
   }
-  
-  static final String empty[] = new String[0];
+
+  static final String[] empty = new String[0];
   
   @Override
   public <T> T newInstance(String name, Class<T> expectedType) {
@@ -808,6 +808,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
    * manipulated using select Solr features (e.g. streaming expressions).
    */
   public static final String USER_FILES_DIRECTORY = "userfiles";
+  public static final String BLOBS_DIRECTORY = "blobs";
   public static void ensureUserFilesDataDir(Path solrHome) {
     final Path userFilesPath = getUserFilesPath(solrHome);
     final File userFilesDirectory = new File(userFilesPath.toString());
@@ -823,10 +824,28 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
     }
   }
 
+  public static void ensureBlobsDir(Path solrHome) {
+    final Path blobsDir = getBlobsDirPath(solrHome);
+    final File blobsFilesDirectory = new File(blobsDir.toString());
+    if (! blobsFilesDirectory.exists()) {
+      try {
+        final boolean created = blobsFilesDirectory.mkdir();
+        if (! created) {
+          log.warn("Unable to create [{}] directory in SOLR_HOME [{}].  Features requiring this directory may fail.", BLOBS_DIRECTORY, solrHome);
+        }
+      } catch (Exception e) {
+          log.warn("Unable to create [" + BLOBS_DIRECTORY + "] directory in SOLR_HOME [" + solrHome + "].  Features requiring this directory may fail.", e);
+      }
+    }
+  }
+
+  public static Path getBlobsDirPath(Path solrHome) {
+    return Paths.get(solrHome.toAbsolutePath().toString(), BLOBS_DIRECTORY).toAbsolutePath();
+  }
+
   public static Path getUserFilesPath(Path solrHome) {
     return Paths.get(solrHome.toAbsolutePath().toString(), USER_FILES_DIRECTORY).toAbsolutePath();
   }
-
   // Logs a message only once per startup
   private static void logOnceInfo(String key, String msg) {
     if (!loggedOnce.contains(key)) {
@@ -923,7 +942,7 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
         }
       }
-      try (OutputStream out = new FileOutputStream(confFile);) {
+      try (OutputStream out = new FileOutputStream(confFile)) {
         out.write(content);
       }
       log.info("Written confile " + resourceName);
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
index 1a1e2b6..2259a0e 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
@@ -17,7 +17,9 @@
 
 package org.apache.solr.handler.admin;
 
+import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -29,6 +31,7 @@ import java.util.Objects;
 
 import org.apache.solr.api.ApiBag;
 import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionApiMapping;
 import org.apache.solr.client.solrj.request.CollectionApiMapping.CommandMeta;
 import org.apache.solr.client.solrj.request.CollectionApiMapping.Meta;
@@ -60,7 +63,7 @@ import static java.util.Arrays.asList;
 import static java.util.Collections.singletonList;
 import static org.apache.solr.common.util.CommandOperation.captureErrors;
 import static org.apache.solr.common.util.StrUtils.formatString;
-import static org.apache.solr.core.RuntimeLib.SHA512;
+import static org.apache.solr.core.RuntimeLib.SHA256;
 
 public class CollectionHandlerApi extends BaseHandlerApiSupport {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -218,8 +221,8 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
         op.addError(StrUtils.formatString("The jar with a name ''{0}'' does not exist", name));
         return false;
       }
-      if (Objects.equals(existing.get(SHA512), op.getDataMap().get(SHA512))) {
-        op.addError("Trying to update a jar with the same sha512");
+      if (Objects.equals(existing.get(SHA256), op.getDataMap().get(SHA256))) {
+        op.addError("Trying to update a jar with the same sha256");
         return false;
       }
     }
@@ -365,4 +368,10 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
     }
   }
 
+  public static void postBlob(String baseUrl, ByteBuffer buf) throws IOException {
+    try(HttpSolrClient client = new HttpSolrClient.Builder(baseUrl+"/____v2/node/blob" ).build()){
+
+    }
+  }
+
 }
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index f1d216a..16b39a4 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -52,6 +52,8 @@ public interface PermissionNameProvider {
     AUTOSCALING_WRITE_PERM("autoscaling-write", null),
     AUTOSCALING_HISTORY_READ_PERM("autoscaling-history-read", null),
     METRICS_HISTORY_READ_PERM("metrics-history-read", null),
+    BLOB_READ("blob-read", null),
+    BLOB_WRITE("blob-write", null),
     ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
     ;
     final String name;
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 90d6b17..a385479 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -82,9 +82,9 @@ import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
 import org.apache.solr.security.PublicKeyHandler;
 import org.apache.solr.util.SolrFileCleaningTracker;
-import org.apache.solr.util.tracing.GlobalTracer;
 import org.apache.solr.util.StartupLoggingUtils;
 import org.apache.solr.util.configuration.SSLConfigurationsFactory;
+import org.apache.solr.util.tracing.GlobalTracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -180,6 +180,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       final Path solrHomePath = solrHome == null ? SolrResourceLoader.locateSolrHome() : Paths.get(solrHome);
       coresInit = createCoreContainer(solrHomePath, extraProperties);
       SolrResourceLoader.ensureUserFilesDataDir(solrHomePath);
+      SolrResourceLoader.ensureBlobsDir(solrHomePath);
       this.httpClient = coresInit.getUpdateShardHandler().getDefaultHttpClient();
       setupJvmMetrics(coresInit);
       log.debug("user.dir=" + System.getProperty("user.dir"));
@@ -626,8 +627,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
             public void close() {
               // even though we skip closes, we let local tests know not to close so that a full understanding can take
               // place
-              assert Thread.currentThread().getStackTrace()[2].getClassName().matches(
-                  "org\\.apache\\.(?:solr|lucene).*") ? false : true : CLOSE_STREAM_MSG;
+              assert !Thread.currentThread().getStackTrace()[2].getClassName().matches(
+                  "org\\.apache\\.(?:solr|lucene).*") : CLOSE_STREAM_MSG;
               this.stream = ClosedServletInputStream.CLOSED_SERVLET_INPUT_STREAM;
             }
           };
@@ -661,9 +662,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
             public void close() {
               // even though we skip closes, we let local tests know not to close so that a full understanding can take
               // place
-              assert Thread.currentThread().getStackTrace()[2].getClassName().matches(
-                  "org\\.apache\\.(?:solr|lucene).*") ? false
-                      : true : CLOSE_STREAM_MSG;
+              assert !Thread.currentThread().getStackTrace()[2].getClassName().matches(
+                  "org\\.apache\\.(?:solr|lucene).*") : CLOSE_STREAM_MSG;
               stream = ClosedServletOutputStream.CLOSED_SERVLET_OUTPUT_STREAM;
             }
           };
diff --git a/solr/core/src/test-files/runtimecode/sig.txt b/solr/core/src/test-files/runtimecode/sig.txt
index 8e1ba98..44ead80 100644
--- a/solr/core/src/test-files/runtimecode/sig.txt
+++ b/solr/core/src/test-files/runtimecode/sig.txt
@@ -71,3 +71,27 @@ openssl dgst -sha512 cache.jar.bin
 openssl dgst -sha512 cache_v2.jar.bin
 
 aa3f42fb640636dd8126beca36ac389486d0fcb1c3a2e2c387d043d57637535ce8db3b17983853322f78bb8f447ed75fe7b405675debe652ed826ee95e8ce328
+
+=============sha256============================
+
+openssl dgst -sha256 runtimelibs.jar.bin
+
+e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc
+
+openssl dgst -sha512 runtimelibs_v2.jar.bin
+
+79298d7d5c3e60d91154efe7d72f4536eac46698edfa22ab894b85492d562ed4
+
+openssl dgst -sha256 runtimelibs_v3.jar.bin
+
+20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3
+
+openssl dgst -sha256 cache.jar.bin
+
+380c2a61759f01b4d5d2570496c3d2737e3cc6968347faa94d93e906e03e077f
+
+openssl dgst -sha256 cache_v2.jar.bin
+
+22551e42e6fd9646a641ebc1380472ec66fba62f35febad46c8165376b41161d
+
+
diff --git a/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java b/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
index 4a0f1ba..ce1f68e 100644
--- a/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
+++ b/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
@@ -36,7 +36,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.eq;
 import static org.mockito.Mockito.mock;
@@ -63,6 +62,7 @@ public class BlobRepositoryMockingTest {
   boolean blobFetched = false;
   String blobKey = "";
   String url = null;
+  String sha256 = null;
   ByteBuffer filecontent = null;
   
   @BeforeClass
@@ -92,6 +92,14 @@ public class BlobRepositoryMockingTest {
       }
 
       @Override
+      BlobContentRef getBlobIncRef(String key, Decoder decoder, String url, String sha256) {
+        if(!Objects.equals(sha256, BlobRepositoryMockingTest.this.sha256)) return null;
+        blobKey = key;
+        blobFetched = true;
+        return new BlobContentRef(new BlobContent(key, filecontent)) ;
+      }
+
+      @Override
       ConcurrentHashMap<String, BlobContent> createMap() {
         return mapMock;
       }
@@ -130,21 +138,13 @@ public class BlobRepositoryMockingTest {
     when(mockContainer.isZooKeeperAware()).thenReturn(true);
     filecontent = TestDynamicLoading.getFileContent("runtimecode/runtimelibs_v2.jar.bin");
     url = "http://localhost:8080/myjar/location.jar";
+    sha256 = "79298d7d5c3e60d91154efe7d72f4536eac46698edfa22ab894b85492d562ed4";
     BlobRepository.BlobContentRef ref = repository.getBlobIncRef( "filefoo",null,url,
-        "bc5ce45ad281b6a08fb7e529b1eb475040076834816570902acb6ebdd809410e31006efdeaa7f78a6c35574f3504963f5f7e4d92247d0eb4db3fc9abdda5d417");
+        "79298d7d5c3e60d91154efe7d72f4536eac46698edfa22ab894b85492d562ed4");
     assertTrue("filefoo".equals(blobKey));
     assertTrue(blobFetched);
     assertNotNull(ref.blob);
     assertEquals(filecontent, ref.blob.get());
-    verify(mockContainer).isZooKeeperAware();
-    try {
-      repository.getBlobIncRef( "filefoo",null,url,
-          "WRONG-SHA512-KEY");
-      fail("expected exception");
-    } catch (Exception e) {
-      assertTrue(e.getMessage().contains(" expected sha512 hash : WRONG-SHA512-KEY , actual :"));
-    }
-
     url = null;
     filecontent = null;
   }
diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java b/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java
index 575cf9e..8fec3a4 100644
--- a/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java
+++ b/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java
@@ -77,7 +77,7 @@ public class TestDynamicLoadingUrl extends AbstractFullDistribZkTestBase {
     try {
       String payload = "{\n" +
           "'add-runtimelib' : { 'name' : 'urljar', url : 'http://localhost:" + port + "/jar1.jar'" +
-          "  'sha512':'e01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}" +
+          "  'sha256':'e01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}" +
           "}";
       RestTestHarness client = randomRestTestHarness();
       TestSolrConfigHandler.runConfigCommandExpectFailure(client, "/config", payload, "Invalid jar");
@@ -85,7 +85,7 @@ public class TestDynamicLoadingUrl extends AbstractFullDistribZkTestBase {
 
       payload = "{\n" +
           "'add-runtimelib' : { 'name' : 'urljar', url : 'http://localhost:" + port + "/jar1.jar'" +
-          "  'sha512':'d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}" +
+          "  'sha256':'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}" +
           "}";
       client = randomRestTestHarness();
       TestSolrConfigHandler.runConfigCommand(client, "/config", payload);
@@ -93,8 +93,8 @@ public class TestDynamicLoadingUrl extends AbstractFullDistribZkTestBase {
           null,
           "/config/overlay",
           null,
-          Arrays.asList("overlay", "runtimeLib", "urljar", "sha512"),
-          "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420", 120);
+          Arrays.asList("overlay", "runtimeLib", "urljar", "sha256"),
+          "e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc", 120);
 
       payload = "{\n" +
           "'create-requesthandler' : { 'name' : '/runtime', 'class': 'org.apache.solr.core.RuntimeLibReqHandler', 'runtimeLib' : true}" +
diff --git a/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java b/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
index 19a941b..a09f215 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestContainerReqHandler.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
 import java.lang.invoke.MethodHandles;
+import java.net.URL;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -37,12 +38,14 @@ import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.response.SimpleSolrResponse;
 import org.apache.solr.client.solrj.response.V2Response;
 import org.apache.solr.cloud.ConfigRequest;
 import org.apache.solr.cloud.MiniSolrCloudCluster;
@@ -50,6 +53,7 @@ import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Pair;
@@ -150,19 +154,19 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       String payload = null;
       try {
         payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
-            "sha512 : 'wrong-sha512'}}";
+            "sha256 : 'wrong-sha256'}}";
         new V2Request.Builder("/cluster")
             .withPayload(payload)
             .withMethod(SolrRequest.METHOD.POST)
             .build().process(cluster.getSolrClient());
         fail("Expected error");
       } catch (BaseHttpSolrClient.RemoteExecutionException e) {
-        assertTrue("actual output : " + Utils.toJSONString(e.getMetaData()), e.getMetaData()._getStr("error/details[0]/errorMessages[0]", "").contains("expected sha512 hash :"));
+        assertTrue("actual output : " + Utils.toJSONString(e.getMetaData()), e.getMetaData()._getStr("error/details[0]/errorMessages[0]", "").contains("expected sha256 hash :"));
       }
 
       try {
         payload = "{add-package:{name : 'foo', url: 'http://localhost:" + port + "/jar0.jar', " +
-            "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
+            "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
         new V2Request.Builder("/cluster")
             .withPayload(payload)
             .withMethod(SolrRequest.METHOD.POST)
@@ -173,13 +177,13 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       }
 
       payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
-          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
+          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
 
 
       new V2Request.Builder("/cluster")
@@ -203,13 +207,13 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
 
       payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
-          "sha512 : '60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922'}}";
+          "sha256 : '20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -239,6 +243,13 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           (Predicate<Object>) o -> o instanceof List && ((List) o).isEmpty()));
 
 
+      URL baseUrl = cluster.getRandomJetty(random()).getBaseUrl();
+      try(HttpSolrClient client = new HttpSolrClient.Builder(baseUrl.toString()).build()){
+        SimpleSolrResponse rsp = new GenericSolrRequest(SolrRequest.METHOD.GET, "/____v2/node/blob", new ModifiableSolrParams()).process(client);
+        List l = (List) rsp.nl.get("blob");
+        assertTrue(l.contains("e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc"));
+        assertTrue(l.contains("20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3"));
+      }
     } finally {
       cluster.shutdown();
       server.first().stop();
@@ -266,7 +277,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       String payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : 'EdYkvRpMZbvElN93/xUmyKXcj6xHP16AVk71TlTascEwCb5cFQ2AeKhPIlwYpkLWXEOcLZKfeXoWwOLaV5ZNhg==' ," +
-          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
+          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
       try {
         new V2Request.Builder("/cluster")
             .withPayload(payload)
@@ -280,14 +291,14 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : '" + signature + "'," +
-          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
+          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
 
       new V2Request.Builder("/cluster")
           .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler' package : global}}")
@@ -310,14 +321,14 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
           "sig : 'YxFr6SpYrDwG85miDfRWHTjU9UltjtIWQZEhcV55C2rczRUVowCYBxmsDv5mAM8j0CTv854xpI1DtBT86wpoTdbF95LQuP9FJId4TS1j8bZ9cxHP5Cqyz1uBHFfUUNUrnpzTHQkVTp02O9NAjh3c2W41bL4U7j6jQ32+4CW2M+x00TDG0y0H75rQDR8zbLt31oWCz+sBOdZ3rGKJgAvdoGm/wVCTmsabZN+xoz4JaDeBXF16O9Uk9SSq4G0dz5YXFuLxHK7ciB5t0+q6pXlF/tdlDqF76Abze0R3d2/0MhXBzyNp3UxJmj6DiprgysfB0TbQtJG0XGfdSmx0VChvcA==' ," +
-          "sha512 : '60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922'}}";
+          "sha256 : '20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -357,14 +368,14 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       String payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
           "sig : '" + signature + "'," +
-          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
+          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
 
       new V2Request.Builder("/cluster")
           .withPayload("{add-requesthandler:{name : 'bar', class : 'org.apache.solr.core.RuntimeLibReqHandler' package : global }}")
@@ -387,14 +398,14 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar3.jar', " +
           "sig : 'a400n4T7FT+2gM0SC6+MfSOExjud8MkhTSFylhvwNjtWwUgKdPFn434Wv7Qc4QEqDVLhQoL3WqYtQmLPti0G4Q==' ," +
-          "sha512 : '60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922'}}";
+          "sha256 : '20e0bfaec71b2e93c4da9f2ed3745dda04dc3fc915b66cc0275863982e73b2a3'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
 
 
       request = new V2Request.Builder("/node/ext/bar")
@@ -468,16 +479,16 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
       cluster.waitForActiveCollection(COLLECTION_NAME, 2, 2);
       String payload = "{add-package:{name : 'global', url: 'http://localhost:" + port + "/jar1.jar', " +
-          "sha512 : 'd01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}}";
+          "sha256 : 'e1f9e23988c19619402f1040c9251556dcd6e02b9d3e3b966a129ea1be5c70fc'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      String sha512 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512");
+      String sha256 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256");
       String url = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/url");
 
-      assertEquals(sha512,
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
+      assertEquals(sha256,
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
 
 
       payload = "{\n" +
@@ -501,7 +512,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/queryResponseWriter/json1", params),
           Utils.makeMap(
               "/config/queryResponseWriter/json1/_packageinfo_/url", url,
-              "/config/queryResponseWriter/json1/_meta_/sha512", sha512
+              "/config/queryResponseWriter/json1/_meta_/sha256", sha256
           ));
 
       params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
@@ -513,7 +524,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/searchComponent/get", params),
           Utils.makeMap(
               "config/searchComponent/get/_packageinfo_/url", url,
-              "config/searchComponent/get/_packageinfo_/sha512", sha512
+              "config/searchComponent/get/_packageinfo_/sha256", sha256
           ));
 
       params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
@@ -525,7 +536,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/requestHandler/runtime", params),
           Utils.makeMap(
               ":config:requestHandler:/runtime:_packageinfo_:url", url,
-              ":config:requestHandler:/runtime:_packageinfo_:sha512", sha512
+              ":config:requestHandler:/runtime:_packageinfo_:sha256", sha256
           ));
 
 
@@ -578,16 +589,16 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
 
       payload = "{update-package:{name : 'global', url: 'http://localhost:" + port + "/jar2.jar', " +
-          "sha512 : 'bc5ce45ad281b6a08fb7e529b1eb475040076834816570902acb6ebdd809410e31006efdeaa7f78a6c35574f3504963f5f7e4d92247d0eb4db3fc9abdda5d417'}}";
+          "sha256 : '79298d7d5c3e60d91154efe7d72f4536eac46698edfa22ab894b85492d562ed4'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      sha512 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512");
+      sha256 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256");
       url = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/url");
 
-      assertEquals(sha512,
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha512"));
+      assertEquals(sha256,
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/global/sha256"));
 
       params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
           WT, JAVABIN,
@@ -598,7 +609,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/queryResponseWriter/json1", params),
           Utils.makeMap(
               "/config/queryResponseWriter/json1/_packageinfo_/url", url,
-              "/config/queryResponseWriter/json1/_packageinfo_/sha512", sha512
+              "/config/queryResponseWriter/json1/_packageinfo_/sha256", sha256
           ));
 
       params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
@@ -610,7 +621,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/searchComponent/get", params),
           Utils.makeMap(
               "/config/searchComponent/get/_packageinfo_/url", url,
-              "/config/searchComponent/get/_packageinfo_/sha512", sha512
+              "/config/searchComponent/get/_packageinfo_/sha256", sha256
           ));
 
       params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
@@ -622,7 +633,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/requestHandler/runtime", params),
           Utils.makeMap(
               ":config:requestHandler:/runtime:_packageinfo_:url", url,
-              ":config:requestHandler:/runtime:_packageinfo_:sha512", sha512
+              ":config:requestHandler:/runtime:_packageinfo_:sha256", sha256
           ));
 
 
@@ -634,7 +645,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
             .build().process(cluster.getSolrClient());
         fail("should have failed");
       } catch (BaseHttpSolrClient.RemoteExecutionException e) {
-        assertTrue("actual output : " + Utils.toJSONString(e.getMetaData()), e.getMetaData()._getStr("error/details[0]/errorMessages[0]", "").contains("Trying to update a jar with the same sha512"));
+        assertTrue("actual output : " + Utils.toJSONString(e.getMetaData()), e.getMetaData()._getStr("error/details[0]/errorMessages[0]", "").contains("Trying to update a jar with the same sha256"));
       }
 
 
@@ -673,14 +684,14 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
         .configure();
     try {
       String payload = "{add-package:{name : 'cache_pkg', url: 'http://localhost:" + port + "/jar1.jar', " +
-          "sha512 : '1a3739b629ce85895c9b2a8c12dd7d98161ff47634b0693f1e1c5b444fb38343f95c6ee955cd99103bd24cfde6c205234b63823818660ac08392cdc626caf585'}}";
+          "sha256 : '380c2a61759f01b4d5d2570496c3d2737e3cc6968347faa94d93e906e03e077f'}}";
 
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/cache_pkg/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/cache_pkg/sha256"));
 
       CollectionAdminRequest
           .createCollection(COLLECTION_NAME, "conf", 2, 1)
@@ -694,7 +705,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       NamedList<Object> rsp = cluster.getSolrClient().request(new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/overlay", params));
       assertEquals("org.apache.solr.core.MyDocCache", rsp._getStr("overlay/props/query/documentCache/class", null));
 
-      String sha512 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha512");
+      String sha256 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/sha256");
       String url = (String) getObjectByPath(Utils.fromJSONString(payload), true, "add-package/url");
 
 
@@ -707,7 +718,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/query/documentCache", params),
           Utils.makeMap(
               "/config/query/documentCache/_packageinfo_/url", url,
-              "/config/query/documentCache/_packageinfo_/sha512", sha512
+              "/config/query/documentCache/_packageinfo_/sha256", sha256
           ));
 
 
@@ -726,15 +737,15 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
 
 
       payload = "{update-package:{name : 'cache_pkg', url: 'http://localhost:" + port + "/jar2.jar', " +
-          "sha512 : 'aa3f42fb640636dd8126beca36ac389486d0fcb1c3a2e2c387d043d57637535ce8db3b17983853322f78bb8f447ed75fe7b405675debe652ed826ee95e8ce328'}}";
+          "sha256 : '22551e42e6fd9646a641ebc1380472ec66fba62f35febad46c8165376b41161d'}}";
       new V2Request.Builder("/cluster")
           .withPayload(payload)
           .withMethod(SolrRequest.METHOD.POST)
           .build().process(cluster.getSolrClient());
-      sha512 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512");
+      sha256 = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256");
       url = (String) getObjectByPath(Utils.fromJSONString(payload), true, "update-package/url");
-      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha512"),
-          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/cache_pkg/sha512"));
+      assertEquals(getObjectByPath(Utils.fromJSONString(payload), true, "update-package/sha256"),
+          getObjectByPath(new ClusterProperties(cluster.getZkClient()).getClusterProperties(), true, "package/cache_pkg/sha256"));
 
       params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
           WT, JAVABIN,
@@ -745,7 +756,7 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/config/query/documentCache", params),
           Utils.makeMap(
               "/config/query/documentCache/_packageinfo_/url", url,
-              "/config/query/documentCache/_packageinfo_/sha512", sha512
+              "/config/query/documentCache/_packageinfo_/sha256", sha256
           ));
       req = new UpdateRequest();
       req.add("id", "2", "desc_s", "document 1")
@@ -765,8 +776,6 @@ public class TestContainerReqHandler extends SolrCloudTestCase {
       cluster.shutdown();
       server.first().stop();
     }
-
-
   }
 
 }
diff --git a/solr/solr-ref-guide/src/adding-custom-plugins-in-solrcloud-mode.adoc b/solr/solr-ref-guide/src/adding-custom-plugins-in-solrcloud-mode.adoc
index 88ec077..5738c6f 100644
--- a/solr/solr-ref-guide/src/adding-custom-plugins-in-solrcloud-mode.adoc
+++ b/solr/solr-ref-guide/src/adding-custom-plugins-in-solrcloud-mode.adoc
@@ -128,11 +128,11 @@ Step 1: Download a jar from github to the current directory
 ----
  curl -o runtimelibs.jar   -LO https://github.com/apache/lucene-solr/blob/master/solr/core/src/test-files/runtimecode/runtimelibs.jar.bin?raw=true
 ----
-Step 2: Get the `sha512` hash of the jar
+Step 2: Get the `sha256` hash of the jar
 
 [source,bash]
 ----
- openssl dgst -sha512 runtimelibs.jar
+ openssl dgst -sha256 runtimelibs.jar
 ----
 
 Step 3 :  Start solr with runtime lib enabled
@@ -156,7 +156,7 @@ Step 5: Add the jar to your collection `gettingstarted`
  curl http://localhost:8983/solr/gettingstarted/config -H 'Content-type:application/json' -d '{
     "add-package": { "name" : "my-pkg",
     "url":"http://localhost:8000/runtimelibs.jar" ,
-    "sha512" : "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420"}
+    "sha256" : "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420"}
     }'
 ----
 
@@ -198,7 +198,7 @@ output:
 Example:
 
 * Host the new jar to a new url. eg:  http://localhost:8000/runtimelibs_v2.jar
-* get the `sha512` hash of the new jar
+* get the `sha256` hash of the new jar
 * run the update-runtime lib command
 
 [source,bash]
@@ -206,7 +206,7 @@ Example:
  curl http://localhost:8983/solr/gettingstarted/config -H 'Content-type:application/json' -d '{
     "update-package": { "name" : "my-pkg",
     "url":"http://localhost:8000/runtimelibs_v2.jar" ,
-    "sha512" : "<replace-the-new-sha512-digest-here>"}
+    "sha256" : "<replace-the-new-sha256-digest-here>"}
     }'
 ----
 NOTE: Always upload your jar to a new url as the Solr cluster is still referring to the old jar. If the existing jar is modified it can cause errors as the hash may not match
diff --git a/solr/solrj/src/resources/apispec/cluster.Commands.runtimelib.properties.json b/solr/solrj/src/resources/apispec/cluster.Commands.runtimelib.properties.json
index 5a8a66f..ab334b5 100644
--- a/solr/solrj/src/resources/apispec/cluster.Commands.runtimelib.properties.json
+++ b/solr/solrj/src/resources/apispec/cluster.Commands.runtimelib.properties.json
@@ -9,15 +9,15 @@
       "type": "string",
       "description": "The remote url"
     },
-    "sha512": {
+    "sha256": {
       "type": "string",
-      "description": "The sha512 hash of the jar"
+      "description": "The sha256 hash of the jar"
     },
     "sig": {
       "type": "string",
       "description": "the signature of the jar"
     }
   },
-  "required" : ["name","url","sha512"]
+  "required" : ["name","url","sha256"]
 
 }
\ No newline at end of file
diff --git a/solr/solrj/src/resources/apispec/node.blob.GET.json b/solr/solrj/src/resources/apispec/node.blob.GET.json
new file mode 100644
index 0000000..273333e
--- /dev/null
+++ b/solr/solrj/src/resources/apispec/node.blob.GET.json
@@ -0,0 +1,11 @@
+{
+  "methods": [
+    "GET"
+  ],
+  "url": {
+    "paths": [
+      "/node/blob",
+      "/node/blob/{sha256}"
+    ]
+  }
+}