Skip to content

Commit

Permalink
SOLR-15334 Return error response when failing auth in PKIAuthPlugin (a…
Browse files Browse the repository at this point in the history
  • Loading branch information
madrob authored Apr 19, 2021
1 parent 4a78b45 commit 749af1b
Show file tree
Hide file tree
Showing 3 changed files with 78 additions and 71 deletions.
2 changes: 2 additions & 0 deletions solr/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -272,6 +272,8 @@ Bug Fixes

* SOLR-15317: Correctly handle user principals with whitespace in PKIAuthPlugin (Dominik Dresel, Mike Drob)

* SOLR-15334: Return error response when failing auth in PKIAuthPlugin (Mike Drob)

================== 8.9.0 ==================

Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@

import org.apache.http.HttpEntity;
import org.apache.http.HttpException;
import org.apache.http.HttpHeaders;
import org.apache.http.HttpRequest;
import org.apache.http.HttpRequestInterceptor;
import org.apache.http.HttpResponse;
Expand Down Expand Up @@ -84,46 +85,48 @@ public void init(Map<String, Object> pluginConfig) {
@SuppressForbidden(reason = "Needs currentTimeMillis to compare against time in header")
@Override
public boolean doAuthenticate(HttpServletRequest request, HttpServletResponse response, FilterChain filterChain) throws Exception {
// Getting the received time must be the first thing we do, processing the request can take time
long receivedTime = System.currentTimeMillis();

String requestURI = request.getRequestURI();
if (requestURI.endsWith(PublicKeyHandler.PATH)) {
assert false : "Should already be handled by SolrDispatchFilter.authenticateRequest";

numPassThrough.inc();
filterChain.doFilter(request, response);
return true;
}
long receivedTime = System.currentTimeMillis();

String header = request.getHeader(HEADER);
if (header == null) {
//this must not happen
log.error("No SolrAuth header present");
numMissingCredentials.inc();
filterChain.doFilter(request, response);
return true;
}
assert header != null : "Should have been checked by SolrDispatchFilter.authenticateRequest";

List<String> authInfo = StrUtils.splitWS(header, false);
if (authInfo.size() < 2) {
log.error("Invalid SolrAuth Header {}", header);
if (authInfo.size() != 2) {
numErrors.mark();
filterChain.doFilter(request, response);
return true;
log.error("Invalid SolrAuth header: {}", header);
response.setHeader(HttpHeaders.WWW_AUTHENTICATE, HEADER);
response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "Invalid SolrAuth header");
return false;
}

String nodeName = authInfo.get(0);
String cipher = authInfo.get(1);

PKIHeaderData decipher = decipherHeader(nodeName, cipher);
if (decipher == null) {
log.error("Could not decipher a header {} . No principal set", header);
numMissingCredentials.inc();
filterChain.doFilter(request, response);
return true;
log.error("Could not load principal from SolrAuth header.");
response.setHeader(HttpHeaders.WWW_AUTHENTICATE, HEADER);
response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "Could not load principal from SolrAuth header.");
return false;
}
if ((receivedTime - decipher.timestamp) > MAX_VALIDITY) {
log.error("Invalid key request timestamp: {} , received timestamp: {} , TTL: {}", decipher.timestamp, receivedTime, MAX_VALIDITY);
long elapsed = receivedTime - decipher.timestamp;
if (elapsed > MAX_VALIDITY) {
numErrors.mark();
filterChain.doFilter(request, response);
return true;
log.error("Expired key request timestamp, elapsed={}, TTL={}", elapsed, MAX_VALIDITY);
response.setHeader(HttpHeaders.WWW_AUTHENTICATE, HEADER);
response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "Expired key request timestamp");
return false;
}

final Principal principal = "$".equals(decipher.userName) ?
Expand Down Expand Up @@ -158,7 +161,7 @@ private PKIHeaderData decipherHeader(String nodeName, String cipherBase64) {
}
}

private static PKIHeaderData parseCipher(String cipher, PublicKey key) {
private static PKIHeaderData parseCipher(String cipher, PublicKey key) {
byte[] bytes;
try {
bytes = CryptoKeys.decryptRSA(Base64.base64ToByteArray(cipher), key);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,6 @@
import javax.servlet.http.HttpServletRequest;
import java.security.Principal;
import java.security.PublicKey;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;

import org.apache.http.Header;
Expand All @@ -45,10 +43,11 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
static class MockPKIAuthenticationPlugin extends PKIAuthenticationPlugin {
SolrRequestInfo solrRequestInfo;

Map<String, PublicKey> remoteKeys = new ConcurrentHashMap<>();
final PublicKey myKey;

public MockPKIAuthenticationPlugin(CoreContainer cores, String node) {
super(cores, node, new PublicKeyHandler());
myKey = CryptoKeys.deserializeX509PublicKey(getPublicKey());
}

@Override
Expand All @@ -57,8 +56,8 @@ SolrRequestInfo getRequestInfo() {
}

@Override
PublicKey getRemotePublicKey(String nodename) {
return remoteKeys.get(nodename);
PublicKey getRemotePublicKey(String ignored) {
return myKey;
}

@Override
Expand All @@ -67,60 +66,76 @@ boolean isSolrThread() {
}
}

final AtomicReference<Principal> principal = new AtomicReference<>();
final AtomicReference<Header> header = new AtomicReference<>();
final AtomicReference<ServletRequest> wrappedRequestByFilter = new AtomicReference<>();

final FilterChain filterChain = (servletRequest, servletResponse) -> wrappedRequestByFilter.set(servletRequest);
final HttpServletRequest mockReq = createMockRequest(header);
final String nodeName = "node_x_233";

final LocalSolrQueryRequest localSolrQueryRequest = new LocalSolrQueryRequest(null, new ModifiableSolrParams()) {
@Override
public Principal getUserPrincipal() {
return principal.get();
}
};

MockPKIAuthenticationPlugin mock;
BasicHttpRequest request;

public void test() throws Exception {
@Override
public void setUp() throws Exception {
super.setUp();
assumeWorkingMockito();

AtomicReference<Principal> principal = new AtomicReference<>();
String nodeName = "node_x_233";

final MockPKIAuthenticationPlugin mock = new MockPKIAuthenticationPlugin(null, nodeName);
LocalSolrQueryRequest localSolrQueryRequest = new LocalSolrQueryRequest(null, new ModifiableSolrParams()) {
@Override
public Principal getUserPrincipal() {
return principal.get();
}
};
PublicKey correctKey = CryptoKeys.deserializeX509PublicKey(mock.getPublicKey());
mock.remoteKeys.put(nodeName, correctKey);
principal.set(null);
header.set(null);
wrappedRequestByFilter.set(null);

mock = new MockPKIAuthenticationPlugin(null, nodeName);
request = new BasicHttpRequest("GET", "http://localhost:56565");
}

@Override
public void tearDown() throws Exception {
mock.close();
super.tearDown();
}

public void testBasicRequest() throws Exception {
String username = "solr user"; // with spaces
principal.set(new BasicUserPrincipal(username));
mock.solrRequestInfo = new SolrRequestInfo(localSolrQueryRequest, new SolrQueryResponse());
BasicHttpRequest request = new BasicHttpRequest("GET", "http://localhost:56565");
mock.setHeader(request);
header.set(request.getFirstHeader(PKIAuthenticationPlugin.HEADER));
assertNotNull(header.get());
assertTrue(header.get().getValue().startsWith(nodeName));
HttpServletRequest mockReq = createMockRequest(header);
mock.authenticate(mockReq, null, filterChain);

assertNotNull(wrappedRequestByFilter.get());
assertNotNull(((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal());
assertEquals(username, ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
}

//test 2
principal.set(null); // no user
header.set(null);
wrappedRequestByFilter.set(null);//
request = new BasicHttpRequest("GET", "http://localhost:56565");
mock.setHeader(request);
assertNull(request.getFirstHeader(PKIAuthenticationPlugin.HEADER));
mock.authenticate(mockReq, null, filterChain);
assertNotNull(wrappedRequestByFilter.get());
assertNull(((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal());
public void testSuperUser() throws Exception {
// Simulate the restart of a node, this will return a different key on subsequent invocations.
// Create it in advance because it can take some time and should be done before header is set
MockPKIAuthenticationPlugin mock1 = new MockPKIAuthenticationPlugin(null, nodeName) {
boolean firstCall = true;

//test 3 . No user request . Request originated from Solr
//create pub key in advance because it can take time and it should be
//created before the header is set
PublicKey key = new CryptoKeys.RSAKeyPair().getPublicKey();
@Override
PublicKey getRemotePublicKey(String ignored) {
try {
return firstCall ? myKey : mock.myKey;
} finally {
firstCall = false;
}
}
};

// Setup regular superuser request
mock.solrRequestInfo = null;
header.set(null);
wrappedRequestByFilter.set(null);
request = new BasicHttpRequest("GET", "http://localhost:56565");
mock.setHeader(request);
header.set(request.getFirstHeader(PKIAuthenticationPlugin.HEADER));
assertNotNull(header.get());
Expand All @@ -130,24 +145,11 @@ public Principal getUserPrincipal() {
assertNotNull(wrappedRequestByFilter.get());
assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());

/*test4 mock the restart of a node*/
MockPKIAuthenticationPlugin mock1 = new MockPKIAuthenticationPlugin(null, nodeName) {
int called = 0;
@Override
PublicKey getRemotePublicKey(String nodename) {
try {
return called == 0 ? key : correctKey;
} finally {
called++;
}
}
};

// With the simulated restart
mock1.authenticate(mockReq, null,filterChain );
assertNotNull(wrappedRequestByFilter.get());
assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
mock1.close();
mock.close();
}

private HttpServletRequest createMockRequest(final AtomicReference<Header> header) {
Expand Down

0 comments on commit 749af1b

Please sign in to comment.