mirror of https://github.com/apache/lucene.git
SOLR-7126: Secure loading of runtime external jars
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1665207 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
68d180346a
commit
31a3e55508
|
@ -156,6 +156,8 @@ New Features
|
|||
cluster is not ready, and can now take an optional timeout argument to wait
|
||||
for the cluster. (Alan Woodward, shalin, yonik, Mark Miller, Vitaliy Zhovtyuk)
|
||||
|
||||
* SOLR-7126: Secure loading of runtime external jars (Noble Paul)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.File;
|
|||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -109,12 +108,13 @@ public class CloudUtil {
|
|||
/**Read the list of public keys from ZK
|
||||
*/
|
||||
|
||||
public static Map<String, byte[]> getTrustedKeys(SolrZkClient zk){
|
||||
Map<String,byte[]> result = new HashMap<>();
|
||||
public static Map<String, byte[]> getTrustedKeys(SolrZkClient zk, String dir) {
|
||||
Map<String, byte[]> result = new HashMap<>();
|
||||
try {
|
||||
List<String> children = zk.getChildren("/keys", null, true);
|
||||
List<String> children = zk.getChildren("/keys/" + dir, null, true);
|
||||
for (String key : children) {
|
||||
result.put(key, zk.getData("/keys/"+key,null,null,true));
|
||||
if (key.endsWith(".der")) result.put(key, zk.getData("/keys/" + dir +
|
||||
"/" + key, null, null, true));
|
||||
}
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
log.warn("Error fetching key names");
|
||||
|
@ -128,4 +128,5 @@ public class CloudUtil {
|
|||
return result;
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ 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.handler.admin.CollectionsHandler;
|
||||
import org.apache.solr.util.CryptoKeys;
|
||||
import org.apache.solr.util.SimplePostTool;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -204,6 +205,10 @@ public class JarRepository {
|
|||
return null;
|
||||
}
|
||||
|
||||
public String checkSignature(String base64Sig, CryptoKeys keys) {
|
||||
return keys.verify(base64Sig, buffer);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static class JarContentRef {
|
||||
|
|
|
@ -57,6 +57,7 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
|
|||
for (PluginBag.RuntimeLib lib : libs) {
|
||||
try {
|
||||
lib.loadJar();
|
||||
lib.verify();
|
||||
} catch (Exception exception) {
|
||||
if (exception instanceof SolrException) throw (SolrException) exception;
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Atleast one runtimeLib could not be loaded", exception);
|
||||
|
@ -116,7 +117,7 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
|
|||
try {
|
||||
buf = lib.getFileContent(path);
|
||||
if (buf != null) {
|
||||
jarName.set(lib.name);
|
||||
jarName.set(lib.getName());
|
||||
break;
|
||||
}
|
||||
} catch (Exception exp) {
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.solr.core;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.text.MessageFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -29,10 +30,12 @@ import java.util.Set;
|
|||
|
||||
import org.apache.lucene.analysis.util.ResourceLoader;
|
||||
import org.apache.lucene.analysis.util.ResourceLoaderAware;
|
||||
import org.apache.solr.cloud.CloudUtil;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.handler.RequestHandlerBase;
|
||||
import org.apache.solr.handler.component.SearchComponent;
|
||||
import org.apache.solr.request.SolrRequestHandler;
|
||||
import org.apache.solr.util.CryptoKeys;
|
||||
import org.apache.solr.util.plugin.NamedListInitializedPlugin;
|
||||
import org.apache.solr.util.plugin.PluginInfoInitialized;
|
||||
import org.apache.solr.util.plugin.SolrCoreAware;
|
||||
|
@ -323,10 +326,10 @@ 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 {
|
||||
String name;
|
||||
String version;
|
||||
private String name, version, sig;
|
||||
private JarRepository.JarContentRef jarContent;
|
||||
private final JarRepository jarRepository;
|
||||
private final CoreContainer coreContainer;
|
||||
private boolean verified = false;
|
||||
|
||||
@Override
|
||||
public void init(PluginInfo info) {
|
||||
|
@ -336,10 +339,11 @@ public class PluginBag<T> implements AutoCloseable {
|
|||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "runtimeLib must have name and version");
|
||||
}
|
||||
version = String.valueOf(v);
|
||||
sig = info.attributes.get("sig");
|
||||
}
|
||||
|
||||
public RuntimeLib(SolrCore core) {
|
||||
jarRepository = core.getCoreDescriptor().getCoreContainer().getJarRepository();
|
||||
coreContainer = core.getCoreDescriptor().getCoreContainer();
|
||||
}
|
||||
|
||||
|
||||
|
@ -347,10 +351,23 @@ public class PluginBag<T> implements AutoCloseable {
|
|||
if (jarContent != null) return;
|
||||
synchronized (this) {
|
||||
if (jarContent != null) return;
|
||||
jarContent = jarRepository.getJarIncRef(name + "/" + version);
|
||||
jarContent = coreContainer.getJarRepository().getJarIncRef(name + "/" + version);
|
||||
}
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public String getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
public String getSig() {
|
||||
return sig;
|
||||
|
||||
}
|
||||
|
||||
public ByteBuffer getFileContent(String entryName) throws IOException {
|
||||
if (jarContent == null)
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "jar not available: " + name + "/" + version);
|
||||
|
@ -360,7 +377,7 @@ public class PluginBag<T> implements AutoCloseable {
|
|||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
if (jarContent != null) jarRepository.decrementJarRefCount(jarContent);
|
||||
if (jarContent != null) coreContainer.getJarRepository().decrementJarRefCount(jarContent);
|
||||
}
|
||||
|
||||
public static List<RuntimeLib> getLibObjects(SolrCore core, List<PluginInfo> libs) {
|
||||
|
@ -372,5 +389,38 @@ public class PluginBag<T> implements AutoCloseable {
|
|||
}
|
||||
return l;
|
||||
}
|
||||
|
||||
public void verify() throws Exception {
|
||||
if (verified) return;
|
||||
if (jarContent == null) {
|
||||
log.error("Calling verify before loading the jar");
|
||||
return;
|
||||
}
|
||||
|
||||
if (!coreContainer.isZooKeeperAware())
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Signing jar is possible only in cloud");
|
||||
Map<String, byte[]> keys = CloudUtil.getTrustedKeys(coreContainer.getZkController().getZkClient(), "exe");
|
||||
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);
|
||||
}
|
||||
} else if (sig == null) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, MessageFormat.format("runtimelib {0} should be signed with one of the keys in ZK /keys/exe ", name));
|
||||
}
|
||||
|
||||
try {
|
||||
String matchedKey = jarContent.jar.checkSignature(sig, new CryptoKeys(keys));
|
||||
if (matchedKey == null)
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No key matched signature for jar : " + name + " version: " + version);
|
||||
log.info("Jar {} signed with {} successfully verified", name, matchedKey);
|
||||
} catch (Exception e) {
|
||||
if (e instanceof SolrException) throw e;
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error verifying key ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,9 +37,8 @@ import org.slf4j.LoggerFactory;
|
|||
*/
|
||||
public final class CryptoKeys {
|
||||
private static final Logger log = LoggerFactory.getLogger(CryptoKeys.class);
|
||||
|
||||
private final Map<String, PublicKey> keys;
|
||||
|
||||
private Exception exception;
|
||||
|
||||
public CryptoKeys(Map<String, byte[]> trustedKeys) throws Exception {
|
||||
HashMap<String, PublicKey> m = new HashMap<>();
|
||||
|
@ -53,15 +52,16 @@ public final class CryptoKeys {
|
|||
/**
|
||||
* Try with all signatures and return the name of the signature that matched
|
||||
*/
|
||||
public String verify(String sig, byte[] data) {
|
||||
|
||||
public String verify(String sig, ByteBuffer data) {
|
||||
exception = null;
|
||||
for (Map.Entry<String, PublicKey> entry : keys.entrySet()) {
|
||||
boolean verified;
|
||||
try {
|
||||
verified = CryptoKeys.verify(entry.getValue(), Base64.base64ToByteArray(sig), ByteBuffer.wrap(data));
|
||||
verified = CryptoKeys.verify(entry.getValue(), Base64.base64ToByteArray(sig), data);
|
||||
log.info("verified {} ", verified);
|
||||
if (verified) return entry.getKey();
|
||||
} catch (Exception e) {
|
||||
exception = e;
|
||||
log.info("NOT verified ");
|
||||
}
|
||||
|
||||
|
@ -89,6 +89,7 @@ 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;
|
||||
try {
|
||||
signature = Signature.getInstance("SHA1withRSA");
|
||||
|
@ -99,6 +100,9 @@ public final class CryptoKeys {
|
|||
|
||||
} catch (NoSuchAlgorithmException e) {
|
||||
//will not happen
|
||||
} finally {
|
||||
//Signature.update resets the position. set it back to old
|
||||
data.position(oldPos);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -19,18 +19,56 @@ package org.apache.solr.cloud;
|
|||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.solr.client.solrj.SolrClient;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrClient;
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.core.ConfigOverlay;
|
||||
import org.apache.solr.core.MemClassLoader;
|
||||
import org.apache.solr.core.TestDynamicLoading;
|
||||
import org.apache.solr.core.TestSolrConfigHandler;
|
||||
import org.apache.solr.handler.TestBlobHandler;
|
||||
import org.apache.solr.util.CryptoKeys;
|
||||
import org.apache.solr.util.RESTfulServerProvider;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static org.apache.solr.handler.TestSolrConfigHandlerCloud.compareValues;
|
||||
|
||||
public class TestCryptoKeys extends AbstractFullDistribZkTestBase {
|
||||
private static final Logger logger = LoggerFactory.getLogger(TestCryptoKeys.class);
|
||||
private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
|
||||
|
||||
private void setupHarnesses() {
|
||||
for (final SolrClient client : clients) {
|
||||
RestTestHarness harness = new RestTestHarness(new RESTfulServerProvider() {
|
||||
@Override
|
||||
public String getBaseURL() {
|
||||
return ((HttpSolrClient) client).getBaseURL();
|
||||
}
|
||||
});
|
||||
restTestHarnesses.add(harness);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void distribTearDown() throws Exception {
|
||||
super.distribTearDown();
|
||||
for (RestTestHarness r : restTestHarnesses) {
|
||||
r.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
|
@ -46,30 +84,27 @@ public class TestCryptoKeys extends AbstractFullDistribZkTestBase {
|
|||
|
||||
@Test
|
||||
public void test() throws Exception {
|
||||
System.setProperty("enable.runtime.lib", "true");
|
||||
setupHarnesses();
|
||||
String pk1sig = "G8LEW7uJ1is81Aqqfl3Sld3qDtOxPuVFeTLJHFJWecgDvUkmJNFXmf7nkHOVlXnDWahp1vqZf0W02VHXg37lBw==";
|
||||
String pk2sig = "pCyBQycB/0YvLVZfKLDIIqG1tFwM/awqzkp2QNpO7R3ThTqmmrj11wEJFDRLkY79efuFuQPHt40EE7jrOKoj9jLNELsfEqvU3jw9sZKiDONY+rV9Bj9QPeW8Pgt+F9Y1";
|
||||
String wrongKeySig = "xTk2hTipfpb+J5s4x3YZGOXkmHWtnJz05Vvd8RTm/Q1fbQVszR7vMk6dQ1URxX08fcg4HvxOo8g9bG2TSMOGjg==";
|
||||
String result = null;
|
||||
CryptoKeys cryptoKeys = null;
|
||||
SolrZkClient zk = getCommonCloudSolrClient().getZkStateReader().getZkClient();
|
||||
cryptoKeys = new CryptoKeys(CloudUtil.getTrustedKeys(zk));
|
||||
byte[] samplefile = readFile("samplefile.bin");
|
||||
//there are no keys
|
||||
cryptoKeys = new CryptoKeys(CloudUtil.getTrustedKeys(zk, "exe"));
|
||||
ByteBuffer samplefile = ByteBuffer.wrap(readFile("cryptokeys/samplefile.bin"));
|
||||
//there are no keys yet created in ZK
|
||||
|
||||
result = cryptoKeys.verify( pk1sig,samplefile);
|
||||
assertNull(result);
|
||||
|
||||
zk.makePath("/keys", true);
|
||||
|
||||
createNode(zk, "pubk1.der");
|
||||
createNode(zk, "pubk2.der");
|
||||
|
||||
Map<String, byte[]> trustedKeys = CloudUtil.getTrustedKeys(zk);
|
||||
|
||||
zk.makePath("/keys/exe", true);
|
||||
zk.create("/keys/exe/pubk1.der", readFile("cryptokeys/pubk1.der"), CreateMode.PERSISTENT, true);
|
||||
zk.create("/keys/exe/pubk2.der", readFile("cryptokeys/pubk2.der"), CreateMode.PERSISTENT, true);
|
||||
Map<String, byte[]> trustedKeys = CloudUtil.getTrustedKeys(zk, "exe");
|
||||
|
||||
cryptoKeys = new CryptoKeys(trustedKeys);
|
||||
|
||||
|
||||
result = cryptoKeys.verify(pk2sig, samplefile);
|
||||
assertEquals("pubk2.der", result);
|
||||
|
||||
|
@ -83,30 +118,123 @@ public class TestCryptoKeys extends AbstractFullDistribZkTestBase {
|
|||
} catch (Exception e) {
|
||||
//pass
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
result = cryptoKeys.verify( "SGVsbG8gV29ybGQhCg==", samplefile);
|
||||
assertNull(result);
|
||||
} catch (Exception e) {
|
||||
//pass
|
||||
|
||||
}
|
||||
|
||||
|
||||
HttpSolrClient randomClient = (HttpSolrClient) clients.get(random().nextInt(clients.size()));
|
||||
String baseURL = randomClient.getBaseURL();
|
||||
baseURL = baseURL.substring(0, baseURL.lastIndexOf('/'));
|
||||
|
||||
TestBlobHandler.createSystemCollection(new HttpSolrClient(baseURL, randomClient.getHttpClient()));
|
||||
waitForRecoveriesToFinish(".system", true);
|
||||
|
||||
ByteBuffer jar = TestDynamicLoading.getFileContent("runtimecode/runtimelibs.jar.bin");
|
||||
String blobName = "signedjar";
|
||||
TestBlobHandler.postAndCheck(cloudClient, baseURL, blobName, jar, 1);
|
||||
|
||||
String payload = "{\n" +
|
||||
"'create-requesthandler' : { 'name' : '/runtime', 'class': 'org.apache.solr.core.RuntimeLibReqHandler' , 'runtimeLib':true }" +
|
||||
"}";
|
||||
RestTestHarness client = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));
|
||||
TestSolrConfigHandler.runConfigCommand(client, "/config?wt=json", payload);
|
||||
|
||||
TestSolrConfigHandler.testForResponseElement(client,
|
||||
null,
|
||||
"/config/overlay?wt=json",
|
||||
null,
|
||||
Arrays.asList("overlay", "requestHandler", "/runtime", "class"),
|
||||
"org.apache.solr.core.RuntimeLibReqHandler", 10);
|
||||
|
||||
|
||||
payload = "{\n" +
|
||||
"'add-runtimelib' : { 'name' : 'signedjar' ,'version':1}\n" +
|
||||
"}";
|
||||
client = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));
|
||||
TestSolrConfigHandler.runConfigCommand(client, "/config?wt=json", payload);
|
||||
TestSolrConfigHandler.testForResponseElement(client,
|
||||
null,
|
||||
"/config/overlay?wt=json",
|
||||
null,
|
||||
Arrays.asList("overlay", "runtimeLib", blobName, "version"),
|
||||
1l, 10);
|
||||
|
||||
Map map = TestSolrConfigHandler.getRespMap("/runtime?wt=json", client);
|
||||
String s = (String) ConfigOverlay.getObjectByPath(map, false, Arrays.asList("error", "msg"));
|
||||
assertNotNull(TestBlobHandler.getAsString(map), s);
|
||||
assertTrue(TestBlobHandler.getAsString(map), s.contains("should be signed with one of the keys in ZK /keys/exe"));
|
||||
|
||||
String wrongSig = "QKqHtd37QN02iMW9UEgvAO9g9qOOuG5vEBNkbUsN7noc2hhXKic/ABFIOYJA9PKw61mNX2EmNFXOcO3WClYdSw==";
|
||||
|
||||
payload = "{\n" +
|
||||
"'update-runtimelib' : { 'name' : 'signedjar' ,'version':1, 'sig': 'QKqHtd37QN02iMW9UEgvAO9g9qOOuG5vEBNkbUsN7noc2hhXKic/ABFIOYJA9PKw61mNX2EmNFXOcO3WClYdSw=='}\n" +
|
||||
"}";
|
||||
client = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));
|
||||
TestSolrConfigHandler.runConfigCommand(client, "/config?wt=json", payload);
|
||||
TestSolrConfigHandler.testForResponseElement(client,
|
||||
null,
|
||||
"/config/overlay?wt=json",
|
||||
null,
|
||||
Arrays.asList("overlay", "runtimeLib", blobName, "sig"),
|
||||
wrongSig, 10);
|
||||
|
||||
map = TestSolrConfigHandler.getRespMap("/runtime?wt=json", client);
|
||||
s = (String) ConfigOverlay.getObjectByPath(map, false, Arrays.asList("error", "msg"));
|
||||
assertNotNull(TestBlobHandler.getAsString(map), s);//No key matched signature for jar
|
||||
assertTrue(TestBlobHandler.getAsString(map), s.contains("No key matched signature for jar"));
|
||||
|
||||
String rightSig = "YkTQgOtvcM/H/5EQdABGl3wjjrPhonAGlouIx59vppBy2cZEofX3qX1yZu5sPNRmJisNXEuhHN2149dxeUmk2Q==";
|
||||
|
||||
payload = "{\n" +
|
||||
"'update-runtimelib' : { 'name' : 'signedjar' ,'version':1, 'sig': 'YkTQgOtvcM/H/5EQdABGl3wjjrPhonAGlouIx59vppBy2cZEofX3qX1yZu5sPNRmJisNXEuhHN2149dxeUmk2Q=='}\n" +
|
||||
"}";
|
||||
client = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));
|
||||
TestSolrConfigHandler.runConfigCommand(client, "/config?wt=json", payload);
|
||||
TestSolrConfigHandler.testForResponseElement(client,
|
||||
null,
|
||||
"/config/overlay?wt=json",
|
||||
null,
|
||||
Arrays.asList("overlay", "runtimeLib", blobName, "sig"),
|
||||
rightSig, 10);
|
||||
|
||||
map = TestSolrConfigHandler.testForResponseElement(client,
|
||||
null,
|
||||
"/runtime?wt=json",
|
||||
null,
|
||||
Arrays.asList("class"),
|
||||
"org.apache.solr.core.RuntimeLibReqHandler", 10);
|
||||
compareValues(map, MemClassLoader.class.getName(), asList("loader"));
|
||||
|
||||
rightSig = "VJPMTxDf8Km3IBj2B5HWkIOqeM/o+HHNobOYCNA3WjrEVfOMZbMMqS1Lo7uLUUp//RZwOGkOhrUhuPNY1z2CGEIKX2/m8VGH64L14d52oSvFiwhoTDDuuyjW1TFGu35D";
|
||||
payload = "{\n" +
|
||||
"'update-runtimelib' : { 'name' : 'signedjar' ,'version':1, 'sig': 'VJPMTxDf8Km3IBj2B5HWkIOqeM/o+HHNobOYCNA3WjrEVfOMZbMMqS1Lo7uLUUp//RZwOGkOhrUhuPNY1z2CGEIKX2/m8VGH64L14d52oSvFiwhoTDDuuyjW1TFGu35D'}\n" +
|
||||
"}";
|
||||
client = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));
|
||||
TestSolrConfigHandler.runConfigCommand(client, "/config?wt=json", payload);
|
||||
TestSolrConfigHandler.testForResponseElement(client,
|
||||
null,
|
||||
"/config/overlay?wt=json",
|
||||
null,
|
||||
Arrays.asList("overlay", "runtimeLib", blobName, "sig"),
|
||||
rightSig, 10);
|
||||
|
||||
map = TestSolrConfigHandler.testForResponseElement(client,
|
||||
null,
|
||||
"/runtime?wt=json",
|
||||
null,
|
||||
Arrays.asList("class"),
|
||||
"org.apache.solr.core.RuntimeLibReqHandler", 10);
|
||||
compareValues(map, MemClassLoader.class.getName(), asList("loader"));
|
||||
}
|
||||
|
||||
|
||||
|
||||
private void createNode(SolrZkClient zk, String fname) throws IOException, KeeperException, InterruptedException {
|
||||
byte[] buf = readFile(fname);
|
||||
zk.create("/keys/" + fname, buf, CreateMode.PERSISTENT, true);
|
||||
|
||||
}
|
||||
|
||||
private byte[] readFile(String fname) throws IOException {
|
||||
byte[] buf = null;
|
||||
try (FileInputStream fis = new FileInputStream(getFile("cryptokeys/" + fname))) {
|
||||
try (FileInputStream fis = new FileInputStream(getFile(fname))) {
|
||||
buf = new byte[fis.available()];
|
||||
fis.read(buf);
|
||||
}
|
||||
|
|
|
@ -267,7 +267,7 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase {
|
|||
10);
|
||||
}
|
||||
|
||||
private ByteBuffer getFileContent(String f) throws IOException {
|
||||
public static ByteBuffer getFileContent(String f) throws IOException {
|
||||
ByteBuffer jar;
|
||||
try (FileInputStream fis = new FileInputStream(getFile(f))) {
|
||||
byte[] buf = new byte[fis.available()];
|
||||
|
|
Loading…
Reference in New Issue