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/10/08 11:29:56 UTC

[lucene-solr] branch branch_8x updated: SOLR-13821: Package Store for storing package artifacts (#929)

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new f19aaa8  SOLR-13821: Package Store for storing package artifacts (#929)
f19aaa8 is described below

commit f19aaa8097e756991afbf4e5dbdea773e87a872d
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Tue Oct 8 14:24:33 2019 +1100

    SOLR-13821: Package Store for storing package artifacts (#929)
    
    SOLR-13821: Package Store for storing package artifacts
---
 solr/CHANGES.txt                                   |   2 +
 .../java/org/apache/solr/core/BlobRepository.java  |   2 +-
 .../java/org/apache/solr/core/CoreContainer.java   |   8 +
 .../apache/solr/filestore/DistribPackageStore.java | 495 +++++++++++++++++++++
 .../org/apache/solr/filestore/PackageStore.java    | 122 +++++
 .../org/apache/solr/filestore/PackageStoreAPI.java | 273 ++++++++++++
 .../solr/security/PermissionNameProvider.java      |   2 +
 .../apache/solr/servlet/SolrRequestParsers.java    |   4 +-
 .../src/java/org/apache/solr/util/CryptoKeys.java  |  47 +-
 .../core/src/test-files/cryptokeys/priv_key512.pem |   9 +
 solr/core/src/test-files/cryptokeys/pub_key512.der | Bin 0 -> 94 bytes
 .../test-files/runtimecode/runtimelibs_v3.jar.bin  | Bin 0 -> 7337 bytes
 solr/core/src/test-files/runtimecode/sig.txt       | 105 +++++
 .../solr/filestore/TestDistribPackageStore.java    | 250 +++++++++++
 .../org/apache/solr/client/solrj/SolrResponse.java |   9 +-
 .../solr/client/solrj/request/V2Request.java       |  39 +-
 .../apache/solr/common/params/CommonParams.java    |   3 +
 .../java/org/apache/solr/common/util/StrUtils.java | 207 +++++----
 .../java/org/apache/solr/common/util/Utils.java    | 144 ++++--
 .../org/apache/solr/cloud/SolrCloudTestCase.java   |   4 +-
 20 files changed, 1590 insertions(+), 135 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5a01693..0790216 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -79,6 +79,8 @@ New Features
 
 * SOLR-8241: Add CaffeineCache, an efficient implementation of SolrCache.(Ben Manes, Shawn Heisey, David Smiley, Andrzej Bialecki)
 
+* SOLR-13821: A Package store to store and load package artefacts (noble, Ishan Chattopadhyaya)
+
 Improvements
 ----------------------
 
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 24bb88e..59bd795 100644
--- a/solr/core/src/java/org/apache/solr/core/BlobRepository.java
+++ b/solr/core/src/java/org/apache/solr/core/BlobRepository.java
@@ -62,7 +62,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
 public class BlobRepository {
   private static final long MAX_JAR_SIZE = Long.parseLong(System.getProperty("runtme.lib.size", String.valueOf(5 * 1024 * 1024)));
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  static final Random RANDOM;
+  public static final Random RANDOM;
   static final Pattern BLOB_KEY_PATTERN_CHECKER = Pattern.compile(".*/\\d+");
 
   static {
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 7040610..69aa976 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -47,6 +47,7 @@ import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.store.Directory;
+import org.apache.solr.api.AnnotatedApi;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
@@ -76,6 +77,7 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
+import org.apache.solr.filestore.PackageStoreAPI;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.SnapShooter;
 import org.apache.solr.handler.admin.AutoscalingHistoryHandler;
@@ -218,6 +220,8 @@ public class CoreContainer {
 
   protected volatile AutoscalingHistoryHandler autoscalingHistoryHandler;
 
+  private PackageStoreAPI packageStoreAPI;
+
 
   // Bits for the state variable.
   public final static long LOAD_COMPLETE = 0x1L;
@@ -600,6 +604,10 @@ public class CoreContainer {
       }
     }
 
+    packageStoreAPI = new PackageStoreAPI(this);
+    containerHandlers.getApiBag().register(new AnnotatedApi(packageStoreAPI.readAPI), Collections.EMPTY_MAP);
+    containerHandlers.getApiBag().register(new AnnotatedApi(packageStoreAPI.writeAPI), Collections.EMPTY_MAP);
+
     metricManager = new SolrMetricManager(loader, cfg.getMetricsConfig());
 
     coreContainerWorkExecutor = MetricUtils.instrumentedExecutorService(
diff --git a/solr/core/src/java/org/apache/solr/filestore/DistribPackageStore.java b/solr/core/src/java/org/apache/solr/filestore/DistribPackageStore.java
new file mode 100644
index 0000000..910f29b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/filestore/DistribPackageStore.java
@@ -0,0 +1,495 @@
+/*
+ * 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.filestore;
+
+import java.io.ByteArrayOutputStream;
+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.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.BlobRepository;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.filestore.PackageStoreAPI.MetaData;
+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;
+
+
+public class DistribPackageStore implements PackageStore {
+  static final long MAX_PKG_SIZE = Long.parseLong(System.getProperty("max.file.store.size", String.valueOf(100 * 1024 * 1024)));
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private final CoreContainer coreContainer;
+  private Map<String, FileInfo> tmpFiles = new ConcurrentHashMap<>();
+  public DistribPackageStore(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+    ensurePackageStoreDir(coreContainer.getResourceLoader().getInstancePath());
+
+  }
+
+  private String myNode() {
+    return coreContainer.getZkController().getNodeName();
+  }
+
+
+  /**
+   * get a list of nodes randomly shuffled
+   * * @lucene.internal
+   */
+  public ArrayList<String> shuffledNodes() {
+    Set<String> liveNodes = coreContainer.getZkController().getZkStateReader().getClusterState().getLiveNodes();
+    ArrayList<String> l = new ArrayList(liveNodes);
+    l.remove(myNode());
+    Collections.shuffle(l, BlobRepository.RANDOM);
+    return l;
+  }
+
+
+  @Override
+  public Path getRealpath(String path) {
+    if (File.separatorChar == '\\') {
+      path = path.replaceAll("/", File.separator);
+    }
+    if (path.charAt(0) != File.separatorChar) {
+      path = File.separator + path;
+    }
+    return new File(this.coreContainer.getResourceLoader().getInstancePath() +
+        "/" + PackageStoreAPI.PACKAGESTORE_DIRECTORY + path).toPath();
+  }
+
+  class FileInfo {
+    final String path;
+    String metaPath;
+    ByteBuffer fileData, metaData;
+
+
+    FileInfo(String path) {
+      this.path = path;
+    }
+
+    public String getMetaPath() {
+      if (metaPath == null) {
+        int idx = path.lastIndexOf('/');
+        metaPath = path.substring(0, idx + 1) + "." + path.substring(idx + 1) + ".json";
+      }
+      return metaPath;
+    }
+
+
+    private void persistToFile(ByteBuffer data, ByteBuffer meta) throws IOException {
+      synchronized (DistribPackageStore.this) {
+        this.metaData = meta;
+        this.fileData = data;
+        Path realpath = getRealpath(path);
+        File file = realpath.toFile();
+        File parent = file.getParentFile();
+        if (!parent.exists()) {
+          parent.mkdirs();
+        }
+        Map m = (Map) Utils.fromJSON(meta.array());
+        if (m == null || m.isEmpty()) {
+          throw new SolrException(SERVER_ERROR, "invalid metadata , discarding : " + path);
+        }
+
+
+        File metdataFile = getRealpath(getMetaPath()).toFile();
+
+        try (FileOutputStream fos = new FileOutputStream(metdataFile)) {
+          fos.write(meta.array(), 0, meta.limit());
+        }
+        IOUtils.fsync(metdataFile.toPath(), false);
+
+        try (FileOutputStream fos = new FileOutputStream(file)) {
+          fos.write(data.array(), 0, data.limit());
+        }
+        log.info("persisted a file {} and metadata. sizes {} {}", path, data.limit(), meta.limit());
+        IOUtils.fsync(file.toPath(), false);
+      }
+    }
+
+
+    public boolean exists(boolean validateContent, boolean fetchMissing) throws IOException {
+      File file = getRealpath(path).toFile();
+      if (!file.exists()) {
+        if (fetchMissing) {
+          return fetchFromAnyNode();
+        } else {
+          return false;
+        }
+      }
+
+      if (validateContent) {
+        MetaData metaData = readMetaData();
+        if (metaData == null) return false;
+        try (InputStream is = new FileInputStream(getRealpath(path).toFile())) {
+          if (!Objects.equals(DigestUtils.sha512Hex(is), metaData.sha512)) {
+            deleteFile();
+          } else {
+            return true;
+          }
+        } catch (Exception e) {
+          throw new SolrException(SERVER_ERROR, "unable to parse metadata json file");
+        }
+      } else {
+        return true;
+      }
+
+      return false;
+    }
+
+    private void deleteFile() {
+      try {
+        IOUtils.deleteFilesIfExist(getRealpath(path), getRealpath(getMetaPath()));
+      } catch (IOException e) {
+        log.error("Unable to delete files: "+path);
+      }
+
+    }
+
+    private boolean fetchFileFromNodeAndPersist(String fromNode) {
+      log.info("fetching a file {} from {} ", path, fromNode);
+      String url = coreContainer.getZkController().getZkStateReader().getBaseUrlForNodeName(fromNode);
+      if (url == null) throw new SolrException(BAD_REQUEST, "No such node");
+      String baseUrl = url.replace("/solr", "/api");
+
+      ByteBuffer metadata = null;
+      Map m = null;
+      try {
+        metadata = Utils.executeGET(coreContainer.getUpdateShardHandler().getDefaultHttpClient(),
+            baseUrl + "/node/files" + getMetaPath(),
+            Utils.newBytesConsumer((int) MAX_PKG_SIZE));
+        m = (Map) Utils.fromJSON(metadata.array());
+      } catch (SolrException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error fetching metadata", e);
+      }
+
+      try {
+        ByteBuffer filedata = Utils.executeGET(coreContainer.getUpdateShardHandler().getDefaultHttpClient(),
+            baseUrl + "/node/files" + path,
+            Utils.newBytesConsumer((int) MAX_PKG_SIZE));
+        String sha512 = DigestUtils.sha512Hex(new ByteBufferInputStream(filedata));
+        String expected = (String) m.get("sha512");
+        if (!sha512.equals(expected)) {
+          throw new SolrException(SERVER_ERROR, "sha512 mismatch downloading : " + path + " from node : " + fromNode);
+        }
+        persistToFile(filedata, metadata);
+        return true;
+      } catch (SolrException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error fetching data", e);
+      } catch (IOException ioe) {
+        throw new SolrException(SERVER_ERROR, "Error persisting file", ioe);
+      }
+
+    }
+
+    boolean fetchFromAnyNode() {
+
+      ArrayList<String> l = shuffledNodes();
+      ZkStateReader stateReader = coreContainer.getZkController().getZkStateReader();
+      for (String liveNode : l) {
+        try {
+          String baseurl = stateReader.getBaseUrlForNodeName(liveNode);
+          String url = baseurl.replace("/solr", "/api");
+          String reqUrl = url + "/node/files" + path +
+              "?meta=true&wt=javabin&omitHeader=true";
+          boolean nodeHasBlob = false;
+          Object nl = Utils.executeGET(coreContainer.getUpdateShardHandler().getDefaultHttpClient(), reqUrl, Utils.JAVABINCONSUMER);
+          if (Utils.getObjectByPath(nl, false, Arrays.asList("files", getMetaPath())) != null) {
+            nodeHasBlob = true;
+          }
+
+          if (nodeHasBlob) {
+            boolean success = fetchFileFromNodeAndPersist(liveNode);
+            if (success) return true;
+          }
+        } catch (Exception e) {
+          //it's OK for some nodes to fail
+        }
+      }
+
+      return false;
+    }
+
+    String getSimpleName() {
+      int idx = path.lastIndexOf("/");
+      if (idx == -1) return path;
+      return path.substring(idx + 1);
+    }
+
+    public Path realPath() {
+      return getRealpath(path);
+    }
+
+    MetaData readMetaData() throws IOException {
+      File file = getRealpath(getMetaPath()).toFile();
+      if (file.exists()) {
+        try (InputStream fis = new FileInputStream(file)) {
+          return new MetaData((Map) Utils.fromJSON(fis));
+        }
+      }
+      return null;
+
+    }
+
+
+
+
+    public FileDetails getDetails() {
+      FileType type = getType(path);
+
+      return new FileDetails() {
+        @Override
+        public MetaData getMetaData() {
+          try {
+            return readMetaData();
+          } catch (Exception e){
+            throw new RuntimeException(e);
+          }
+        }
+
+        @Override
+        public Date getTimeStamp() {
+          return new Date(realPath().toFile().lastModified());
+        }
+
+        @Override
+        public boolean isDir() {
+          return type == FileType.DIRECTORY;
+        }
+
+        @Override
+        public void writeMap(EntryWriter ew) throws IOException {
+          MetaData metaData = readMetaData();
+          ew.put(CommonParams.NAME, getSimpleName());
+          if (type == FileType.DIRECTORY) {
+            ew.put("dir", true);
+            return;
+          }
+          ew.put("timestamp", getTimeStamp());
+          metaData.writeMap(ew);
+
+        }
+      };
+
+
+    }
+
+    public void readData(Consumer<FileEntry> consumer) throws IOException {
+      MetaData meta = readMetaData();
+      try (InputStream is = new FileInputStream(realPath().toFile())) {
+        consumer.accept(new FileEntry(null, meta,path ){
+          @Override
+          public InputStream getInputStream() {
+            return is;
+          }
+        });
+      }
+    }
+  }
+
+
+  @Override
+  public void put(FileEntry entry) throws IOException {
+    FileInfo info = new FileInfo(entry.path);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    Utils.writeJson(entry.getMetaData(), baos, true);
+    byte[] bytes = baos.toByteArray();
+    info.persistToFile(entry.buf, ByteBuffer.wrap(bytes, 0, bytes.length));
+    tmpFiles.put(entry.getPath(), info);
+    List<String> nodes = shuffledNodes();
+    int i = 0;
+    int FETCHFROM_SRC = 50;
+    String myNodeName = myNode();
+    try {
+      for (String node : nodes) {
+        String baseUrl = coreContainer.getZkController().getZkStateReader().getBaseUrlForNodeName(node);
+        String url = baseUrl.replace("/solr", "/api") + "/node/files" + entry.getPath() + "?getFrom=";
+        if (i < FETCHFROM_SRC) {
+          // this is to protect very large clusters from overwhelming a single node
+          // the first FETCHFROM_SRC nodes will be asked to fetch from this node.
+          // it's there in  the memory now. So , it must be served fast
+          url += myNodeName;
+        } else {
+          if (i == FETCHFROM_SRC) {
+            // This is just an optimization
+            // at this point a bunch of nodes are already downloading from me
+            // I'll wait for them to finish before asking other nodes to download from each other
+            try {
+              Thread.sleep(2 * 1000);
+            } catch (Exception e) {
+            }
+          }
+          // trying to avoid the thundering herd problem when there are a very large no:of nodes
+          // others should try to fetch it from any node where it is available. By now,
+          // almost FETCHFROM_SRC other nodes may have it
+          url += "*";
+        }
+        try {
+          //fire and forget
+          Utils.executeGET(coreContainer.getUpdateShardHandler().getDefaultHttpClient(), url, null);
+        } catch (Exception e) {
+          log.info("Node: " + node +
+              " failed to respond for blob notification", e);
+          //ignore the exception
+          // some nodes may be down or not responding
+        }
+        i++;
+      }
+    } finally {
+      new Thread(() -> {
+        try {
+          // keep the jar in memory for 10 secs , so that
+          //every node can download it from memory without the file system
+          Thread.sleep(10 * 1000);
+        } catch (Exception e) {
+          //don't care
+        } finally {
+          tmpFiles.remove(entry.getPath());
+        }
+      }).start();
+
+
+    }
+
+  }
+
+  @Override
+  public synchronized boolean fetch(String path, String from) {
+    if (path == null || path.isEmpty()) return false;
+    FileInfo f = new FileInfo(path);
+    try {
+      if(f.exists(true, false)){
+        return true;
+      }
+    } catch (IOException e) {
+      log.error("Error fetching file ", e);
+      return false;
+
+    }
+
+    if (from == null || "*".equals(from)) {
+      f.fetchFromAnyNode();
+
+    } else {
+      f.fetchFileFromNodeAndPersist(from);
+    }
+
+    return false;
+  }
+
+  @Override
+  public synchronized void get(String path, Consumer<FileEntry> consumer) throws IOException {
+    File file = getRealpath(path).toFile();
+    String simpleName = file.getName();
+    if (isMetaDataFile(simpleName)) {
+      try (InputStream is = new FileInputStream(file)) {
+        consumer.accept(new FileEntry(null, null, path) {
+          //no metadata for metadata file
+          @Override
+          public InputStream getInputStream() {
+            return is;
+          }
+        });
+      }
+      return;
+    }
+
+    new FileInfo(path).readData(consumer);
+  }
+
+
+  @Override
+  public synchronized List list(String path, Predicate<String> predicate) {
+    File file = getRealpath(path).toFile();
+    List<FileDetails> fileDetails = new ArrayList<>();
+    FileType type = getType(path);
+    if (type == FileType.DIRECTORY) {
+      file.list((dir, name) -> {
+        if (predicate == null || predicate.test(name)) {
+          if (!isMetaDataFile(name)) {
+            fileDetails.add(new FileInfo(path + "/" + name).getDetails());
+          }
+        }
+        return false;
+      });
+
+    } else if (type == FileType.FILE) {
+
+      fileDetails.add(new FileInfo(path).getDetails());
+    }
+
+    return fileDetails;
+  }
+
+
+  @Override
+  public synchronized FileType getType(String path) {
+    File file = getRealpath(path).toFile();
+    if (!file.exists()) return FileType.NOFILE;
+    if (file.isDirectory()) return FileType.DIRECTORY;
+    return isMetaDataFile(file.getName()) ? FileType.METADATA : FileType.FILE;
+  }
+
+  private boolean isMetaDataFile(String file) {
+    return file.charAt(0) == '.' && file.endsWith(".json");
+  }
+
+  private void ensurePackageStoreDir(Path solrHome) {
+    final File packageStoreDir = getPackageStoreDirPath(solrHome).toFile();
+    if (!packageStoreDir.exists()) {
+      try {
+        final boolean created = packageStoreDir.mkdirs();
+        if (!created) {
+          log.warn("Unable to create [{}] directory in SOLR_HOME [{}].  Features requiring this directory may fail.", packageStoreDir, solrHome);
+        }
+      } catch (Exception e) {
+        log.warn("Unable to create [" + packageStoreDir + "] directory in SOLR_HOME [" + solrHome + "].  Features requiring this directory may fail.", e);
+      }
+    }
+  }
+
+  public static Path getPackageStoreDirPath(Path solrHome) {
+    return Paths.get(solrHome.toAbsolutePath().toString(), PackageStoreAPI.PACKAGESTORE_DIRECTORY).toAbsolutePath();
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/filestore/PackageStore.java b/solr/core/src/java/org/apache/solr/filestore/PackageStore.java
new file mode 100644
index 0000000..b9be691
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/filestore/PackageStore.java
@@ -0,0 +1,122 @@
+/*
+ * 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.filestore;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.filestore.PackageStoreAPI.MetaData;
+import org.apache.zookeeper.server.ByteBufferInputStream;
+
+/**
+ * The interface to be implemented by any package store provider
+ * * @lucene.experimental
+ */
+public interface PackageStore {
+
+  /**
+   * Store a file into the filestore. This should ensure that it is replicated
+   * across all nodes in the cluster
+   */
+  void put(FileEntry fileEntry) throws IOException;
+
+  /**
+   * read file content from a given path
+   */
+  void get(String path, Consumer<FileEntry> filecontent) throws IOException;
+
+  /**
+   * Fetch a resource from another node
+   * internal
+   */
+  boolean fetch(String path, String from);
+
+  List<FileDetails> list(String path, Predicate<String> predicate);
+
+  /**
+   * get the real path on filesystem
+   */
+  Path getRealpath(String path);
+
+  /**
+   * The type of the resource
+   */
+  FileType getType(String path);
+
+  public class FileEntry {
+    final ByteBuffer buf;
+    final MetaData meta;
+    final String path;
+
+    FileEntry(ByteBuffer buf, MetaData meta, String path) {
+      this.buf = buf;
+      this.meta = meta;
+      this.path = path;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+
+    public InputStream getInputStream() {
+      if (buf != null) return new ByteBufferInputStream(buf);
+      return null;
+
+    }
+
+    /**
+     * For very large files , only a stream would be available
+     * This method would return null;
+     */
+    public ByteBuffer getBuffer() {
+      return buf;
+
+    }
+
+    public MetaData getMetaData() {
+      return meta;
+    }
+
+
+  }
+
+  enum FileType {
+    FILE, DIRECTORY, NOFILE, METADATA
+  }
+
+  interface FileDetails extends MapWriter {
+
+    MetaData getMetaData();
+
+    Date getTimeStamp();
+
+    boolean isDir();
+
+
+  }
+
+
+}
diff --git a/solr/core/src/java/org/apache/solr/filestore/PackageStoreAPI.java b/solr/core/src/java/org/apache/solr/filestore/PackageStoreAPI.java
new file mode 100644
index 0000000..71ee9d8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/filestore/PackageStoreAPI.java
@@ -0,0 +1,273 @@
+/*
+ * 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.filestore;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.solr.api.Command;
+import org.apache.solr.api.EndPoint;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.cloud.CloudUtil;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.SolrException;
+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.ContentStream;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.PermissionNameProvider;
+import org.apache.solr.util.CryptoKeys;
+import org.apache.solr.util.SimplePostTool;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.server.ByteBufferInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.solr.handler.ReplicationHandler.FILE_STREAM;
+
+
+public class PackageStoreAPI {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  public static final String PACKAGESTORE_DIRECTORY = "filestore";
+
+
+  private final CoreContainer coreContainer;
+  PackageStore packageStore;
+  public final FSRead readAPI = new FSRead();
+  public final FSWrite writeAPI = new FSWrite();
+
+  public PackageStoreAPI(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+    packageStore = new DistribPackageStore(coreContainer);
+  }
+
+  public PackageStore getPackageStore() {
+    return packageStore;
+  }
+
+  @EndPoint(
+      path = "/cluster/files/*",
+      method = SolrRequest.METHOD.PUT,
+      permission = PermissionNameProvider.Name.FILESTORE_WRITE_PERM)
+  public class FSWrite {
+
+    static final String TMP_ZK_NODE = "/packageStoreWriteInProgress";
+
+    @Command
+    public void upload(SolrQueryRequest req, SolrQueryResponse rsp) {
+      try {
+        coreContainer.getZkController().getZkClient().create(TMP_ZK_NODE, "true".getBytes(UTF_8),
+            CreateMode.EPHEMERAL, true);
+
+        Iterable<ContentStream> streams = req.getContentStreams();
+        if (streams == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no payload");
+        String path = req.getPathTemplateValues().get("*");
+        if (path == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No path");
+        }
+        validateName(path);
+        ContentStream stream = streams.iterator().next();
+        try {
+          ByteBuffer buf = SimplePostTool.inputStreamToByteArray(stream.getStream());
+          String sha512 = DigestUtils.sha512Hex(new ByteBufferInputStream(buf));
+          List<String> signatures = readSignatures(req, buf);
+          Map<String, Object> vals = new HashMap<>();
+          vals.put(MetaData.SHA512, sha512);
+          if (signatures != null) {
+            vals.put("sig", signatures);
+          }
+          packageStore.put(new PackageStore.FileEntry(buf, new MetaData(vals), path));
+          rsp.add(CommonParams.FILE, path);
+        } catch (IOException e) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+        }
+      } catch (InterruptedException e) {
+        log.error("Unexpected error", e);
+      } catch (KeeperException.NodeExistsException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "A write is already in process , try later");
+      } catch (KeeperException e) {
+        log.error("Unexpected error", e);
+      } finally {
+        try {
+          coreContainer.getZkController().getZkClient().delete(TMP_ZK_NODE, -1, true);
+        } catch (Exception e) {
+          log.error("Unexpected error  ", e);
+        }
+      }
+    }
+
+    private List<String> readSignatures(SolrQueryRequest req, ByteBuffer buf)
+        throws SolrException {
+      String[] signatures = req.getParams().getParams("sig");
+      if (signatures == null || signatures.length == 0) return null;
+      List<String> sigs = Arrays.asList(signatures);
+      validate(sigs, buf);
+      return sigs;
+    }
+
+    public void validate(List<String> sigs,
+                         ByteBuffer buf) throws SolrException {
+      Map<String, byte[]> keys = CloudUtil.getTrustedKeys(
+          coreContainer.getZkController().getZkClient(), "exe");
+      if (keys == null || keys.isEmpty()) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "ZK does not have any keys");
+      }
+      CryptoKeys cryptoKeys = null;
+      try {
+        cryptoKeys = new CryptoKeys(keys);
+      } catch (Exception e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Error parsing public keyts in ZooKeeper");
+      }
+      for (String sig : sigs) {
+        if (cryptoKeys.verify(sig, buf) == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Signature does not match any public key : " + sig);
+        }
+
+      }
+    }
+
+  }
+
+  @EndPoint(
+      path = "/node/files/*",
+      method = SolrRequest.METHOD.GET,
+      permission = PermissionNameProvider.Name.FILESTORE_READ_PERM)
+  public class FSRead {
+    @Command
+    public void read(SolrQueryRequest req, SolrQueryResponse rsp) {
+      String path = req.getPathTemplateValues().get("*");
+      String pathCopy = path;
+      String getFrom = req.getParams().get("getFrom");
+      if (getFrom != null) {
+        coreContainer.getUpdateShardHandler().getUpdateExecutor().submit(() -> {
+          log.debug("Downloading file {}", pathCopy);
+          try {
+            packageStore.fetch(pathCopy, getFrom);
+          } catch (Exception e) {
+            log.error("Failed to download file: " + pathCopy, e);
+          }
+          log.info("downloaded file: {}", pathCopy);
+        });
+        return;
+
+      }
+      if (path == null) {
+        path = "";
+      }
+
+      PackageStore.FileType typ = packageStore.getType(path);
+      if (typ == PackageStore.FileType.NOFILE) {
+        rsp.add("files", Collections.singletonMap(path, null));
+        return;
+      }
+      if (typ == PackageStore.FileType.DIRECTORY) {
+        rsp.add("files", Collections.singletonMap(path, packageStore.list(path, null)));
+        return;
+      }
+      if (req.getParams().getBool("meta", false)) {
+        if (typ == PackageStore.FileType.FILE) {
+          int idx = path.lastIndexOf('/');
+          String fileName = path.substring(idx + 1);
+          String parentPath = path.substring(0, path.lastIndexOf('/'));
+          List l = packageStore.list(parentPath, s -> s.equals(fileName));
+          rsp.add("files", Collections.singletonMap(path, l.isEmpty() ? null : l.get(0)));
+          return;
+        }
+      } else {
+        writeRawFile(req, rsp, path);
+      }
+    }
+
+    private void writeRawFile(SolrQueryRequest req, SolrQueryResponse rsp, String path) {
+      ModifiableSolrParams solrParams = new ModifiableSolrParams();
+      solrParams.add(CommonParams.WT, FILE_STREAM);
+      req.setParams(SolrParams.wrapDefaults(solrParams, req.getParams()));
+      rsp.add(FILE_STREAM, (SolrCore.RawWriter) os -> {
+        packageStore.get(path, (it) -> {
+          try {
+            org.apache.commons.io.IOUtils.copy(it.getInputStream(), os);
+          } catch (IOException e) {
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading file" + path);
+          }
+        });
+
+      });
+    }
+
+  }
+
+  static class MetaData implements MapWriter {
+    public static final String SHA512 = "sha512";
+    String sha512;
+    List<String> signatures;
+    Map<String, Object> otherAttribs;
+
+    public MetaData(Map m) {
+      m = Utils.getDeepCopy(m, 3);
+      this.sha512 = (String) m.remove(SHA512);
+      this.signatures = (List<String>) m.remove("sig");
+      this.otherAttribs = m;
+    }
+
+    @Override
+    public void writeMap(EntryWriter ew) throws IOException {
+      ew.putIfNotNull("sha512", sha512);
+      ew.putIfNotNull("sig", signatures);
+      if (!otherAttribs.isEmpty()) {
+        otherAttribs.forEach(ew.getBiConsumer());
+      }
+    }
+  }
+
+  static final String INVALIDCHARS = " /\\#&*\n\t%@~`=+^$><?{}[]|:;!";
+
+  public static void validateName(String path) {
+    if (path == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "empty path");
+    }
+    List<String> parts = StrUtils.splitSmart(path, '/', true);
+    for (String part : parts) {
+      if (part.charAt(0) == '.') {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "cannot start with period");
+      }
+      for (int i = 0; i < part.length(); i++) {
+        for (int j = 0; j < INVALIDCHARS.length(); j++) {
+          if (part.charAt(i) == INVALIDCHARS.charAt(j))
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unsupported char in file name: " + part);
+        }
+      }
+    }
+  }
+}
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 79b4d29..a4c7c0d 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -51,6 +51,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),
+    FILESTORE_READ_PERM("filestore-read", null),
+    FILESTORE_WRITE_PERM("filestore-write", null),
     ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
     ;
     final String name;
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
index 19a4a30..7c21ad1 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
@@ -63,6 +63,7 @@ import org.apache.solr.util.RTimerTree;
 import org.apache.solr.util.SolrFileCleaningTracker;
 import org.apache.solr.util.tracing.GlobalTracer;
 
+import static org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONTENT_TYPE;
 import static org.apache.solr.common.params.CommonParams.PATH;
 
 
@@ -732,6 +733,7 @@ public class SolrRequestParsers
       String contentType = req.getContentType();
       String method = req.getMethod(); // No need to uppercase... HTTP verbs are case sensitive
       String uri = req.getRequestURI();
+      boolean isRawPut = "PUT".equals(method) && BINARY_CONTENT_TYPE.equals(contentType);
       boolean isPost = "POST".equals(method);
 
       // SOLR-6787 changed the behavior of a POST without content type.  Previously it would throw an exception,
@@ -747,7 +749,7 @@ public class SolrRequestParsers
       // POST was handled normally, but other methods (PUT/DELETE)
       // were handled by restlet if the URI contained /schema or /config
       // "handled by restlet" means that we don't attempt to handle any request body here.
-      if (!isPost) {
+      if (!isPost && !isRawPut) {
         if (contentType == null) {
           return parseQueryString(req.getQueryString());
         }
diff --git a/solr/core/src/java/org/apache/solr/util/CryptoKeys.java b/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
index faf67fd..cb368e7 100644
--- a/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
+++ b/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
@@ -21,7 +21,8 @@ import javax.crypto.Cipher;
 import javax.crypto.IllegalBlockSizeException;
 import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
-
+import java.io.IOException;
+import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
@@ -42,6 +43,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.Base64;
 import org.slf4j.Logger;
@@ -61,7 +63,7 @@ public final class CryptoKeys {
       m.put(e.getKey(), getX509PublicKey(e.getValue()));
 
     }
-    this.keys = m;
+    this.keys = ImmutableMap.copyOf(m);
   }
 
   /**
@@ -73,11 +75,11 @@ public final class CryptoKeys {
       boolean verified;
       try {
         verified = CryptoKeys.verify(entry.getValue(), Base64.base64ToByteArray(sig), data);
-        log.info("verified {} ", verified);
+        log.debug("verified {} ", verified);
         if (verified) return entry.getKey();
       } catch (Exception e) {
         exception = e;
-        log.info("NOT verified  ");
+        log.debug("NOT verified  ");
       }
 
     }
@@ -104,24 +106,43 @@ public final class CryptoKeys {
    * @param data      The data tha is signed
    */
   public static boolean verify(PublicKey publicKey, byte[] sig, ByteBuffer data) throws InvalidKeyException, SignatureException {
-    int oldPos = data.position();
-    Signature signature = null;
+    data = ByteBuffer.wrap(data.array(), data.arrayOffset(), data.limit());
     try {
-      signature = Signature.getInstance("SHA1withRSA");
+      Signature signature = Signature.getInstance("SHA1withRSA");
       signature.initVerify(publicKey);
       signature.update(data);
-      boolean verify = signature.verify(sig);
-      return verify;
+      return signature.verify(sig);
+    } catch (NoSuchAlgorithmException e) {
+      //wil not happen
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    }
 
+  }
+
+  public static boolean verify(PublicKey publicKey, byte[] sig, InputStream is)
+      throws InvalidKeyException, SignatureException, IOException {
+    try {
+      Signature signature = Signature.getInstance("SHA1withRSA");
+      signature.initVerify(publicKey);
+      byte[] buf = new byte[1024];
+      while (true) {
+        int sz = is.read(buf);
+        if (sz == -1) break;
+        signature.update(buf, 0, sz);
+      }
+      try {
+        return signature.verify(sig);
+      } catch (SignatureException e) {
+        return false;
+      }
     } catch (NoSuchAlgorithmException e) {
       //will not happen
-    } finally {
-      //Signature.update resets the position. set it back to old
-      data.position(oldPos);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
-    return false;
+
   }
 
+
   private static byte[][] evpBytesTokey(int key_len, int iv_len, MessageDigest md,
                                         byte[] salt, byte[] data, int count) {
     byte[][] both = new byte[2][];
diff --git a/solr/core/src/test-files/cryptokeys/priv_key512.pem b/solr/core/src/test-files/cryptokeys/priv_key512.pem
new file mode 100644
index 0000000..53c032c
--- /dev/null
+++ b/solr/core/src/test-files/cryptokeys/priv_key512.pem
@@ -0,0 +1,9 @@
+-----BEGIN RSA PRIVATE KEY-----
+MIIBOQIBAAJBAMgmSVfl+E2Nj/oKAP2TceWh17pk6Ugsw5A5nLH+OeMB/WeWJZg/
+NEDda8SXfQDEVRmw5P+2IZypPASzfCrc6yECAwEAAQJAbZFwEztky+fUSrhRRIAE
+GQaZV4PIpWdEA99WJaabv+YsWN5UUd7y+Evu50mhH3RQIxQd+R6SYs1ke9OlHlV2
+cQIhAP8367gybVEu2A+Cg1fE9vbHfnHrurpDQrh9r0ZKooTtAiEAyMMxvlHlSh6Q
+2cUTSxuyUEaQfN+W4efehgfIWBVlzIUCIEHBMZ0qeNnCvO36DUbuu0ZHjb9iIaDd
+tXH9B8yPbCHdAiAaV3o0ZZx3MDGDUVdpuHWaENguekva0kihP24rGIul3QIgNqZS
+EzA2aoQdNPl5oDfkhqAGjs5pb7qLgtmXJvVhi/Q=
+-----END RSA PRIVATE KEY-----
diff --git a/solr/core/src/test-files/cryptokeys/pub_key512.der b/solr/core/src/test-files/cryptokeys/pub_key512.der
new file mode 100644
index 0000000..4c926dd
Binary files /dev/null and b/solr/core/src/test-files/cryptokeys/pub_key512.der differ
diff --git a/solr/core/src/test-files/runtimecode/runtimelibs_v3.jar.bin b/solr/core/src/test-files/runtimecode/runtimelibs_v3.jar.bin
new file mode 100644
index 0000000..6b5bad6
Binary files /dev/null and b/solr/core/src/test-files/runtimecode/runtimelibs_v3.jar.bin differ
diff --git a/solr/core/src/test-files/runtimecode/sig.txt b/solr/core/src/test-files/runtimecode/sig.txt
new file mode 100644
index 0000000..4ef8e9c
--- /dev/null
+++ b/solr/core/src/test-files/runtimecode/sig.txt
@@ -0,0 +1,105 @@
+================priv_key2048.pem===================
+
+openssl dgst -sha1 -sign ../cryptokeys/priv_key2048.pem runtimelibs.jar.bin | openssl enc -base64
+
+NaTm3+i99/ZhS8YRsLc3NLz2Y6VuwEbu7DihY8GAWwWIGm+jpXgn1JiuaenfxFCc
+fNKCC9WgZmEgbTZTzmV/OZMVn90u642YJbF3vTnzelW1pHB43ZRAJ1iesH0anM37
+w03n3es+vFWQtuxc+2Go888fJoMkUX2C6Zk6Jn116KE45DWjeyPM4mp3vvGzwGvd
+RxP5K9Q3suA+iuI/ULXM7m9mV4ruvs/MZvL+ELm5Jnmk1bBtixVJhQwJP2z++8tQ
+KJghhyBxPIC/2fkAHobQpkhZrXu56JjP+v33ul3Ku4bbvfVMY/LVwCAEnxlvhk+C
+6uRCKCeFMrzQ/k5inasXLw==
+
+
+openssl dgst -sha1 -sign ../cryptokeys/priv_key2048.pem runtimelibs_v2.jar.bin | openssl enc -base64
+
+jsPpNMs74ogRbx9M4n/OH3j3s85KOq9dOtgGJkUf6O5D8T9d9zU2lDwxnTYjQCaW
+cRTLGH3Z8vpc0wyT3g4aXepgLUTSnrepbPffSFhQtFrCNxurPOLzbp6ERhwjZ0RL
+GvZrlbbjR2SxqZ3BpHiGxslj0tPCkdevNCEy1glLhl8RWG5xsLCrRL1mrEtLg97A
+53oCCrfGAHLEvW+olGeB1r7jqUaSrbfAUfDMSIvZfOIV+xdlvabkNiuzvsAc+B6Q
+pXWm+Em2f5TO/bkOh2m/UInGXcNHCa0oqRMGKP1H252Cv9eXm/d0h3Dqxv+f80Gz
+LfyA6/OKQ9FfskY4pltCsQ==
+
+openssl dgst -sha1 -sign ../cryptokeys/priv_key2048.pem runtimelibs_v3.jar.bin | openssl enc -base64
+
+
+YxFr6SpYrDwG85miDfRWHTjU9UltjtIWQZEhcV55C2rczRUVowCYBxmsDv5mAM8j
+0CTv854xpI1DtBT86wpoTdbF95LQuP9FJId4TS1j8bZ9cxHP5Cqyz1uBHFfUUNUr
+npzTHQkVTp02O9NAjh3c2W41bL4U7j6jQ32+4CW2M+x00TDG0y0H75rQDR8zbLt3
+1oWCz+sBOdZ3rGKJgAvdoGm/wVCTmsabZN+xoz4JaDeBXF16O9Uk9SSq4G0dz5YX
+FuLxHK7ciB5t0+q6pXlF/tdlDqF76Abze0R3d2/0MhXBzyNp3UxJmj6DiprgysfB
+0TbQtJG0XGfdSmx0VChvcA==
+
+YxFr6SpYrDwG85miDfRWHTjU9UltjtIWQZEhcV55C2rczRUVowCYBxmsDv5mAM8j0CTv854xpI1DtBT86wpoTdbF95LQuP9FJId4TS1j8bZ9cxHP5Cqyz1uBHFfUUNUrnpzTHQkVTp02O9NAjh3c2W41bL4U7j6jQ32+4CW2M+x00TDG0y0H75rQDR8zbLt31oWCz+sBOdZ3rGKJgAvdoGm/wVCTmsabZN+xoz4JaDeBXF16O9Uk9SSq4G0dz5YXFuLxHK7ciB5t0+q6pXlF/tdlDqF76Abze0R3d2/0MhXBzyNp3UxJmj6DiprgysfB0TbQtJG0XGfdSmx0VChvcA==
+
+=====================priv_key512.pem=====================
+openssl dgst -sha1 -sign ../cryptokeys/priv_key512.pem runtimelibs.jar.bin | openssl enc -base64
+
+L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1
+f/U3bOlMPINlSOM6LK3JpQ==
+
+openssl dgst -sha1 -sign ../cryptokeys/priv_key512.pem runtimelibs_v2.jar.bin | openssl enc -base64
+
+j+Rflxi64tXdqosIhbusqi6GTwZq8znunC/dzwcWW0/dHlFGKDurOaE1Nz9FSPJu
+XbHkVLj638yZ0Lp1ssnoYA==
+
+openssl dgst -sha1 -sign ../cryptokeys/priv_key512.pem runtimelibs_v3.jar.bin | openssl enc -base64
+
+a400n4T7FT+2gM0SC6+MfSOExjud8MkhTSFylhvwNjtWwUgKdPFn434Wv7Qc4QEq
+DVLhQoL3WqYtQmLPti0G4Q==
+
+openssl dgst -sha1 -sign ../cryptokeys/priv_key512.pem cache.jar.bin | openssl enc -base64
+
+A2CDnReirpII005KRN1C3pvt4NM4kItsagQPNaa3ljj/5R3LKVgiPuNvqBsffU8n
+81LOAfr5VMyGFcb4QMHpyg==
+
+openssl dgst -sha1 -sign ../cryptokeys/priv_key512.pem cache_v2.jar.bin | openssl enc -base64
+
+SOrekHt+uup+z2z+nZU5indk2huRRfmbM+W+vQ0variHrcZEG9EXt5LuPFl8Ki9A
+hr6klMHdVP8nj4wuQhu/Hg==
+
+====================sha512====================
+
+openssl dgst -sha512 runtimelibs.jar.bin
+
+d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420
+
+
+openssl dgst -sha512 runtimelibs_v2.jar.bin
+
+bc5ce45ad281b6a08fb7e529b1eb475040076834816570902acb6ebdd809410e31006efdeaa7f78a6c35574f3504963f5f7e4d92247d0eb4db3fc9abdda5d417
+
+openssl dgst -sha512 runtimelibs_v3.jar.bin
+
+60ec88c2a2e9b409f7afc309273383810a0d07a078b482434eda9674f7e25b8adafa8a67c9913c996cbfb78a7f6ad2b9db26dbd4fe0ca4068f248d5db563f922
+
+openssl dgst -sha512 cache.jar.bin
+
+8946650ba88919cea2f81e4771c418411f61837b2a276088c2f2c86ef2d730f152ccf5975fa8a2c7035a1f00de1994a7788676d95dc7ccea6aaf28c7fff1f46b
+
+openssl dgst -sha512 cache_v2.jar.bin
+
+873337e67b90b7ff99df012b2e9093c63079c37a564643d34861a88c4cbaf0698ebb096905929d69cdbde3b4d29d55e31db24ee05c01b39c0b75a16e54eb4335
+
+=============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
+
+32e8b5b2a95ea306538b52017f0954aa1b0f8a8b2d0acbc498fd0e66a223f7bd
+
+openssl dgst -sha256 cache_v2.jar.bin
+
+0f670f6dcc2b00f9a448a7ebd457d4ff985ab702c85cdb3608dcae9889e8d702
+
+
diff --git a/solr/core/src/test/org/apache/solr/filestore/TestDistribPackageStore.java b/solr/core/src/test/org/apache/solr/filestore/TestDistribPackageStore.java
new file mode 100644
index 0000000..a99028a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/filestore/TestDistribPackageStore.java
@@ -0,0 +1,250 @@
+/*
+ * 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.filestore;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.function.Predicate;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.commons.codec.digest.DigestUtils;
+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.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient.RemoteExecutionException;
+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.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.NavigableObject;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.LogLevel;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.server.ByteBufferInputStream;
+
+import static org.apache.solr.common.util.Utils.JAVABINCONSUMER;
+import static org.apache.solr.core.TestDynamicLoading.getFileContent;
+
+@LogLevel("org.apache.solr.core.PackageStoreAPI=DEBUG;org.apache.solr.core.DistribPackageStore=DEBUG")
+public class TestDistribPackageStore extends SolrCloudTestCase {
+
+  public void testPackageStoreManagement() throws Exception {
+    MiniSolrCloudCluster cluster =
+        configureCluster(4)
+        .withJettyConfig(jetty -> jetty.enableV2(true))
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    try {
+
+      byte[] derFile = readFile("cryptokeys/pub_key512.der");
+      cluster.getZkClient().makePath("/keys/exe", true);
+      cluster.getZkClient().create("/keys/exe/pub_key512.der", derFile, CreateMode.PERSISTENT, true);
+
+      try {
+        postFile(cluster.getSolrClient(), getFileContent("runtimecode/runtimelibs.jar.bin"),
+            "/package/mypkg/v1.0/runtimelibs.jar",
+            "j+Rflxi64tXdqosIhbusqi6GTwZq8znunC/dzwcWW0/dHlFGKDurOaE1Nz9FSPJuXbHkVLj638yZ0Lp1ssnoYA=="
+        );
+        fail("should have failed because of wrong signature ");
+      } catch (RemoteExecutionException e) {
+        assertTrue(e.getMessage().contains("Signature does not match"));
+      }
+
+      postFile(cluster.getSolrClient(), getFileContent("runtimecode/runtimelibs.jar.bin"),
+          "/package/mypkg/v1.0/runtimelibs.jar",
+          "L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1f/U3bOlMPINlSOM6LK3JpQ=="
+          );
+
+      assertResponseValues(10,
+          cluster.getSolrClient(),
+          new V2Request.Builder("/node/files/package/mypkg/v1.0")
+              .withMethod(SolrRequest.METHOD.GET)
+              .build(),
+          Utils.makeMap(
+              ":files:/package/mypkg/v1.0[0]:name", "runtimelibs.jar",
+              ":files:/package/mypkg/v1.0[0]:sha512", "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420",
+              ":files:/package/mypkg/v1.0[0]:sig[0]", "L3q/qIGs4NaF6JiO0ZkMUFa88j0OmYc+I6O7BOdNuMct/xoZ4h73aZHZGc0+nmI1f/U3bOlMPINlSOM6LK3JpQ=="
+          )
+      );
+
+      assertResponseValues(10,
+          cluster.getSolrClient(),
+          new V2Request.Builder("/node/files/package/mypkg")
+              .withMethod(SolrRequest.METHOD.GET)
+              .build(),
+          Utils.makeMap(
+              ":files:/package/mypkg[0]:name", "v1.0",
+              ":files:/package/mypkg[0]:dir", "true"
+          )
+      );
+
+      class Fetcher implements Callable {
+        String url;
+        JettySolrRunner jetty;
+        Fetcher(String s, JettySolrRunner jettySolrRunner){
+          this.url = s;
+          this.jetty = jettySolrRunner;
+        }
+        @Override
+        public NavigableObject call() throws Exception {
+          try (HttpSolrClient solrClient = (HttpSolrClient) jetty.newClient()) {
+            return (NavigableObject) Utils.executeGET(solrClient.getHttpClient(), this.url, JAVABINCONSUMER);
+          }
+        }
+
+        @Override
+        public String toString() {
+          return url;
+        }
+
+      }
+
+      Map expected = Utils.makeMap(
+          ":files:/package/mypkg/v1.0/runtimelibs.jar:name", "runtimelibs.jar",
+          ":files:/package/mypkg/v1.0[0]:sha512", "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420"
+
+      );
+      for (JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
+        String baseUrl = jettySolrRunner.getBaseUrl().toString().replace("/solr", "/api");
+        String url = baseUrl + "/node/files/package/mypkg/v1.0/runtimelibs.jar?wt=javabin&meta=true";
+
+        assertResponseValues(10, new Fetcher(url, jettySolrRunner), expected);
+
+        try (HttpSolrClient solrClient = (HttpSolrClient) jettySolrRunner.newClient()) {
+          ByteBuffer buf = Utils.executeGET(solrClient.getHttpClient(), baseUrl + "/node/files/package/mypkg/v1.0/runtimelibs.jar",
+              Utils.newBytesConsumer(Integer.MAX_VALUE));
+          assertEquals(
+              "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420",
+              DigestUtils.sha512Hex(new ByteBufferInputStream(buf))
+          );
+
+        }
+
+      }
+
+      postFile(cluster.getSolrClient(), getFileContent("runtimecode/runtimelibs_v2.jar.bin"),
+          "/package/mypkg/v1.0/runtimelibs_v2.jar",
+          null
+      );
+
+      expected = Utils.makeMap(
+          ":files:/package/mypkg/v1.0", (Predicate<Object>) o -> {
+            List l = (List) o;
+            assertEquals(2, l.size());
+            Set expectedKeys = ImmutableSet.of("runtimelibs_v2.jar", "runtimelibs.jar");
+            for (Object file : l) {
+              if(! expectedKeys.contains(Utils.getObjectByPath(file, true, "name"))) return false;
+            }
+
+            return true;
+          }
+      );
+      for (JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
+        String baseUrl = jettySolrRunner.getBaseUrl().toString().replace("/solr", "/api");
+        String url = baseUrl + "/node/files/package/mypkg/v1.0?wt=javabin";
+
+        assertResponseValues(10, new Fetcher(url, jettySolrRunner), expected);
+
+      }
+
+
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  public static NavigableObject assertResponseValues(int repeats, SolrClient client, SolrRequest req, Map vals) throws Exception {
+    Callable<NavigableObject> callable = () -> req.process(client);
+
+    return assertResponseValues(repeats, callable,vals);
+  }
+
+  public static NavigableObject assertResponseValues(int repeats,  Callable<NavigableObject> callable,Map vals) throws Exception {
+    NavigableObject rsp = null;
+
+    for (int i = 0; i < repeats; i++) {
+      if (i > 0) {
+        Thread.sleep(100);
+      }
+      try {
+        rsp = callable.call();
+      } catch (Exception e) {
+        if (i >= repeats - 1) throw e;
+        continue;
+      }
+      for (Object e : vals.entrySet()) {
+        Map.Entry entry = (Map.Entry) e;
+        String k = (String) entry.getKey();
+        List<String> key = StrUtils.split(k, '/');
+
+        Object val = entry.getValue();
+        Predicate p = val instanceof Predicate ? (Predicate) val : o -> {
+          String v = o == null ? null : String.valueOf(o);
+          return Objects.equals(val, o);
+        };
+        boolean isPass = p.test(rsp._get(key, null));
+        if (isPass) return rsp;
+        else if (i >= repeats - 1) {
+          fail("req: " + callable.toString() +" . attempt: " + i + " Mismatch for value : '" + key + "' in response , " + Utils.toJSONString(rsp));
+        }
+
+      }
+
+    }
+    return rsp;
+  }
+
+
+
+  private void postFile(SolrClient client, ByteBuffer buffer, String name, String sig)
+      throws SolrServerException, IOException {
+    String resource = "/cluster/files" + name;
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    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);
+    assertEquals(name, rsp.getResponse().get(CommonParams.FILE));
+  }
+
+  public static byte[] readFile(String fname) throws IOException {
+    byte[] buf = null;
+    try (FileInputStream fis = new FileInputStream(getFile(fname))) {
+      buf = new byte[fis.available()];
+      fis.read(buf);
+    }
+    return buf;
+  }
+}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrResponse.java b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrResponse.java
index ef52eb2..73eb863 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrResponse.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrResponse.java
@@ -18,10 +18,12 @@ package org.apache.solr.client.solrj;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
 
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
@@ -32,7 +34,7 @@ import org.apache.solr.common.util.NamedList;
  * 
  * @since solr 1.3
  */
-public abstract class SolrResponse implements Serializable {
+public abstract class SolrResponse implements Serializable, MapWriter {
 
   /** Elapsed time in milliseconds for the request as seen from the client. */
   public abstract long getElapsedTime();
@@ -43,6 +45,11 @@ public abstract class SolrResponse implements Serializable {
   
   public abstract NamedList<Object> getResponse();
 
+  @Override
+  public void writeMap(EntryWriter ew) throws IOException {
+    getResponse().writeMap(ew);
+  }
+
   public Exception getException() {
     NamedList exp = (NamedList) getResponse().get("exception");
     if (exp == null) {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
index 4236177..5334edd 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
@@ -18,11 +18,15 @@
 package org.apache.solr.client.solrj.request;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.response.V2Response;
@@ -42,8 +46,10 @@ public class V2Request extends SolrRequest<V2Response> implements MapWriter {
   private SolrParams solrParams;
   public final boolean useBinary;
   private String collection;
+  private String mimeType;
   private boolean forceV2 = false;
   private boolean isPerCollectionRequest = false;
+  private ResponseParser parser;
 
   private V2Request(METHOD m, String resource, boolean useBinary) {
     super(m, resource);
@@ -56,7 +62,7 @@ public class V2Request extends SolrRequest<V2Response> implements MapWriter {
 
   }
 
-  public boolean isForceV2(){
+  public boolean isForceV2() {
     return forceV2;
   }
 
@@ -75,6 +81,15 @@ public class V2Request extends SolrRequest<V2Response> implements MapWriter {
     return new RequestWriter.ContentWriter() {
       @Override
       public void write(OutputStream os) throws IOException {
+        if (payload instanceof ByteBuffer) {
+          ByteBuffer b = (ByteBuffer) payload;
+          os.write(b.array(), b.arrayOffset(), b.limit());
+          return;
+        }
+        if (payload instanceof InputStream) {
+          IOUtils.copy((InputStream) payload, os);
+          return;
+        }
         if (useBinary) {
           new JavaBinCodec().marshal(payload, os);
         } else {
@@ -84,6 +99,7 @@ public class V2Request extends SolrRequest<V2Response> implements MapWriter {
 
       @Override
       public String getContentType() {
+        if (mimeType != null) return mimeType;
         return useBinary ? JAVABIN_MIME : JSON_MIME;
       }
     };
@@ -111,6 +127,12 @@ public class V2Request extends SolrRequest<V2Response> implements MapWriter {
     ew.putIfNotNull("command", payload);
   }
 
+  @Override
+  public ResponseParser getResponseParser() {
+    if (parser != null) return parser;
+    return super.getResponseParser();
+  }
+
   public static class Builder {
     private String resource;
     private METHOD method = METHOD.GET;
@@ -119,6 +141,8 @@ public class V2Request extends SolrRequest<V2Response> implements MapWriter {
     private boolean useBinary = false;
 
     private boolean forceV2EndPoint = false;
+    private ResponseParser parser;
+    private String mimeType;
 
     /**
      * Create a Builder object based on the provided resource.
@@ -173,11 +197,24 @@ public class V2Request extends SolrRequest<V2Response> implements MapWriter {
       return this;
     }
 
+    public Builder withResponseParser(ResponseParser parser) {
+      this.parser = parser;
+      return this;
+    }
+
+    public Builder withMimeType(String mimeType) {
+      this.mimeType = mimeType;
+      return this;
+
+    }
+
     public V2Request build() {
       V2Request v2Request = new V2Request(method, resource, useBinary);
       v2Request.solrParams = params;
       v2Request.payload = payload;
       v2Request.forceV2 = forceV2EndPoint;
+      v2Request.mimeType = mimeType;
+      v2Request.parser = parser;
       return v2Request;
     }
   }
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
index 39a0242..6eda49b 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
@@ -296,5 +296,8 @@ public interface CommonParams {
 
   String JAVABIN_MIME = "application/javabin";
 
+  String FILE = "file";
+  String FILES = "files";
+
 }
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java b/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
index c0b19f5..d153657 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java
@@ -30,8 +30,8 @@ import org.apache.solr.common.SolrException;
  *
  */
 public class StrUtils {
-  public static final char[] HEX_DIGITS = { '0', '1', '2', '3', '4', '5', '6',
-      '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f' };
+  public static final char[] HEX_DIGITS = {'0', '1', '2', '3', '4', '5', '6',
+      '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'};
 
   public static List<String> splitSmart(String s, char separator) {
     ArrayList<String> lst = new ArrayList<>(4);
@@ -40,9 +40,19 @@ public class StrUtils {
 
   }
 
+  static final String DELIM_CHARS = "/:;.,%#";
+
+  public static List<String> split(String s, char sep) {
+    if (DELIM_CHARS.indexOf(s.charAt(0)) > -1) {
+      sep = s.charAt(0);
+    }
+    return splitSmart(s, sep, true);
+
+  }
+
   public static List<String> splitSmart(String s, char separator, boolean trimEmpty) {
     List<String> l = splitSmart(s, separator);
-    if(trimEmpty){
+    if (trimEmpty) {
       if (l.size() > 0 && l.get(0).isEmpty()) l.remove(0);
     }
     return l;
@@ -54,77 +64,88 @@ public class StrUtils {
    * outside strings.
    */
   public static void splitSmart(String s, char separator, List<String> lst) {
-    int pos=0, start=0, end=s.length();
-    char inString=0;
-    char ch=0;
+    int pos = 0, start = 0, end = s.length();
+    char inString = 0;
+    char ch = 0;
     while (pos < end) {
-      char prevChar=ch;
+      char prevChar = ch;
       ch = s.charAt(pos++);
-      if (ch=='\\') {    // skip escaped chars
+      if (ch == '\\') {    // skip escaped chars
         pos++;
-      } else if (inString != 0 && ch==inString) {
-        inString=0;
-      } else if (ch=='\'' || ch=='"') {
+      } else if (inString != 0 && ch == inString) {
+        inString = 0;
+      } else if (ch == '\'' || ch == '"') {
         // If char is directly preceeded by a number or letter
         // then don't treat it as the start of a string.
         // Examples: 50" TV, or can't
         if (!Character.isLetterOrDigit(prevChar)) {
-          inString=ch;
+          inString = ch;
         }
-      } else if (ch==separator && inString==0) {
-        lst.add(s.substring(start,pos-1));
-        start=pos;
+      } else if (ch == separator && inString == 0) {
+        lst.add(s.substring(start, pos - 1));
+        start = pos;
       }
     }
     if (start < end) {
-      lst.add(s.substring(start,end));
+      lst.add(s.substring(start, end));
     }
 
     /***
-    if (SolrCore.log.isLoggable(Level.FINEST)) {
-      SolrCore.log.trace("splitCommand=" + lst);
-    }
-    ***/
+     if (SolrCore.log.isLoggable(Level.FINEST)) {
+     SolrCore.log.trace("splitCommand=" + lst);
+     }
+     ***/
 
   }
 
-  /** Splits a backslash escaped string on the separator.
+  /**
+   * Splits a backslash escaped string on the separator.
    * <p>
    * Current backslash escaping supported:
    * <br> \n \t \r \b \f are escaped the same as a Java String
    * <br> Other characters following a backslash are produced verbatim (\c =&gt; c)
    *
-   * @param s  the string to split
+   * @param s         the string to split
    * @param separator the separator to split on
-   * @param decode decode backslash escaping
+   * @param decode    decode backslash escaping
    * @return not null
    */
   public static List<String> splitSmart(String s, String separator, boolean decode) {
     ArrayList<String> lst = new ArrayList<>(2);
     StringBuilder sb = new StringBuilder();
-    int pos=0, end=s.length();
+    int pos = 0, end = s.length();
     while (pos < end) {
-      if (s.startsWith(separator,pos)) {
+      if (s.startsWith(separator, pos)) {
         if (sb.length() > 0) {
           lst.add(sb.toString());
-          sb=new StringBuilder();
+          sb = new StringBuilder();
         }
-        pos+=separator.length();
+        pos += separator.length();
         continue;
       }
 
       char ch = s.charAt(pos++);
-      if (ch=='\\') {
+      if (ch == '\\') {
         if (!decode) sb.append(ch);
-        if (pos>=end) break;  // ERROR, or let it go?
+        if (pos >= end) break;  // ERROR, or let it go?
         ch = s.charAt(pos++);
         if (decode) {
-          switch(ch) {
-            case 'n' : ch='\n'; break;
-            case 't' : ch='\t'; break;
-            case 'r' : ch='\r'; break;
-            case 'b' : ch='\b'; break;
-            case 'f' : ch='\f'; break;
+          switch (ch) {
+            case 'n':
+              ch = '\n';
+              break;
+            case 't':
+              ch = '\t';
+              break;
+            case 'r':
+              ch = '\r';
+              break;
+            case 'b':
+              ch = '\b';
+              break;
+            case 'f':
+              ch = '\f';
+              break;
           }
         }
       }
@@ -158,14 +179,15 @@ public class StrUtils {
     return result;
   }
 
-  /** 
-   * Creates a backslash escaped string, joining all the items. 
+  /**
+   * Creates a backslash escaped string, joining all the items.
+   *
    * @see #escapeTextWithSeparator
    */
   public static String join(Collection<?> items, char separator) {
     if (items == null) return "";
     StringBuilder sb = new StringBuilder(items.size() << 3);
-    boolean first=true;
+    boolean first = true;
     for (Object o : items) {
       String item = String.valueOf(o);
       if (first) {
@@ -179,32 +201,41 @@ public class StrUtils {
   }
 
 
-
   public static List<String> splitWS(String s, boolean decode) {
     ArrayList<String> lst = new ArrayList<>(2);
     StringBuilder sb = new StringBuilder();
-    int pos=0, end=s.length();
+    int pos = 0, end = s.length();
     while (pos < end) {
       char ch = s.charAt(pos++);
       if (Character.isWhitespace(ch)) {
         if (sb.length() > 0) {
           lst.add(sb.toString());
-          sb=new StringBuilder();
+          sb = new StringBuilder();
         }
         continue;
       }
 
-      if (ch=='\\') {
+      if (ch == '\\') {
         if (!decode) sb.append(ch);
-        if (pos>=end) break;  // ERROR, or let it go?
+        if (pos >= end) break;  // ERROR, or let it go?
         ch = s.charAt(pos++);
         if (decode) {
-          switch(ch) {
-            case 'n' : ch='\n'; break;
-            case 't' : ch='\t'; break;
-            case 'r' : ch='\r'; break;
-            case 'b' : ch='\b'; break;
-            case 'f' : ch='\f'; break;
+          switch (ch) {
+            case 'n':
+              ch = '\n';
+              break;
+            case 't':
+              ch = '\t';
+              break;
+            case 'r':
+              ch = '\r';
+              break;
+            case 'b':
+              ch = '\b';
+              break;
+            case 'f':
+              ch = '\f';
+              break;
           }
         }
       }
@@ -228,46 +259,48 @@ public class StrUtils {
   }
 
 
-
-  /** Return if a string starts with '1', 't', or 'T'
-   *  and return false otherwise.
+  /**
+   * Return if a string starts with '1', 't', or 'T'
+   * and return false otherwise.
    */
   public static boolean parseBoolean(String s) {
-    char ch = s.length()>0 ? s.charAt(0) : 0;
-    return (ch=='1' || ch=='t' || ch=='T');
+    char ch = s.length() > 0 ? s.charAt(0) : 0;
+    return (ch == '1' || ch == 't' || ch == 'T');
   }
-  
-  /** how to transform a String into a boolean... more flexible than
+
+  /**
+   * how to transform a String into a boolean... more flexible than
    * Boolean.parseBoolean() to enable easier integration with html forms.
    */
   public static boolean parseBool(String s) {
-    if( s != null ) {
-      if( s.startsWith("true") || s.startsWith("on") || s.startsWith("yes") ) {
+    if (s != null) {
+      if (s.startsWith("true") || s.startsWith("on") || s.startsWith("yes")) {
         return true;
       }
-      if( s.startsWith("false") || s.startsWith("off") || s.equals("no") ) {
+      if (s.startsWith("false") || s.startsWith("off") || s.equals("no")) {
         return false;
       }
     }
-    throw new SolrException( SolrException.ErrorCode.BAD_REQUEST, "invalid boolean value: "+s );
+    throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "invalid boolean value: " + s);
   }
 
   /**
    * {@link NullPointerException} and {@link SolrException} free version of {@link #parseBool(String)}
+   *
    * @return parsed boolean value (or def, if s is null or invalid)
    */
   public static boolean parseBool(String s, boolean def) {
-    if( s != null ) {
-      if( s.startsWith("true") || s.startsWith("on") || s.startsWith("yes") ) {
+    if (s != null) {
+      if (s.startsWith("true") || s.startsWith("on") || s.startsWith("yes")) {
         return true;
       }
-      if( s.startsWith("false") || s.startsWith("off") || s.equals("no") ) {
+      if (s.startsWith("false") || s.startsWith("off") || s.equals("no")) {
         return false;
       }
     }
     return def;
   }
-  
+
   /**
    * URLEncodes a value, replacing only enough chars so that
    * the URL may be unambiguously pasted back into a browser.
@@ -276,7 +309,7 @@ public class StrUtils {
    * &amp;,=,%,+,space are encoded.
    */
   public static void partialURLEncodeVal(Appendable dest, String val) throws IOException {
-    for (int i=0; i<val.length(); i++) {
+    for (int i = 0; i < val.length(); i++) {
       char ch = val.charAt(i);
       if (ch < 32) {
         dest.append('%');
@@ -284,46 +317,60 @@ public class StrUtils {
         dest.append(Integer.toHexString(ch));
       } else {
         switch (ch) {
-          case ' ': dest.append('+'); break;
-          case '&': dest.append("%26"); break;
-          case '%': dest.append("%25"); break;
-          case '=': dest.append("%3D"); break;
-          case '+': dest.append("%2B"); break;
-          default : dest.append(ch); break;
+          case ' ':
+            dest.append('+');
+            break;
+          case '&':
+            dest.append("%26");
+            break;
+          case '%':
+            dest.append("%25");
+            break;
+          case '=':
+            dest.append("%3D");
+            break;
+          case '+':
+            dest.append("%2B");
+            break;
+          default:
+            dest.append(ch);
+            break;
         }
       }
     }
   }
 
-  /** 
+  /**
    * Creates a new copy of the string with the separator backslash escaped.
+   *
    * @see #join
    */
   public static String escapeTextWithSeparator(String item, char separator) {
     StringBuilder sb = new StringBuilder(item.length() * 2);
     appendEscapedTextToBuilder(sb, item, separator);
     return sb.toString();
-  }  
+  }
 
   /**
-   * writes chars from item to out, backslash escaping as needed based on separator -- 
+   * writes chars from item to out, backslash escaping as needed based on separator --
    * but does not append the separator itself
    */
-  public static void appendEscapedTextToBuilder(StringBuilder out, 
-                                                 String item, 
-                                                 char separator) {
+  public static void appendEscapedTextToBuilder(StringBuilder out,
+                                                String item,
+                                                char separator) {
     for (int i = 0; i < item.length(); i++) {
       char ch = item.charAt(i);
-      if (ch == '\\' || ch == separator) { 
+      if (ch == '\\' || ch == separator) {
         out.append('\\');
       }
       out.append(ch);
     }
   }
 
-  /**Format using MesssageFormat but with the ROOT locale
+  /**
+   * Format using MesssageFormat but with the ROOT locale
    */
-  public static String formatString(String pattern, Object... args)  {
+  public static String formatString(String pattern, Object... args) {
     return new MessageFormat(pattern, Locale.ROOT).format(args);
   }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index 4a8d987..db6ef37 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -28,6 +28,7 @@ import java.io.Writer;
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
 import java.net.URLDecoder;
+import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.AbstractMap;
@@ -53,6 +54,9 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 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.http.util.EntityUtils;
 import org.apache.solr.client.solrj.cloud.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
@@ -91,7 +95,7 @@ public class Utils {
   public static final Function NEW_SYNCHRONIZED_ARRAYLIST_FUN = o -> Collections.synchronizedList(new ArrayList<>());
   public static final Function NEW_HASHSET_FUN = o -> new HashSet<>();
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+
   public static Map getDeepCopy(Map map, int maxDepth) {
     return getDeepCopy(map, maxDepth, true, false);
   }
@@ -101,17 +105,17 @@ public class Utils {
   }
 
   public static Map getDeepCopy(Map map, int maxDepth, boolean mutable, boolean sorted) {
-    if(map == null) return null;
+    if (map == null) return null;
     if (maxDepth < 1) return map;
     Map copy;
     if (sorted) {
       copy = new TreeMap();
     } else {
-      copy = map instanceof LinkedHashMap?  new LinkedHashMap(map.size()): new HashMap(map.size());
+      copy = map instanceof LinkedHashMap ? new LinkedHashMap(map.size()) : new HashMap(map.size());
     }
     for (Object o : map.entrySet()) {
       Map.Entry e = (Map.Entry) o;
-      copy.put(e.getKey(), makeDeepCopy(e.getValue(),maxDepth, mutable, sorted));
+      copy.put(e.getKey(), makeDeepCopy(e.getValue(), maxDepth, mutable, sorted));
     }
     return mutable ? copy : Collections.unmodifiableMap(copy);
   }
@@ -151,7 +155,7 @@ public class Utils {
     } else if (v instanceof IteratorWriter && maxDepth > 1) {
       v = ((IteratorWriter) v).toList(new ArrayList<>());
       if (sorted) {
-        Collections.sort((List)v);
+        Collections.sort((List) v);
       }
     }
 
@@ -166,8 +170,8 @@ public class Utils {
   public static InputStream toJavabin(Object o) throws IOException {
     try (final JavaBinCodec jbc = new JavaBinCodec()) {
       BinaryRequestWriter.BAOS baos = new BinaryRequestWriter.BAOS();
-      jbc.marshal(o,baos);
-      return new ByteBufferInputStream(ByteBuffer.wrap(baos.getbuf(),0,baos.size()));
+      jbc.marshal(o, baos);
+      return new ByteBufferInputStream(ByteBuffer.wrap(baos.getbuf(), 0, baos.size()));
     }
   }
 
@@ -178,10 +182,10 @@ public class Utils {
   public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable, boolean sorted) {
     if (c == null || maxDepth < 1) return c;
     Collection result = c instanceof Set ?
-        ( sorted? new TreeSet() : new HashSet()) : new ArrayList();
+        (sorted ? new TreeSet() : new HashSet()) : new ArrayList();
     for (Object o : c) result.add(makeDeepCopy(o, maxDepth, mutable, sorted));
     if (sorted && (result instanceof List)) {
-      Collections.sort((List)result);
+      Collections.sort((List) result);
     }
     return mutable ? result : result instanceof Set ? unmodifiableSet((Set) result) : unmodifiableList((List) result);
   }
@@ -208,7 +212,7 @@ public class Utils {
     @Override
     public void handleUnknownClass(Object o) {
       if (o instanceof MapWriter) {
-        Map m = ((MapWriter)o).toMap(new LinkedHashMap<>());
+        Map m = ((MapWriter) o).toMap(new LinkedHashMap<>());
         write(m);
       } else {
         super.handleUnknownClass(o);
@@ -217,13 +221,13 @@ public class Utils {
   }
 
   public static byte[] toJSON(Object o) {
-    if(o == null) return new byte[0];
+    if (o == null) return new byte[0];
     CharArr out = new CharArr();
     if (!(o instanceof List) && !(o instanceof Map)) {
-      if (o instanceof MapWriter)  {
-        o = ((MapWriter)o).toMap(new LinkedHashMap<>());
-      } else if(o instanceof IteratorWriter){
-        o = ((IteratorWriter)o).toList(new ArrayList<>());
+      if (o instanceof MapWriter) {
+        o = ((MapWriter) o).toMap(new LinkedHashMap<>());
+      } else if (o instanceof IteratorWriter) {
+        o = ((IteratorWriter) o).toList(new ArrayList<>());
       }
     }
     new MapWriterJSONWriter(out, 2).write(o); // indentation by default
@@ -274,10 +278,11 @@ public class Utils {
     return propMap;
   }
 
-  public static Object fromJSON(InputStream is){
+  public static Object fromJSON(InputStream is) {
     return fromJSON(new InputStreamReader(is, UTF_8));
   }
-  public static Object fromJSON(Reader is){
+
+  public static Object fromJSON(Reader is) {
     try {
       return STANDARDOBJBUILDER.apply(getJSONParser(is)).getVal();
     } catch (IOException e) {
@@ -295,7 +300,7 @@ public class Utils {
   };
   public static final Function<JSONParser, ObjectBuilder> MAPWRITEROBJBUILDER = jsonParser -> {
     try {
-      return new ObjectBuilder(jsonParser){
+      return new ObjectBuilder(jsonParser) {
         @Override
         public Object newObject() {
           return new LinkedHashMapWriter();
@@ -308,7 +313,7 @@ public class Utils {
 
   public static final Function<JSONParser, ObjectBuilder> MAPOBJBUILDER = jsonParser -> {
     try {
-      return new ObjectBuilder(jsonParser){
+      return new ObjectBuilder(jsonParser) {
         @Override
         public Object newObject() {
           return new HashMap();
@@ -336,10 +341,11 @@ public class Utils {
       return fromJSON(stream);
     } catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-                              "Resource error: " + e.getMessage(), e);
+          "Resource error: " + e.getMessage(), e);
     }
   }
-  public static JSONParser getJSONParser(Reader reader){
+
+  public static JSONParser getJSONParser(Reader reader) {
     JSONParser parser = new JSONParser(reader);
     parser.setFlags(parser.getFlags() |
         JSONParser.ALLOW_MISSING_COLON_COMMA_BEFORE_OBJECT |
@@ -347,11 +353,11 @@ public class Utils {
     return parser;
   }
 
-  public static Object fromJSONString(String json)  {
+  public static Object fromJSONString(String json) {
     try {
       return STANDARDOBJBUILDER.apply(getJSONParser(new StringReader(json))).getVal();
     } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error : "+ json, e );
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error : " + json, e);
     }
   }
 
@@ -424,8 +430,8 @@ public class Utils {
 
 
   public static Object getObjectByPath(Object root, boolean onlyPrimitive, List<String> hierarchy) {
-    if(root == null) return null;
-    if(!isMapLike(root)) return null;
+    if (root == null) return null;
+    if (!isMapLike(root)) return null;
     Object obj = root;
     for (int i = 0; i < hierarchy.size(); i++) {
       int idx = -1;
@@ -518,6 +524,7 @@ public class Utils {
       try {
         ((MapWriter) obj).writeMap(new MapWriter.EntryWriter() {
           int count = -1;
+
           @Override
           public MapWriter.EntryWriter put(CharSequence k, Object v) {
             if (result[0] != null) return this;
@@ -533,15 +540,14 @@ public class Utils {
         throw new RuntimeException(e);
       }
       return result[0];
-    }
-    else if (obj instanceof Map) return ((Map) obj).get(key);
+    } else if (obj instanceof Map) return ((Map) obj).get(key);
     else throw new RuntimeException("must be a NamedList or Map");
   }
 
   /**
    * If the passed entity has content, make sure it is fully
    * read and closed.
-   * 
+   *
    * @param entity to consume or null
    */
   public static void consumeFully(HttpEntity entity) {
@@ -562,13 +568,14 @@ public class Utils {
 
   /**
    * Make sure the InputStream is fully read.
-   * 
+   *
    * @param is to read
    * @throws IOException on problem with IO
    */
   private static void readFully(InputStream is) throws IOException {
     is.skip(is.available());
-    while (is.read() != -1) {}
+    while (is.read() != -1) {
+    }
   }
 
   public static Map<String, Object> getJson(DistribStateManager distribStateManager, String path) throws InterruptedException, IOException, KeeperException {
@@ -585,8 +592,8 @@ public class Utils {
   /**
    * Assumes data in ZooKeeper is a JSON string, deserializes it and returns as a Map
    *
-   * @param zkClient the zookeeper client
-   * @param path the path to the znode being read
+   * @param zkClient        the zookeeper client
+   * @param path            the path to the znode being read
    * @param retryOnConnLoss whether to retry the operation automatically on connection loss, see {@link org.apache.solr.common.cloud.ZkCmdExecutor#retryOperation(ZkOperation)}
    * @return a Map if the node exists and contains valid JSON or an empty map if znode does not exist or has a null data
    */
@@ -630,11 +637,12 @@ public class Utils {
     }
   }
 
-  /**Applies one json over other. The 'input' is applied over the sink
+  /**
+   * Applies one json over other. The 'input' is applied over the sink
    * The values in input isapplied over the values in 'sink' . If a value is 'null'
    * that value is removed from sink
    *
-   * @param sink the original json object to start with. Ensure that this Map is mutable
+   * @param sink  the original json object to start with. Ensure that this Map is mutable
    * @param input the json with new values
    * @return whether there was any change made to sink or not.
    */
@@ -677,9 +685,9 @@ public class Utils {
     if (_offset < 0) {
       throw new IllegalArgumentException("nodeName does not contain expected '_' separator: " + nodeName);
     }
-    final String hostAndPort = nodeName.substring(0,_offset);
+    final String hostAndPort = nodeName.substring(0, _offset);
     try {
-      final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
+      final String path = URLDecoder.decode(nodeName.substring(1 + _offset), "UTF-8");
       return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
     } catch (UnsupportedEncodingException e) {
       throw new IllegalStateException("JVM Does not seem to support UTF-8", e);
@@ -713,4 +721,68 @@ public class Utils {
     return def;
   }
 
+  public interface InputStreamConsumer<T> {
+
+    T accept(InputStream is) throws IOException;
+
+  }
+
+  public static final InputStreamConsumer<?> JAVABINCONSUMER = is -> new JavaBinCodec().unmarshal(is);
+  public static final InputStreamConsumer<?> JSONCONSUMER = is -> Utils.fromJSON(is);
+
+  public static InputStreamConsumer<ByteBuffer> newBytesConsumer(int maxSize) {
+    return is -> {
+      try (BinaryRequestWriter.BAOS bos = new BinaryRequestWriter.BAOS()) {
+        long sz = 0;
+        int next = is.read();
+        while (next > -1) {
+          if (++sz > maxSize) throw new BufferOverflowException();
+          bos.write(next);
+          next = is.read();
+        }
+        bos.flush();
+        return ByteBuffer.wrap(bos.getbuf(), 0, bos.size());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    };
+
+  }
+
+
+  public static <T> T executeGET(HttpClient client, String url, InputStreamConsumer<T> consumer) throws SolrException {
+    T result = null;
+    HttpGet httpGet = new HttpGet(url);
+    HttpResponse rsp = null;
+    try {
+      rsp = client.execute(httpGet);
+    } catch (IOException e) {
+      log.error("Error in request to url : " + url, e);
+      throw new SolrException(SolrException.ErrorCode.UNKNOWN, "error sending request");
+    }
+    int statusCode = rsp.getStatusLine().getStatusCode();
+    if (statusCode != 200) {
+      try {
+        log.error("Failed a request to: {}, status: {}, body: {}", url, rsp.getStatusLine(), EntityUtils.toString(rsp.getEntity(), StandardCharsets.UTF_8));
+      } catch (IOException e) {
+        log.error("could not print error", e);
+      }
+      throw new SolrException(SolrException.ErrorCode.getErrorCode(statusCode), "Unknown error");
+    }
+    HttpEntity entity = rsp.getEntity();
+    try {
+      InputStream is = entity.getContent();
+      if (consumer != null) {
+
+        result = consumer.accept(is);
+      }
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.UNKNOWN, e);
+    } finally {
+      Utils.consumeFully(entity);
+    }
+    return result;
+  }
+
+
 }
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
index cb66ae9..4ce7a5e 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
@@ -206,8 +206,8 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
      *
      * @throws Exception if an error occurs on startup
      */
-    public void configure() throws Exception {
-      cluster = build();
+    public MiniSolrCloudCluster configure() throws Exception {
+      return cluster = build();
     }
 
     /**