SOLR-8326: If BasicAuth enabled, inter node requests fail after node restart

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1716675 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Noble Paul 2015-11-26 14:23:27 +00:00
parent 4913d99542
commit 19b101566d
5 changed files with 92 additions and 55 deletions

View File

@ -455,6 +455,8 @@ Bug Fixes
* SOLR-8341: Fix JSON Facet API excludeTags when specified in the
form of domain:{excludeTags:mytag} (yonik)
* SOLR-8326: If BasicAuth enabled, inter node requests fail after node restart (noble, Anshum Gupta)
Optimizations
----------------------

View File

@ -24,8 +24,6 @@ import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletRequestWrapper;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.security.InvalidKeyException;
import java.security.Principal;
import java.security.PublicKey;
import java.util.List;
@ -58,13 +56,15 @@ import org.apache.solr.util.CryptoKeys;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.nio.charset.StandardCharsets.UTF_8;
public class PKIAuthenticationPlugin extends AuthenticationPlugin implements HttpClientInterceptorPlugin {
static final Logger log = LoggerFactory.getLogger(PKIAuthenticationPlugin.class);
static private final Logger log = LoggerFactory.getLogger(PKIAuthenticationPlugin.class);
private final Map<String, PublicKey> keyCache = new ConcurrentHashMap<>();
private CryptoKeys.RSAKeyPair keyPair = new CryptoKeys.RSAKeyPair();
private final CryptoKeys.RSAKeyPair keyPair = new CryptoKeys.RSAKeyPair();
private final CoreContainer cores;
private int maxValidity = 5000;
private final int MAX_VALIDITY = 5000;
private final String myNodeName;
private boolean interceptorRegistered = false;
@ -99,6 +99,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
long receivedTime = System.currentTimeMillis();
String header = ((HttpServletRequest) request).getHeader(HEADER);
if (header == null) {
//this must not happen
log.error("No SolrAuth header present");
filterChain.doFilter(request, response);
return;
@ -106,40 +107,29 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
List<String> authInfo = StrUtils.splitWS(header, false);
if (authInfo.size() < 2) {
log.error("Invalid SolrAuth Header");
log.error("Invalid SolrAuth Header {}", header);
filterChain.doFilter(request, response);
return;
}
String nodeName = authInfo.get(0);
String cipher = authInfo.get(1);
byte[] decipher = decipherData(nodeName, cipher);
PKIHeaderData decipher = decipherHeader(nodeName, cipher);
if (decipher == null) {
log.error("Could not decipher a header {} . No principal set", header);
filterChain.doFilter(request, response);
return;
}
String s = new String(decipher, StandardCharsets.UTF_8).trim();
List<String> pcs = StrUtils.splitWS(s, false);
if (pcs.size() < 2) {
return;
}
final String userName = pcs.get(0);
String timeStr = pcs.get(1);
try {
long timeMillis = Long.parseLong(timeStr);
if ((receivedTime - timeMillis) > maxValidity) {
if ((receivedTime - decipher.timestamp) > MAX_VALIDITY) {
log.error("Invalid key ");
filterChain.doFilter(request, response);
return;
}
} catch (NumberFormatException e) {
log.error("Invalid time " + timeStr, e);
return;
}
final Principal principal = "$".equals(userName) ?
final Principal principal = "$".equals(decipher.userName) ?
SU :
new BasicUserPrincipal(userName);
new BasicUserPrincipal(decipher.userName);
filterChain.doFilter(getWrapper((HttpServletRequest) request, principal), response);
}
@ -153,48 +143,69 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
};
}
private byte[] decipherData(String nodeName, String cipherBase64) {
boolean freshKey = false;
public static class PKIHeaderData {
String userName;
long timestamp;
}
private PKIHeaderData decipherHeader(String nodeName, String cipherBase64) {
PublicKey key = keyCache.get(nodeName);
if (key == null) {
log.debug("No key available for node : {} fetching now ", nodeName);
key = getRemotePublicKey(nodeName);
freshKey = true;
log.debug("public key obtained {} ", key);
}
try {
return CryptoKeys.decryptRSA(Base64.base64ToByteArray(cipherBase64), key);
} catch (InvalidKeyException e) {
if (!freshKey) {
key = getRemotePublicKey(nodeName);
if (key == null) {
return null;
}
try {
return CryptoKeys.decryptRSA(Base64.base64ToByteArray(cipherBase64), key);
} catch (Exception e1) {
log.error("Error decrypting");
return null;
}
}
PKIHeaderData header = parseCipher(cipherBase64, key);
if (header == null) {
log.warn("Failed to decrypt header, trying after refreshing the key ");
key = getRemotePublicKey(nodeName);
return parseCipher(cipherBase64, key);
} else {
return header;
}
}
private static PKIHeaderData parseCipher(String cipher, PublicKey key) {
byte[] bytes;
try {
bytes = CryptoKeys.decryptRSA(Base64.base64ToByteArray(cipher), key);
} catch (Exception e) {
log.error("Error decrypting");
log.error("Decryption failed , key must be wrong", e);
return null;
}
String s = new String(bytes, UTF_8).trim();
String[] ss = s.split(" ");
if (ss.length < 2) {
log.warn("Invalid cipher {} deciphered data {}", cipher, s);
return null;
}
PKIHeaderData headerData = new PKIHeaderData();
try {
headerData.timestamp = Long.parseLong(ss[1]);
headerData.userName = ss[0];
log.debug("Successfully decrypted header {} {}", headerData.userName, headerData.timestamp);
return headerData;
} catch (NumberFormatException e) {
log.warn("Invalid cipher {}", cipher);
return null;
}
return null;
}
PublicKey getRemotePublicKey(String nodename) {
String url = cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodename);
try {
HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient().execute(new HttpGet(url + PATH + "?wt=json&omitHeader=true"));
String uri = url + PATH + "?wt=json&omitHeader=true";
log.debug("Fetching fresh public key from : {}",uri);
HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient().execute(new HttpGet(uri));
byte[] bytes = EntityUtils.toByteArray(rsp.getEntity());
Map m = (Map) Utils.fromJSON(bytes);
String key = (String) m.get("key");
if (key == null) {
log.error("No key available from " + url + PATH);
return null;
} else {
log.info("New Key obtained from node: {} / {}", nodename, key);
}
PublicKey pubKey = CryptoKeys.deserializeX509PublicKey(key);
keyCache.put(nodename, pubKey);
@ -228,9 +239,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
}
public boolean needsAuthorization(HttpServletRequest req) {
if (req.getUserPrincipal() == SU) return false;
return true;
return req.getUserPrincipal() != SU;
}
private class HttpHeaderClientConfigurer extends HttpClientConfigurer implements
@ -247,13 +256,12 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
if (disabled()) return;
setHeader(httpRequest);
}
}
@SuppressForbidden(reason = "Needs currentTimeMillis to set current time in header")
void setHeader(HttpRequest httpRequest) {
SolrRequestInfo reqInfo = getRequestInfo();
String usr = null;
String usr;
if (reqInfo != null) {
Principal principal = reqInfo.getReq().getUserPrincipal();
if (principal == null) {
@ -275,7 +283,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
String s = usr + " " + System.currentTimeMillis();
byte[] payload = s.getBytes(StandardCharsets.UTF_8);
byte[] payload = s.getBytes(UTF_8);
byte[] payloadCipher = keyPair.encrypt(ByteBuffer.wrap(payload));
String base64Cipher = Base64.byteArrayToBase64(payloadCipher);
httpRequest.setHeader(HEADER, myNodeName + " " + base64Cipher);

View File

@ -414,7 +414,7 @@ public class HttpSolrCall {
*/
if (cores.getAuthorizationPlugin() != null && shouldAuthorize()) {
AuthorizationContext context = getAuthCtx();
log.info(context.toString());
log.debug("AuthorizationContext : {}", context);
AuthorizationResponse authResponse = cores.getAuthorizationPlugin().authorize(context);
if (authResponse.statusCode == AuthorizationResponse.PROMPT.statusCode) {
Map<String, String> headers = (Map) getReq().getAttribute(AuthenticationPlugin.class.getName());

View File

@ -307,6 +307,10 @@ public final class CryptoKeys {
return pubKeyStr;
}
public PublicKey getPublicKey() {
return publicKey;
}
public byte[] encrypt(ByteBuffer buffer) {
try {
Cipher rsaCipher = Cipher.getInstance("RSA/ECB/nopadding");

View File

@ -88,7 +88,9 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
return principal.get();
}
};
mock.remoteKeys.put(nodeName, CryptoKeys.deserializeX509PublicKey(mock.getPublicKey()));
PublicKey correctKey = CryptoKeys.deserializeX509PublicKey(mock.getPublicKey());
mock.remoteKeys.put(nodeName, correctKey);
principal.set(new BasicUserPrincipal("solr"));
mock.solrRequestInfo = new SolrRequestInfo(localSolrQueryRequest, new SolrQueryResponse());
BasicHttpRequest request = new BasicHttpRequest("GET", "http://localhost:56565");
@ -136,6 +138,27 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
assertNotNull(wrappedRequestByFilter.get());
assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
MockPKIAuthenticationPlugin mock1 = new MockPKIAuthenticationPlugin(null, nodeName) {
int called = 0;
@Override
PublicKey getRemotePublicKey(String nodename) {
try {
return called == 0 ? new CryptoKeys.RSAKeyPair().getPublicKey() : correctKey;
} finally {
called++;
}
}
};
mock1.doAuthenticate(mockReq, null,filterChain );
assertNotNull(wrappedRequestByFilter.get());
assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
}
private HttpServletRequest createMockRequest(final AtomicReference<Header> header) {