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/06/17 07:00:59 UTC

[lucene-solr] 01/02: SOLR-13534: Dynamic loading of jars from a remote url (#712)

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 562c462ee3c24a7fcbf00ae22554c2b2d3a057ab
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Mon Jun 17 16:49:31 2019 +1000

    SOLR-13534: Dynamic loading of jars from a remote url (#712)
    
    * SOLR-13534 : Dynamic loading to support loading jars from a URL
---
 .../java/org/apache/solr/core/BlobRepository.java  |  58 ++++++++--
 .../java/org/apache/solr/core/ConfigOverlay.java   |   6 +-
 .../java/org/apache/solr/core/MemClassLoader.java  |  21 +++-
 .../src/java/org/apache/solr/core/PluginBag.java   |  56 +++++++--
 .../src/java/org/apache/solr/core/SolrCore.java    |   3 +
 .../org/apache/solr/handler/SolrConfigHandler.java |  14 +++
 .../solr/core/BlobRepositoryMockingTest.java       |  44 ++++++-
 .../apache/solr/core/TestDynamicLoadingUrl.java    | 127 +++++++++++++++++++++
 .../apache/solr/core/TestSolrConfigHandler.java    |   3 +-
 .../adding-custom-plugins-in-solrcloud-mode.adoc   |  97 +++++++++++++++-
 10 files changed, 403 insertions(+), 26 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 0cf6dfb..b9a1e73 100644
--- a/solr/core/src/java/org/apache/solr/core/BlobRepository.java
+++ b/solr/core/src/java/org/apache/solr/core/BlobRepository.java
@@ -16,16 +16,17 @@
  */
 package org.apache.solr.core;
 
-import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
-import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
-
 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.util.ArrayList;
 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;
@@ -44,16 +45,22 @@ 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.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 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.SERVER_ERROR;
+import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
+import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
+
 /**
  * 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.
  */
 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;
   static final Pattern BLOB_KEY_PATTERN_CHECKER = Pattern.compile(".*/\\d+");
@@ -108,6 +115,14 @@ public class BlobRepository {
     return getBlobIncRef(key.concat(decoder.getName()), () -> addBlob(key,decoder));
   }
 
+  BlobContentRef getBlobIncRef(String key, Decoder decoder, String url, String sha512) {
+    StringBuffer keyBuilder = new StringBuffer(key);
+    if (decoder != null) keyBuilder .append( decoder.getName());
+    keyBuilder.append("/").append( sha512);
+
+    return getBlobIncRef(keyBuilder.toString(), () -> new BlobContent<>(key, fetchBlobAndVerify(key, url, sha512), decoder));
+  }
+
   // do the actual work returning the appropriate type...
   private <T> BlobContentRef<T> getBlobIncRef(String key, Callable<BlobContent<T>> blobCreator) {
     BlobContent<T> aBlob;
@@ -149,14 +164,44 @@ public class BlobRepository {
     blobs.put(keyPlusName, aBlob);
     return aBlob;
   }
-  
+  static String INVALID_JAR_MSG = "Invalid jar from {0} , expected sha512 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));
+
+    }
+    return byteBuffer;
+  }
+
+  public static String sha512Digest(ByteBuffer byteBuffer) {
+    MessageDigest digest = null;
+    try {
+      digest = MessageDigest.getInstance("SHA-512");
+    } catch (NoSuchAlgorithmException e) {
+      //unlikely
+      throw new SolrException(SERVER_ERROR, e);
+    }
+    digest.update(byteBuffer);
+    return String.format(
+        Locale.ROOT,
+        "%0128x",
+        new BigInteger(1, digest.digest()));
+  }
+
+
   /**
    *  Package local for unit tests only please do not use elsewhere
    */
   ByteBuffer fetchBlob(String key) {
     Replica replica = getSystemCollReplica();
     String url = replica.getStr(BASE_URL_PROP) + "/" + CollectionAdminParams.SYSTEM_COLL + "/blob/" + key + "?wt=filestream";
+    return fetchFromUrl(key, url);
+  }
 
+  ByteBuffer fetchFromUrl(String key, String url) {
     HttpClient httpClient = coreContainer.getUpdateShardHandler().getDefaultHttpClient();
     HttpGet httpGet = new HttpGet(url);
     ByteBuffer b;
@@ -171,7 +216,7 @@ public class BlobRepository {
       }
 
       try (InputStream is = entity.getContent()) {
-        b = SimplePostTool.inputStreamToByteArray(is);
+        b = SimplePostTool.inputStreamToByteArray(is, MAX_JAR_SIZE);
       }
     } catch (Exception e) {
       if (e instanceof SolrException) {
@@ -240,7 +285,7 @@ public class BlobRepository {
 
     public BlobContent(String key, ByteBuffer buffer, Decoder<T> decoder) {
       this.key = key;
-      this.content = decoder.decode(new ByteBufferInputStream(buffer));
+      this.content = decoder == null ? (T) buffer : decoder.decode(new ByteBufferInputStream(buffer));
     }
 
     @SuppressWarnings("unchecked")
@@ -287,5 +332,4 @@ public class BlobRepository {
       this.blob = blob;
     }
   }
-
 }
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java b/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
index 5aa6f40..687505e 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
@@ -238,9 +238,9 @@ public class ConfigOverlay implements MapSerializable {
 
   public ConfigOverlay addNamedPlugin(Map<String, Object> info, String typ) {
     Map dataCopy = Utils.getDeepCopy(data, 4);
-    Map reqHandler = (Map) dataCopy.get(typ);
-    if (reqHandler == null) dataCopy.put(typ, reqHandler = new LinkedHashMap());
-    reqHandler.put(info.get(CoreAdminParams.NAME), info);
+    Map existing = (Map) dataCopy.get(typ);
+    if (existing == null) dataCopy.put(typ, existing = new LinkedHashMap());
+    existing.put(info.get(CoreAdminParams.NAME), info);
     return new ConfigOverlay(dataCopy, this.znodeVersion);
   }
 
diff --git a/solr/core/src/java/org/apache/solr/core/MemClassLoader.java b/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
index e4f561f..d1a3a7c 100644
--- a/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
@@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.util.StrUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,6 +45,7 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
   private final SolrResourceLoader parentLoader;
   private List<PluginBag.RuntimeLib> libs = new ArrayList<>();
   private Map<String, Class> classCache = new HashMap<>();
+  private List<String> errors = new ArrayList<>();
 
 
   public MemClassLoader(List<PluginBag.RuntimeLib> libs, SolrResourceLoader resourceLoader) {
@@ -51,6 +53,22 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
     this.libs = libs;
   }
 
+  synchronized void loadRemoteJars() {
+    if (allJarsLoaded) return;
+    int count = 0;
+    for (PluginBag.RuntimeLib lib : libs) {
+      if (lib.getUrl() != null) {
+        try {
+          lib.loadJar();
+          lib.verify();
+        } catch (Exception e) {
+          log.error("Error loading runtime library", e);
+        }
+        count++;
+      }
+    }
+    if (count == libs.size()) allJarsLoaded = true;
+  }
 
   public synchronized void loadJars() {
     if (allJarsLoaded) return;
@@ -60,6 +78,7 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
         lib.loadJar();
         lib.verify();
       } catch (Exception exception) {
+        errors.add(exception.getMessage());
         if (exception instanceof SolrException) throw (SolrException) exception;
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Atleast one runtimeLib could not be loaded", exception);
       }
@@ -87,7 +106,7 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
     try {
       buf = getByteBuffer(name, jarName);
     } catch (Exception e) {
-      throw new ClassNotFoundException("class could not be loaded " + name, e);
+      throw new ClassNotFoundException("class could not be loaded " + name + (errors.isEmpty()? "": "Some dynamic libraries could not be loaded: "+ StrUtils.join(errors, '|')), e);
     }
     if (buf == null) throw new ClassNotFoundException("Class not found :" + name);
     ProtectionDomain defaultDomain = null;
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index abd1a44..977a7e6 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -127,7 +127,7 @@ public class PluginBag<T> implements AutoCloseable {
   public PluginHolder<T> createPlugin(PluginInfo info) {
     if ("true".equals(String.valueOf(info.attributes.get("runtimeLib")))) {
       log.debug(" {} : '{}'  created with runtimeLib=true ", meta.getCleanTag(), info.name);
-      LazyPluginHolder<T> holder = new LazyPluginHolder<>(meta, info, core, "true".equals(System.getProperty("enable.runtime.lib")) ?
+      LazyPluginHolder<T> holder = new LazyPluginHolder<>(meta, info, core, RuntimeLib.isEnabled() ?
           core.getMemClassLoader() :
           core.getResourceLoader(), true);
 
@@ -393,7 +393,7 @@ public class PluginBag<T> implements AutoCloseable {
       this.core = core;
       this.resourceLoader = loader;
       if (loader instanceof MemClassLoader) {
-        if (!"true".equals(System.getProperty("enable.runtime.lib"))) {
+        if (!RuntimeLib.isEnabled()) {
           String s = "runtime library loading is not enabled, start Solr with -Denable.runtime.lib=true";
           log.warn(s);
           solrException = new SolrException(SolrException.ErrorCode.SERVER_ERROR, s);
@@ -464,7 +464,7 @@ public class PluginBag<T> implements AutoCloseable {
    * This represents a Runtime Jar. A jar requires two details , name and version
    */
   public static class RuntimeLib implements PluginInfoInitialized, AutoCloseable {
-    private String name, version, sig;
+    private String name, version, sig, sha512, url;
     private BlobRepository.BlobContentRef<ByteBuffer> jarContent;
     private final CoreContainer coreContainer;
     private boolean verified = false;
@@ -472,27 +472,56 @@ public class PluginBag<T> implements AutoCloseable {
     @Override
     public void init(PluginInfo info) {
       name = info.attributes.get(NAME);
-      Object v = info.attributes.get("version");
-      if (name == null || v == null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "runtimeLib must have name and version");
-      }
-      version = String.valueOf(v);
+      url = info.attributes.get("url");
       sig = info.attributes.get("sig");
+      if(url == null) {
+        Object v = info.attributes.get("version");
+        if (name == null || v == null) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "runtimeLib must have name and version");
+        }
+        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");
+        }
+        ByteBuffer buf = null;
+        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)  );
+        }
+        log.info("dynamic library verified {}, sha512: {}", url, sha512);
+
+      }
+
     }
 
     public RuntimeLib(SolrCore core) {
       coreContainer = core.getCoreContainer();
     }
 
+    public String getUrl(){
+      return url;
+    }
 
     void loadJar() {
       if (jarContent != null) return;
       synchronized (this) {
         if (jarContent != null) return;
-        jarContent = coreContainer.getBlobRepository().getBlobIncRef(name + "/" + version);
+
+        jarContent = url == null?
+            coreContainer.getBlobRepository().getBlobIncRef(name + "/" + version):
+            coreContainer.getBlobRepository().getBlobIncRef(name, null,url,sha512);
+
       }
     }
 
+    public static boolean isEnabled() {
+      return "true".equals(System.getProperty("enable.runtime.lib"));
+    }
+
     public String getName() {
       return name;
     }
@@ -508,7 +537,7 @@ public class PluginBag<T> implements AutoCloseable {
 
     public ByteBuffer getFileContent(String entryName) throws IOException {
       if (jarContent == null)
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "jar not available: " + name + "/" + version);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "jar not available: " + name  );
       return getFileContent(jarContent.blob, entryName);
 
     }
@@ -547,7 +576,11 @@ public class PluginBag<T> implements AutoCloseable {
       List<RuntimeLib> l = new ArrayList<>(libs.size());
       for (PluginInfo lib : libs) {
         RuntimeLib rtl = new RuntimeLib(core);
-        rtl.init(lib);
+        try {
+          rtl.init(lib);
+        } catch (Exception e) {
+          log.error("error loading runtime library", e);
+        }
         l.add(rtl);
       }
       return l;
@@ -566,7 +599,6 @@ public class PluginBag<T> implements AutoCloseable {
       if (keys.isEmpty()) {
         if (sig == null) {
           verified = true;
-          log.info("A run time lib {} is loaded  without verification ", name);
           return;
         } else {
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No public keys are available in ZK to verify signature for runtime lib  " + name);
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index e77b70e..317ea21 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1000,6 +1000,9 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       // Initialize the RestManager
       restManager = initRestManager();
 
+      // at this point we can load jars loaded from remote urls.
+      memClassLoader.loadRemoteJars();
+
       // Finally tell anyone who wants to know
       resourceLoader.inform(resourceLoader);
       resourceLoader.inform(this); // last call before the latch is released.
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
index 7269358..11c6404 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -62,6 +62,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.ConfigOverlay;
+import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.RequestParams;
 import org.apache.solr.core.SolrConfig;
@@ -525,6 +526,19 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
       op.getMap(PluginInfo.INVARIANTS, null);
       op.getMap(PluginInfo.APPENDS, null);
       if (op.hasError()) return overlay;
+      if(info.clazz == PluginBag.RuntimeLib.class) {
+        if(!PluginBag.RuntimeLib.isEnabled()){
+          op.addError("Solr not started with -Denable.runtime.lib=true");
+          return overlay;
+        }
+        try {
+          new PluginBag.RuntimeLib(req.getCore()).init(new PluginInfo(info.tag, op.getDataMap()));
+        } catch (Exception e) {
+          op.addError(e.getMessage());
+          log.error("can't load this plugin ", e);
+          return overlay;
+        }
+      }
       if (!verifyClass(op, clz, info.clazz)) return overlay;
       if (pluginExists(info, overlay, name)) {
         if (isCeate) {
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 7b4dc06..4a0f1ba 100644
--- a/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
+++ b/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
@@ -22,6 +22,7 @@ import java.io.InputStream;
 import java.io.StringWriter;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
+import java.util.Objects;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.io.IOUtils;
@@ -31,11 +32,17 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.mockito.Mockito.*;
 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;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class BlobRepositoryMockingTest {
 
@@ -55,6 +62,8 @@ public class BlobRepositoryMockingTest {
   ByteBuffer blobData = ByteBuffer.wrap(BLOBSTR.getBytes(UTF8));
   boolean blobFetched = false;
   String blobKey = "";
+  String url = null;
+  ByteBuffer filecontent = null;
   
   @BeforeClass
   public static void beforeClass() {
@@ -75,6 +84,14 @@ public class BlobRepositoryMockingTest {
       }
 
       @Override
+      ByteBuffer fetchFromUrl(String key, String url) {
+        if(!Objects.equals(url, BlobRepositoryMockingTest.this.url)) return null;
+        blobKey = key;
+        blobFetched = true;
+        return filecontent;
+      }
+
+      @Override
       ConcurrentHashMap<String, BlobContent> createMap() {
         return mapMock;
       }
@@ -109,6 +126,31 @@ public class BlobRepositoryMockingTest {
 
   @SuppressWarnings("unchecked")
   @Test
+  public void testGetBlobIncrRefByUrl() throws Exception{
+    when(mockContainer.isZooKeeperAware()).thenReturn(true);
+    filecontent = TestDynamicLoading.getFileContent("runtimecode/runtimelibs_v2.jar.bin");
+    url = "http://localhost:8080/myjar/location.jar";
+    BlobRepository.BlobContentRef ref = repository.getBlobIncRef( "filefoo",null,url,
+        "bc5ce45ad281b6a08fb7e529b1eb475040076834816570902acb6ebdd809410e31006efdeaa7f78a6c35574f3504963f5f7e4d92247d0eb4db3fc9abdda5d417");
+    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;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
   public void testCachedAlready() {
     when(mockContainer.isZooKeeperAware()).thenReturn(true);
     when(mapMock.get("foo!")).thenReturn(new BlobRepository.BlobContent<BlobRepository>("foo!", blobData));
diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java b/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java
new file mode 100644
index 0000000..19d2d5f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/TestDynamicLoadingUrl.java
@@ -0,0 +1,127 @@
+/*
+ * 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.core;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.RestTestHarness;
+import org.junit.BeforeClass;
+
+import static java.util.Arrays.asList;
+import static org.apache.solr.handler.TestSolrConfigHandlerCloud.compareValues;
+
+public class TestDynamicLoadingUrl extends AbstractFullDistribZkTestBase {
+
+  @BeforeClass
+  public static void enableRuntimeLib() throws Exception {
+    System.setProperty("enable.runtime.lib", "true");
+  }
+
+  public static class JarHandler extends RequestHandlerBase {
+    @Override
+    public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+      rsp.add(ReplicationHandler.FILE_STREAM, (SolrCore.RawWriter) os -> {
+
+        ByteBuffer buf = TestDynamicLoading.getFileContent("runtimecode/runtimelibs.jar.bin");
+        if (buf == null) {
+          //should never happen unless a user wrote this document directly
+          throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "Invalid document . No field called blob");
+        } else {
+          os.write(buf.array(), 0, buf.limit());
+        }
+      });
+
+    }
+
+    @Override
+    public String getDescription() {
+      return "serves jar files";
+    }
+  }
+
+
+  public void testDynamicLoadingUrl() throws Exception {
+    setupRestTestHarnesses();
+    String payload = "{\n" +
+        "'create-requesthandler' : { 'name' : '/jarhandler', 'class': " + JarHandler.class.getName() +
+        ", registerPath: '/solr,/v2' }\n" +
+        "}";
+
+    RestTestHarness client = randomRestTestHarness();
+    TestSolrConfigHandler.runConfigCommand(client, "/config", payload);
+    TestSolrConfigHandler.testForResponseElement(client,
+        null,
+        "/config/overlay",
+        null,
+        Arrays.asList("overlay", "requestHandler", "/jarhandler", "class"),
+        JarHandler.class.getName(), 10);
+      payload = "{\n" +
+          "'add-runtimelib' : { 'name' : 'urljar', url : '" + client.getBaseURL() + "/jarhandler?wt=filestream'" +
+          "  'sha512':'e01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}" +
+          "}";
+      client = randomRestTestHarness();
+    TestSolrConfigHandler.runConfigCommandExpectFailure(client, "/config", payload, "Invalid jar");
+
+
+//    String url = client
+    payload = "{\n" +
+        "'add-runtimelib' : { 'name' : 'urljar', url : '" + client.getBaseURL() + "/jarhandler?wt=filestream'" +
+        "  'sha512':'d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420'}" +
+        "}";
+    client = randomRestTestHarness();
+    TestSolrConfigHandler.runConfigCommand(client, "/config", payload);
+    TestSolrConfigHandler.testForResponseElement(client,
+        null,
+        "/config/overlay",
+        null,
+        Arrays.asList("overlay", "runtimeLib", "urljar", "sha512"),
+        "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420", 10);
+
+    payload = "{\n" +
+        "'create-requesthandler' : { 'name' : '/runtime', 'class': 'org.apache.solr.core.RuntimeLibReqHandler', 'runtimeLib' : true}" +
+        "}";
+    client = randomRestTestHarness();
+    TestSolrConfigHandler.runConfigCommand(client, "/config", payload);
+
+    TestSolrConfigHandler.testForResponseElement(client,
+        null,
+        "/config/overlay",
+        null,
+        Arrays.asList("overlay", "requestHandler", "/runtime", "class"),
+        "org.apache.solr.core.RuntimeLibReqHandler", 10);
+
+    Map result = TestSolrConfigHandler.testForResponseElement(client,
+        null,
+        "/runtime",
+        null,
+        Arrays.asList("class"),
+        "org.apache.solr.core.RuntimeLibReqHandler", 10);
+    compareValues(result, MemClassLoader.class.getName(), asList("loader"));
+
+
+  }
+}
+
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
index 6e95fb5..17494e0 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
@@ -171,13 +171,14 @@ public class TestSolrConfigHandler extends RestTestBase {
     reqhandlertests(restTestHarness, null, null);
   }
 
-  public static void runConfigCommand(RestTestHarness harness, String uri, String payload) throws IOException {
+  public static Map runConfigCommand(RestTestHarness harness, String uri, String payload) throws IOException {
     String json = SolrTestCaseJ4.json(payload);
     log.info("going to send config command. path {} , payload: {}", uri, payload);
     String response = harness.post(uri, json);
     Map map = (Map) Utils.fromJSONString(response);
     assertNull(response, map.get("errorMessages"));
     assertNull(response, map.get("errors")); // Will this ever be returned?
+    return map;
   }
 
   public static void runConfigCommandExpectFailure(RestTestHarness harness, String uri, String payload, String expectedErrorMessage) throws Exception {
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 234ce9b..2f5ed58 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
@@ -30,7 +30,9 @@ Before enabling this feature, users should carefully consider the issues discuss
 
 == Uploading Jar Files
 
-The first step is to use the <<blob-store-api.adoc#blob-store-api,Blob Store API>> to upload your jar files. This will to put your jars in the `.system` collection and distribute them across your SolrCloud nodes. These jars are added to a separate classloader and only accessible to components that are configured with the property `runtimeLib=true`. These components are loaded lazily because the `.system` collection may not be loaded when a particular core is loaded.
+Use your own service to host the jars or you can use Solr itself to host the jars.
+
+Use the <<blob-store-api.adoc#blob-store-api,Blob Store API>> to upload your jar files to Solr. This will to put your jars in the `.system` collection and distribute them across your SolrCloud nodes. These jars are added to a separate classloader and only accessible to components that are configured with the property `runtimeLib=true`. These components are loaded lazily because the `.system` collection may not be loaded when a particular core is loaded.
 
 == Config API Commands to use Jars as Runtime Libraries
 
@@ -116,6 +118,99 @@ curl http://localhost:8983/api/collections/techproducts/config -H 'Content-type:
 ====
 --
 
+== Example: Using external service to host your jars
+
+Hosting your jars externally is more convenient if you have a reliable server to host your jars. There is no need to create and manage a `.system` collection.
+
+Step 1: Download a jar from github to the current directory
+
+[source,bash]
+----
+ 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
+
+[source,bash]
+----
+ openssl dgst -sha512 runtimelibs.jar
+----
+
+Step 3 :  Start solr with runtime lib enabled
+
+[source,bash]
+----
+ bin/solr start -e cloud -a "-Denable.runtime.lib=true" -noprompt
+----
+
+Step 4: Run a local server. Skip this step if you have another place to host your jars. Ensure that the url is set appropriately
+
+[source,bash]
+----
+ python -m SimpleHTTPServer 8000 &
+----
+
+Step 5: Add the jar to your collection `gettingstarted`
+
+[source,bash]
+----
+ curl http://localhost:8983/solr/gettingstarted/config -H 'Content-type:application/json' -d '{
+    "add-runtimelib": { "name" : "testjar",
+    "url":"http://localhost:8000/runtimelibs.jar" ,
+    "sha512" : "d01b51de67ae1680a84a813983b1de3b592fc32f1a22b662fc9057da5953abd1b72476388ba342cad21671cd0b805503c78ab9075ff2f3951fdf75fa16981420"}
+    }'
+----
+
+Step  6 : Create a new request handler '/test' for the collection 'gettingstarted' from the jar we just added
+
+[source,bash]
+----
+curl http://localhost:8983/solr/gettingstarted/config -H 'Content-type:application/json' -d '{
+    "create-requesthandler": { "name" : "/test",
+    'class': 'org.apache.solr.core.RuntimeLibReqHandler', 'runtimeLib' : true}
+    }'
+----
+
+Step 7:  Test your request handler
+
+[source,bash]
+----
+curl  http://localhost:8983/solr/gettingstarted/test
+----
+
+output:
+[source,json]
+----
+{
+  "responseHeader":{
+    "status":0,
+    "QTime":0},
+  "params":{},
+  "context":{
+    "webapp":"/solr",
+    "path":"/test",
+    "httpMethod":"GET"},
+  "class":"org.apache.solr.core.RuntimeLibReqHandler",
+  "loader":"org.apache.solr.core.MemClassLoader"}
+----
+
+=== Updating remote jars
+
+Example:
+
+* Host the new jar to a new url. eg:  http://localhost:8000/runtimelibs_v2.jar
+* get the `sha512` hash of the new jar
+* run the update-runtime lib command
+
+[source,bash]
+----
+ curl http://localhost:8983/solr/gettingstarted/config -H 'Content-type:application/json' -d '{
+    "update-runtimelib": { "name" : "testjar",
+    "url":"http://localhost:8000/runtimelibs_v2.jar" ,
+    "sha512" : "<replace-the-new-sha512-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
+
 == Securing Runtime Libraries
 
 A drawback of this feature is that it could be used to load malicious executable code into the system. However, it is possible to restrict the system to load only trusted jars using http://en.wikipedia.org/wiki/Public_key_infrastructure[PKI] to verify that the executables loaded into the system are trustworthy.