mirror of https://github.com/apache/lucene.git
SOLR-13713: JWTAuthPlugin to support multiple JWKS endpoints
(cherry picked from commit 4599f6e9ee
)
This commit is contained in:
parent
b1bccf7cac
commit
4af601eb10
|
@ -71,6 +71,8 @@ New Features
|
||||||
|
|
||||||
* SOLR-13122: Ability to query aliases in Solr Admin UI (janhoy)
|
* SOLR-13122: Ability to query aliases in Solr Admin UI (janhoy)
|
||||||
|
|
||||||
|
* SOLR-13713: JWTAuthPlugin to support multiple JWKS endpoints (janhoy)
|
||||||
|
|
||||||
Improvements
|
Improvements
|
||||||
----------------------
|
----------------------
|
||||||
|
|
||||||
|
|
|
@ -69,7 +69,6 @@ import org.jose4j.jwt.consumer.InvalidJwtException;
|
||||||
import org.jose4j.jwt.consumer.InvalidJwtSignatureException;
|
import org.jose4j.jwt.consumer.InvalidJwtSignatureException;
|
||||||
import org.jose4j.jwt.consumer.JwtConsumer;
|
import org.jose4j.jwt.consumer.JwtConsumer;
|
||||||
import org.jose4j.jwt.consumer.JwtConsumerBuilder;
|
import org.jose4j.jwt.consumer.JwtConsumerBuilder;
|
||||||
import org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver;
|
|
||||||
import org.jose4j.keys.resolvers.JwksVerificationKeyResolver;
|
import org.jose4j.keys.resolvers.JwksVerificationKeyResolver;
|
||||||
import org.jose4j.keys.resolvers.VerificationKeyResolver;
|
import org.jose4j.keys.resolvers.VerificationKeyResolver;
|
||||||
import org.jose4j.lang.JoseException;
|
import org.jose4j.lang.JoseException;
|
||||||
|
@ -102,6 +101,7 @@ public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider,
|
||||||
private static final String AUTH_REALM = "solr-jwt";
|
private static final String AUTH_REALM = "solr-jwt";
|
||||||
private static final String CLAIM_SCOPE = "scope";
|
private static final String CLAIM_SCOPE = "scope";
|
||||||
private static final long RETRY_INIT_DELAY_SECONDS = 30;
|
private static final long RETRY_INIT_DELAY_SECONDS = 30;
|
||||||
|
private static final long DEFAULT_REFRESH_REPRIEVE_THRESHOLD = 5000;
|
||||||
|
|
||||||
private static final Set<String> PROPS = ImmutableSet.of(PARAM_BLOCK_UNKNOWN, PARAM_JWK_URL, PARAM_JWK, PARAM_ISSUER,
|
private static final Set<String> PROPS = ImmutableSet.of(PARAM_BLOCK_UNKNOWN, PARAM_JWK_URL, PARAM_JWK, PARAM_ISSUER,
|
||||||
PARAM_AUDIENCE, PARAM_REQUIRE_SUBJECT, PARAM_PRINCIPAL_CLAIM, PARAM_REQUIRE_EXPIRATIONTIME, PARAM_ALG_WHITELIST,
|
PARAM_AUDIENCE, PARAM_REQUIRE_SUBJECT, PARAM_PRINCIPAL_CLAIM, PARAM_REQUIRE_EXPIRATIONTIME, PARAM_ALG_WHITELIST,
|
||||||
|
@ -120,7 +120,6 @@ public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider,
|
||||||
private boolean blockUnknown;
|
private boolean blockUnknown;
|
||||||
private List<String> requiredScopes = new ArrayList<>();
|
private List<String> requiredScopes = new ArrayList<>();
|
||||||
private String clientId;
|
private String clientId;
|
||||||
private long jwkCacheDuration;
|
|
||||||
private WellKnownDiscoveryConfig oidcDiscoveryConfig;
|
private WellKnownDiscoveryConfig oidcDiscoveryConfig;
|
||||||
private String confIdpConfigUrl;
|
private String confIdpConfigUrl;
|
||||||
private Map<String, Object> pluginConfig;
|
private Map<String, Object> pluginConfig;
|
||||||
|
@ -128,7 +127,7 @@ public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider,
|
||||||
private String authorizationEndpoint;
|
private String authorizationEndpoint;
|
||||||
private String adminUiScope;
|
private String adminUiScope;
|
||||||
private List<String> redirectUris;
|
private List<String> redirectUris;
|
||||||
private HttpsJwks httpsJkws;
|
private IssuerConfig issuerConfig;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -226,9 +225,9 @@ public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider,
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private void initJwk(Map<String, Object> pluginConfig) {
|
private void initJwk(Map<String, Object> pluginConfig) {
|
||||||
this.pluginConfig = pluginConfig;
|
this.pluginConfig = pluginConfig;
|
||||||
String confJwkUrl = (String) pluginConfig.get(PARAM_JWK_URL);
|
Object confJwkUrl = pluginConfig.get(PARAM_JWK_URL);
|
||||||
Map<String, Object> confJwk = (Map<String, Object>) pluginConfig.get(PARAM_JWK);
|
Map<String, Object> confJwk = (Map<String, Object>) pluginConfig.get(PARAM_JWK);
|
||||||
jwkCacheDuration = Long.parseLong((String) pluginConfig.getOrDefault(PARAM_JWK_CACHE_DURATION, "3600"));
|
long jwkCacheDuration = Long.parseLong((String) pluginConfig.getOrDefault(PARAM_JWK_CACHE_DURATION, "3600"));
|
||||||
|
|
||||||
jwtConsumer = null;
|
jwtConsumer = null;
|
||||||
int jwkConfigured = confIdpConfigUrl != null ? 1 : 0;
|
int jwkConfigured = confIdpConfigUrl != null ? 1 : 0;
|
||||||
|
@ -241,40 +240,35 @@ public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider,
|
||||||
if (jwkConfigured == 0) {
|
if (jwkConfigured == 0) {
|
||||||
log.warn("Initialized JWTAuthPlugin without any JWK config. Requests with jwk header will fail.");
|
log.warn("Initialized JWTAuthPlugin without any JWK config. Requests with jwk header will fail.");
|
||||||
}
|
}
|
||||||
if (oidcDiscoveryConfig != null) {
|
|
||||||
String jwkUrl = oidcDiscoveryConfig.getJwksUrl();
|
HttpsJwksFactory httpsJwksFactory = new HttpsJwksFactory(jwkCacheDuration, DEFAULT_REFRESH_REPRIEVE_THRESHOLD);
|
||||||
setupJwkUrl(jwkUrl);
|
if (confJwkUrl != null) {
|
||||||
} else if (confJwkUrl != null) {
|
try {
|
||||||
setupJwkUrl(confJwkUrl);
|
List<String> urls = (confJwkUrl instanceof List) ? (List<String>)confJwkUrl : Collections.singletonList((String) confJwkUrl);
|
||||||
|
issuerConfig = new IssuerConfig(iss, urls);
|
||||||
|
issuerConfig.setHttpsJwksFactory(httpsJwksFactory);
|
||||||
|
verificationKeyResolver = new JWTVerificationkeyResolver(issuerConfig);
|
||||||
|
} catch (ClassCastException e) {
|
||||||
|
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parameter " + PARAM_JWK_URL + " must be either List or String");
|
||||||
|
}
|
||||||
} else if (confJwk != null) {
|
} else if (confJwk != null) {
|
||||||
try {
|
try {
|
||||||
JsonWebKeySet jwks = parseJwkSet(confJwk);
|
JsonWebKeySet jwks = parseJwkSet(confJwk);
|
||||||
|
issuerConfig = new IssuerConfig(iss, jwks);
|
||||||
verificationKeyResolver = new JwksVerificationKeyResolver(jwks.getJsonWebKeys());
|
verificationKeyResolver = new JwksVerificationKeyResolver(jwks.getJsonWebKeys());
|
||||||
httpsJkws = null;
|
|
||||||
} catch (JoseException e) {
|
} catch (JoseException e) {
|
||||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid JWTAuthPlugin configuration, " + PARAM_JWK + " parse error", e);
|
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid JWTAuthPlugin configuration, " + PARAM_JWK + " parse error", e);
|
||||||
}
|
}
|
||||||
|
} else if (oidcDiscoveryConfig != null) {
|
||||||
|
List<String> urls = Collections.singletonList(oidcDiscoveryConfig.getJwksUrl());
|
||||||
|
issuerConfig = new IssuerConfig(iss, urls);
|
||||||
|
issuerConfig.setHttpsJwksFactory(httpsJwksFactory);
|
||||||
|
verificationKeyResolver = new JWTVerificationkeyResolver(issuerConfig);
|
||||||
}
|
}
|
||||||
initConsumer();
|
initConsumer();
|
||||||
log.debug("JWK configured");
|
log.debug("JWK configured");
|
||||||
}
|
}
|
||||||
|
|
||||||
void setupJwkUrl(String url) {
|
|
||||||
// The HttpsJwks retrieves and caches keys from a the given HTTPS JWKS endpoint.
|
|
||||||
try {
|
|
||||||
URL jwkUrl = new URL(url);
|
|
||||||
if (!"https".equalsIgnoreCase(jwkUrl.getProtocol())) {
|
|
||||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, PARAM_JWK_URL + " must be an HTTPS url");
|
|
||||||
}
|
|
||||||
} catch (MalformedURLException e) {
|
|
||||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, PARAM_JWK_URL + " must be a valid URL");
|
|
||||||
}
|
|
||||||
httpsJkws = new HttpsJwks(url);
|
|
||||||
httpsJkws.setDefaultCacheDuration(jwkCacheDuration);
|
|
||||||
httpsJkws.setRefreshReprieveThreshold(5000);
|
|
||||||
verificationKeyResolver = new HttpsJwksVerificationKeyResolver(httpsJkws);
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
JsonWebKeySet parseJwkSet(Map<String, Object> jwkObj) throws JoseException {
|
JsonWebKeySet parseJwkSet(Map<String, Object> jwkObj) throws JoseException {
|
||||||
JsonWebKeySet webKeySet = new JsonWebKeySet();
|
JsonWebKeySet webKeySet = new JsonWebKeySet();
|
||||||
|
@ -319,10 +313,12 @@ public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider,
|
||||||
}
|
}
|
||||||
|
|
||||||
JWTAuthenticationResponse authResponse = authenticate(header);
|
JWTAuthenticationResponse authResponse = authenticate(header);
|
||||||
if (AuthCode.SIGNATURE_INVALID.equals(authResponse.getAuthCode()) && httpsJkws != null) {
|
if (AuthCode.SIGNATURE_INVALID.equals(authResponse.getAuthCode()) && issuerConfig.usesHttpsJwk()) {
|
||||||
log.warn("Signature validation failed. Refreshing JWKs from IdP before trying again: {}",
|
log.warn("Signature validation failed. Refreshing JWKs from IdP before trying again: {}",
|
||||||
authResponse.getJwtException() == null ? "" : authResponse.getJwtException().getMessage());
|
authResponse.getJwtException() == null ? "" : authResponse.getJwtException().getMessage());
|
||||||
httpsJkws.refresh();
|
for (HttpsJwks httpsJwks : issuerConfig.getHttpsJwks()) {
|
||||||
|
httpsJwks.refresh();
|
||||||
|
}
|
||||||
authResponse = authenticate(header);
|
authResponse = authenticate(header);
|
||||||
}
|
}
|
||||||
String exceptionMessage = authResponse.getJwtException() != null ? authResponse.getJwtException().getMessage() : "";
|
String exceptionMessage = authResponse.getJwtException() != null ? authResponse.getJwtException().getMessage() : "";
|
||||||
|
@ -563,7 +559,6 @@ public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider,
|
||||||
return Base64.byteArrayToBase64(headerJson.getBytes(StandardCharsets.UTF_8));
|
return Base64.byteArrayToBase64(headerJson.getBytes(StandardCharsets.UTF_8));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Response for authentication attempt
|
* Response for authentication attempt
|
||||||
*/
|
*/
|
||||||
|
@ -708,6 +703,104 @@ public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Holds information about an IdP (issuer), such as issuer ID, JWK url(s), keys etc
|
||||||
|
*/
|
||||||
|
public static class IssuerConfig {
|
||||||
|
private HttpsJwksFactory httpsJwksFactory;
|
||||||
|
private String iss;
|
||||||
|
private JsonWebKeySet jsonWebKeySet;
|
||||||
|
private List<String> jwksUrl;
|
||||||
|
private List<HttpsJwks> httpsJwks;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create config
|
||||||
|
* @param iss unique issuer id string
|
||||||
|
* @param jwksUrls list of URLs for JWKs endpoints
|
||||||
|
*/
|
||||||
|
public IssuerConfig(String iss, List<String> jwksUrls) {
|
||||||
|
this.jwksUrl = jwksUrls;
|
||||||
|
this.iss = iss;
|
||||||
|
}
|
||||||
|
|
||||||
|
public IssuerConfig(String iss, JsonWebKeySet jsonWebKeySet) {
|
||||||
|
this.iss = iss;
|
||||||
|
this.jsonWebKeySet = jsonWebKeySet;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getIss() {
|
||||||
|
return iss;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> getJwksUrl() {
|
||||||
|
return jwksUrl;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<HttpsJwks> getHttpsJwks() {
|
||||||
|
if (httpsJwks == null) {
|
||||||
|
if (httpsJwksFactory == null) {
|
||||||
|
httpsJwksFactory = new HttpsJwksFactory(3600, DEFAULT_REFRESH_REPRIEVE_THRESHOLD);
|
||||||
|
log.warn("Created HttpsJwksFactory with default cache duration and reprieveThreshold");
|
||||||
|
}
|
||||||
|
httpsJwks = httpsJwksFactory.createList(getJwksUrl());
|
||||||
|
}
|
||||||
|
return httpsJwks;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setHttpsJwks(List<HttpsJwks> httpsJwks) {
|
||||||
|
this.httpsJwks = httpsJwks;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the factory to use when creating HttpsJwks objects
|
||||||
|
* @param httpsJwksFactory factory with custom settings
|
||||||
|
*/
|
||||||
|
public void setHttpsJwksFactory(HttpsJwksFactory httpsJwksFactory) {
|
||||||
|
this.httpsJwksFactory = httpsJwksFactory;
|
||||||
|
}
|
||||||
|
|
||||||
|
public JsonWebKeySet getJsonWebKeySet() {
|
||||||
|
return jsonWebKeySet;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if the issuer is backed by HttpsJwk url(s)
|
||||||
|
* @return true if keys are fetched over https
|
||||||
|
*/
|
||||||
|
public boolean usesHttpsJwk() {
|
||||||
|
return getJwksUrl() != null && !getJwksUrl().isEmpty();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class HttpsJwksFactory {
|
||||||
|
private final long jwkCacheDuration;
|
||||||
|
private final long refreshReprieveThreshold;
|
||||||
|
|
||||||
|
public HttpsJwksFactory(long jwkCacheDuration, long refreshReprieveThreshold) {
|
||||||
|
this.jwkCacheDuration = jwkCacheDuration;
|
||||||
|
this.refreshReprieveThreshold = refreshReprieveThreshold;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HttpsJwks create(String url) {
|
||||||
|
try {
|
||||||
|
URL jwkUrl = new URL(url);
|
||||||
|
if (!"https".equalsIgnoreCase(jwkUrl.getProtocol())) {
|
||||||
|
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, PARAM_JWK_URL + " must use HTTPS");
|
||||||
|
}
|
||||||
|
} catch (MalformedURLException e) {
|
||||||
|
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Url " + url + " configured in " + PARAM_JWK_URL + " is not a valid URL");
|
||||||
|
}
|
||||||
|
HttpsJwks httpsJkws = new HttpsJwks(url);
|
||||||
|
httpsJkws.setDefaultCacheDuration(jwkCacheDuration);
|
||||||
|
httpsJkws.setRefreshReprieveThreshold(refreshReprieveThreshold);
|
||||||
|
return httpsJkws;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<HttpsJwks> createList(List<String> jwkUrls) {
|
||||||
|
return jwkUrls.stream().map(this::create).collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected boolean interceptInternodeRequest(HttpRequest httpRequest, HttpContext httpContext) {
|
protected boolean interceptInternodeRequest(HttpRequest httpRequest, HttpContext httpContext) {
|
||||||
if (httpContext instanceof HttpClientContext) {
|
if (httpContext instanceof HttpClientContext) {
|
||||||
|
|
|
@ -0,0 +1,113 @@
|
||||||
|
/*
|
||||||
|
* 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.security;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.lang.invoke.MethodHandles;
|
||||||
|
import java.security.Key;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.solr.security.JWTAuthPlugin.IssuerConfig;
|
||||||
|
import org.jose4j.jwk.HttpsJwks;
|
||||||
|
import org.jose4j.jwk.JsonWebKey;
|
||||||
|
import org.jose4j.jwk.VerificationJwkSelector;
|
||||||
|
import org.jose4j.jws.JsonWebSignature;
|
||||||
|
import org.jose4j.jwx.JsonWebStructure;
|
||||||
|
import org.jose4j.keys.resolvers.VerificationKeyResolver;
|
||||||
|
import org.jose4j.lang.JoseException;
|
||||||
|
import org.jose4j.lang.UnresolvableKeyException;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adaption of {@link org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver} to resolve
|
||||||
|
* keys from multiple HttpsJwks endpoints, which is sometimes necessary if the IdP
|
||||||
|
* does not publish all public keys that may have signed a token through the main JWKs endpoint.
|
||||||
|
* Such setups typically have support for multiple signing backends, each serving its own JWKs
|
||||||
|
* endpoint for its keys.
|
||||||
|
*
|
||||||
|
* This implementation collects all keys from all endpoints into a single list and
|
||||||
|
* the rest of the implementation is equivalent to that of HttpsJwksVerificationKeyResolver.
|
||||||
|
*
|
||||||
|
* No attempt is made to keep track of which key came from which JWKs endpoint, and if a
|
||||||
|
* key is not found in any cache, all JWKs endpoints are refreshed before a single retry.
|
||||||
|
*
|
||||||
|
* NOTE: This class can subclass HttpsJwksVerificationKeyResolver once a new version of jose4j is available
|
||||||
|
*/
|
||||||
|
public class JWTVerificationkeyResolver implements VerificationKeyResolver {
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
|
||||||
|
private VerificationJwkSelector verificationJwkSelector = new VerificationJwkSelector();
|
||||||
|
|
||||||
|
private IssuerConfig issuerConfig;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Resolves key from a list of JWKs URLs stored in IssuerConfig
|
||||||
|
* @param issuerConfig Configuration object for the issuer
|
||||||
|
*/
|
||||||
|
public JWTVerificationkeyResolver(IssuerConfig issuerConfig) {
|
||||||
|
this.issuerConfig = issuerConfig;
|
||||||
|
assert(issuerConfig.usesHttpsJwk());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Key resolveKey(JsonWebSignature jws, List<JsonWebStructure> nestingContext) throws UnresolvableKeyException {
|
||||||
|
JsonWebKey theChosenOne;
|
||||||
|
List<JsonWebKey> jsonWebKeys = new ArrayList<>();
|
||||||
|
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Add all keys into a master list
|
||||||
|
for (HttpsJwks hjwks : issuerConfig.getHttpsJwks()) {
|
||||||
|
jsonWebKeys.addAll(hjwks.getJsonWebKeys());
|
||||||
|
}
|
||||||
|
|
||||||
|
theChosenOne = verificationJwkSelector.select(jws, jsonWebKeys);
|
||||||
|
if (theChosenOne == null) {
|
||||||
|
log.debug("Refreshing JWKs from all {} locations, as no suitable verification key for JWS w/ header {} was found in {}",
|
||||||
|
issuerConfig.getHttpsJwks().size(), jws.getHeaders().getFullHeaderAsJsonString(), jsonWebKeys);
|
||||||
|
|
||||||
|
jsonWebKeys.clear();
|
||||||
|
for (HttpsJwks hjwks : issuerConfig.getHttpsJwks()) {
|
||||||
|
hjwks.refresh();
|
||||||
|
jsonWebKeys.addAll(hjwks.getJsonWebKeys());
|
||||||
|
}
|
||||||
|
theChosenOne = verificationJwkSelector.select(jws, jsonWebKeys);
|
||||||
|
}
|
||||||
|
} catch (JoseException | IOException e) {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("Unable to find a suitable verification key for JWS w/ header ").append(jws.getHeaders().getFullHeaderAsJsonString());
|
||||||
|
sb.append(" due to an unexpected exception (").append(e).append(") while obtaining or using keys from JWKS endpoints at ");
|
||||||
|
sb.append(issuerConfig.getJwksUrl());
|
||||||
|
throw new UnresolvableKeyException(sb.toString(), e);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (theChosenOne == null) {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("Unable to find a suitable verification key for JWS w/ header ").append(jws.getHeaders().getFullHeaderAsJsonString());
|
||||||
|
sb.append(" from JWKs ").append(jsonWebKeys).append(" obtained from ").append(issuerConfig.getJwksUrl());
|
||||||
|
throw new UnresolvableKeyException(sb.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
return theChosenOne.getKey();
|
||||||
|
}
|
||||||
|
|
||||||
|
IssuerConfig getIssuerConfig() {
|
||||||
|
return issuerConfig;
|
||||||
|
}
|
||||||
|
}
|
|
@ -24,7 +24,6 @@ import java.nio.file.Path;
|
||||||
import java.security.Principal;
|
import java.security.Principal;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -35,16 +34,12 @@ import org.apache.solr.SolrTestCaseJ4;
|
||||||
import org.apache.solr.common.SolrException;
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.util.Base64;
|
import org.apache.solr.common.util.Base64;
|
||||||
import org.apache.solr.common.util.Utils;
|
import org.apache.solr.common.util.Utils;
|
||||||
import org.jose4j.jwk.HttpsJwks;
|
|
||||||
import org.jose4j.jwk.JsonWebKey;
|
|
||||||
import org.jose4j.jwk.RsaJsonWebKey;
|
import org.jose4j.jwk.RsaJsonWebKey;
|
||||||
import org.jose4j.jwk.RsaJwkGenerator;
|
import org.jose4j.jwk.RsaJwkGenerator;
|
||||||
import org.jose4j.jws.AlgorithmIdentifiers;
|
import org.jose4j.jws.AlgorithmIdentifiers;
|
||||||
import org.jose4j.jws.JsonWebSignature;
|
import org.jose4j.jws.JsonWebSignature;
|
||||||
import org.jose4j.jwt.JwtClaims;
|
import org.jose4j.jwt.JwtClaims;
|
||||||
import org.jose4j.keys.BigEndianBigInteger;
|
import org.jose4j.keys.BigEndianBigInteger;
|
||||||
import org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver;
|
|
||||||
import org.jose4j.lang.JoseException;
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
|
@ -64,7 +59,6 @@ public class JWTAuthPluginTest extends SolrTestCaseJ4 {
|
||||||
private HashMap<String, Object> testConfig;
|
private HashMap<String, Object> testConfig;
|
||||||
private HashMap<String, Object> minimalConfig;
|
private HashMap<String, Object> minimalConfig;
|
||||||
|
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void beforeAll() throws Exception {
|
public static void beforeAll() throws Exception {
|
||||||
// Generate an RSA key pair, which will be used for signing and verification of the JWT, wrapped in a JWK
|
// Generate an RSA key pair, which will be used for signing and verification of the JWT, wrapped in a JWK
|
||||||
|
@ -89,7 +83,7 @@ public class JWTAuthPluginTest extends SolrTestCaseJ4 {
|
||||||
slimHeader = "Bearer" + " " + slimJwt;
|
slimHeader = "Bearer" + " " + slimJwt;
|
||||||
}
|
}
|
||||||
|
|
||||||
static JwtClaims generateClaims() {
|
protected static JwtClaims generateClaims() {
|
||||||
JwtClaims claims = new JwtClaims();
|
JwtClaims claims = new JwtClaims();
|
||||||
claims.setIssuer("IDServer"); // who creates the token and signs it
|
claims.setIssuer("IDServer"); // who creates the token and signs it
|
||||||
claims.setAudience("Solr"); // to whom the token is intended to be sent
|
claims.setAudience("Solr"); // to whom the token is intended to be sent
|
||||||
|
@ -112,10 +106,12 @@ public class JWTAuthPluginTest extends SolrTestCaseJ4 {
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
super.setUp();
|
super.setUp();
|
||||||
|
|
||||||
// Create an auth plugin
|
// Create an auth plugin
|
||||||
plugin = new JWTAuthPlugin();
|
plugin = new JWTAuthPlugin();
|
||||||
|
|
||||||
// Create a JWK config for security.json
|
// Create a JWK config for security.json
|
||||||
|
|
||||||
testJwk = new HashMap<>();
|
testJwk = new HashMap<>();
|
||||||
testJwk.put("kty", rsaJsonWebKey.getKeyType());
|
testJwk.put("kty", rsaJsonWebKey.getKeyType());
|
||||||
testJwk.put("e", BigEndianBigInteger.toBase64Url(rsaJsonWebKey.getRsaPublicKey().getPublicExponent()));
|
testJwk.put("e", BigEndianBigInteger.toBase64Url(rsaJsonWebKey.getRsaPublicKey().getPublicExponent()));
|
||||||
|
@ -185,39 +181,18 @@ public class JWTAuthPluginTest extends SolrTestCaseJ4 {
|
||||||
authConf.put("jwkUrl", "https://127.0.0.1:9999/foo.jwk");
|
authConf.put("jwkUrl", "https://127.0.0.1:9999/foo.jwk");
|
||||||
plugin = new JWTAuthPlugin();
|
plugin = new JWTAuthPlugin();
|
||||||
plugin.init(authConf);
|
plugin.init(authConf);
|
||||||
|
JWTVerificationkeyResolver resolver = (JWTVerificationkeyResolver) plugin.verificationKeyResolver;
|
||||||
|
assertEquals(1, resolver.getIssuerConfig().getJwksUrl().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Simulate a rotate of JWK key in IdP.
|
|
||||||
* Validating of JWK signature will fail since we still use old cached JWK set.
|
|
||||||
* Using a mock {@link HttpsJwks} we validate that plugin calls refresh() and then passes validation
|
|
||||||
*/
|
|
||||||
@Test
|
@Test
|
||||||
public void invalidSigRefreshJwk() throws JoseException {
|
public void initWithJwkUrlArray() {
|
||||||
RsaJsonWebKey rsaJsonWebKey2 = RsaJwkGenerator.generateJwk(2048);
|
HashMap<String, Object> authConf = new HashMap<>();
|
||||||
rsaJsonWebKey2.setKeyId("k2");
|
authConf.put("jwkUrl", Arrays.asList("https://127.0.0.1:9999/foo.jwk", "https://127.0.0.1:9999/foo2.jwk"));
|
||||||
HashMap<String, Object> testJwkWrong = new HashMap<>();
|
plugin = new JWTAuthPlugin();
|
||||||
testJwkWrong.put("kty", rsaJsonWebKey2.getKeyType());
|
plugin.init(authConf);
|
||||||
testJwkWrong.put("e", BigEndianBigInteger.toBase64Url(rsaJsonWebKey2.getRsaPublicKey().getPublicExponent()));
|
JWTVerificationkeyResolver resolver = (JWTVerificationkeyResolver) plugin.verificationKeyResolver;
|
||||||
testJwkWrong.put("use", rsaJsonWebKey2.getUse());
|
assertEquals(2, resolver.getIssuerConfig().getJwksUrl().size());
|
||||||
testJwkWrong.put("kid", rsaJsonWebKey2.getKeyId());
|
|
||||||
testJwkWrong.put("alg", rsaJsonWebKey2.getAlgorithm());
|
|
||||||
testJwkWrong.put("n", BigEndianBigInteger.toBase64Url(rsaJsonWebKey2.getRsaPublicKey().getModulus()));
|
|
||||||
JsonWebKey wrongJwk = JsonWebKey.Factory.newJwk(testJwkWrong);
|
|
||||||
|
|
||||||
// Configure our mock plugin with URL as jwk source
|
|
||||||
JsonWebKey correctJwk = JsonWebKey.Factory.newJwk(testJwk);
|
|
||||||
plugin = new MockJwksUrlPlugin(wrongJwk, correctJwk);
|
|
||||||
HashMap<String, Object> pluginConfigJwkUrl = new HashMap<>();
|
|
||||||
pluginConfigJwkUrl.put("class", "org.apache.solr.security.JWTAuthPlugin");
|
|
||||||
pluginConfigJwkUrl.put("jwkUrl", "dummy");
|
|
||||||
plugin.init(pluginConfigJwkUrl);
|
|
||||||
|
|
||||||
// Validate that plugin will call refresh() on invalid signature, then the call succeeds
|
|
||||||
assertFalse(((MockJwksUrlPlugin)plugin).isRefreshCalled());
|
|
||||||
JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
|
|
||||||
assertTrue(resp.isAuthenticated());
|
|
||||||
assertTrue(((MockJwksUrlPlugin)plugin).isRefreshCalled());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -444,49 +419,4 @@ public class JWTAuthPluginTest extends SolrTestCaseJ4 {
|
||||||
assertEquals("http://acmepaymentscorp/oauth/auz/authorize", parsed.get("authorizationEndpoint"));
|
assertEquals("http://acmepaymentscorp/oauth/auz/authorize", parsed.get("authorizationEndpoint"));
|
||||||
assertEquals("solr-cluster", parsed.get("client_id"));
|
assertEquals("solr-cluster", parsed.get("client_id"));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Mock plugin that simulates a {@link HttpsJwks} with cached JWK that returns
|
|
||||||
* a different JWK after a call to refresh()
|
|
||||||
*/
|
|
||||||
private class MockJwksUrlPlugin extends JWTAuthPlugin {
|
|
||||||
private final JsonWebKey wrongJwk;
|
|
||||||
private final JsonWebKey correctJwk;
|
|
||||||
|
|
||||||
boolean isRefreshCalled() {
|
|
||||||
return refreshCalled;
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean refreshCalled;
|
|
||||||
|
|
||||||
MockJwksUrlPlugin(JsonWebKey wrongJwk, JsonWebKey correctJwk) {
|
|
||||||
this.wrongJwk = wrongJwk;
|
|
||||||
this.correctJwk = correctJwk;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
void setupJwkUrl(String url) {
|
|
||||||
MockHttpsJwks httpsJkws = new MockHttpsJwks(url);
|
|
||||||
verificationKeyResolver = new HttpsJwksVerificationKeyResolver(httpsJkws);
|
|
||||||
}
|
|
||||||
|
|
||||||
private class MockHttpsJwks extends HttpsJwks {
|
|
||||||
MockHttpsJwks(String url) {
|
|
||||||
super(url);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<JsonWebKey> getJsonWebKeys() {
|
|
||||||
return refreshCalled ? Collections.singletonList(correctJwk) : Collections.singletonList(wrongJwk);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void refresh() {
|
|
||||||
if (refreshCalled) {
|
|
||||||
fail("Refresh called twice");
|
|
||||||
}
|
|
||||||
refreshCalled = true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
|
@ -0,0 +1,156 @@
|
||||||
|
/*
|
||||||
|
* 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.security;
|
||||||
|
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.solr.SolrTestCaseJ4;
|
||||||
|
import org.apache.solr.security.JWTAuthPlugin.HttpsJwksFactory;
|
||||||
|
import org.apache.solr.security.JWTAuthPlugin.IssuerConfig;
|
||||||
|
import org.jose4j.jwk.HttpsJwks;
|
||||||
|
import org.jose4j.jwk.JsonWebKey;
|
||||||
|
import org.jose4j.jwk.RsaJsonWebKey;
|
||||||
|
import org.jose4j.jwk.RsaJwkGenerator;
|
||||||
|
import org.jose4j.jws.AlgorithmIdentifiers;
|
||||||
|
import org.jose4j.jws.JsonWebSignature;
|
||||||
|
import org.jose4j.lang.JoseException;
|
||||||
|
import org.jose4j.lang.UnresolvableKeyException;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.junit.MockitoJUnit;
|
||||||
|
import org.mockito.junit.MockitoRule;
|
||||||
|
|
||||||
|
import static java.util.Arrays.asList;
|
||||||
|
import static org.mockito.ArgumentMatchers.anyList;
|
||||||
|
import static org.mockito.Mockito.doAnswer;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests the multi jwks resolver that can fetch keys from multiple JWKs
|
||||||
|
*/
|
||||||
|
public class JWTVerificationkeyResolverTest extends SolrTestCaseJ4 {
|
||||||
|
private JWTVerificationkeyResolver resolver;
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public MockitoRule mockitoRule = MockitoJUnit.rule();
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private HttpsJwks firstJwkList;
|
||||||
|
@Mock
|
||||||
|
private HttpsJwks secondJwkList;
|
||||||
|
@Mock
|
||||||
|
private HttpsJwksFactory httpsJwksFactory;
|
||||||
|
|
||||||
|
private KeyHolder k1;
|
||||||
|
private KeyHolder k2;
|
||||||
|
private KeyHolder k3;
|
||||||
|
private KeyHolder k4;
|
||||||
|
private KeyHolder k5;
|
||||||
|
private List<JsonWebKey> keysToReturnFromSecondJwk;
|
||||||
|
private Iterator refreshSequenceForSecondJwk;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
super.setUp();
|
||||||
|
k1 = new KeyHolder("k1");
|
||||||
|
k2 = new KeyHolder("k2");
|
||||||
|
k3 = new KeyHolder("k3");
|
||||||
|
k4 = new KeyHolder("k4");
|
||||||
|
k5 = new KeyHolder("k5");
|
||||||
|
|
||||||
|
when(firstJwkList.getJsonWebKeys()).thenReturn(asList(k1.getJwk(), k2.getJwk()));
|
||||||
|
doAnswer(invocation -> {
|
||||||
|
keysToReturnFromSecondJwk = (List<JsonWebKey>) refreshSequenceForSecondJwk.next();
|
||||||
|
System.out.println("Refresh called, next to return is " + keysToReturnFromSecondJwk);
|
||||||
|
return null;
|
||||||
|
}).when(secondJwkList).refresh();
|
||||||
|
when(secondJwkList.getJsonWebKeys()).then(inv -> {
|
||||||
|
if (keysToReturnFromSecondJwk == null)
|
||||||
|
keysToReturnFromSecondJwk = (List<JsonWebKey>) refreshSequenceForSecondJwk.next();
|
||||||
|
return keysToReturnFromSecondJwk;
|
||||||
|
});
|
||||||
|
when(httpsJwksFactory.createList(anyList())).thenReturn(asList(firstJwkList, secondJwkList));
|
||||||
|
|
||||||
|
IssuerConfig issuerConfig = new IssuerConfig("foo", asList("url1", "url2"));
|
||||||
|
issuerConfig.setHttpsJwksFactory(httpsJwksFactory);
|
||||||
|
resolver = new JWTVerificationkeyResolver(issuerConfig);
|
||||||
|
|
||||||
|
assumeWorkingMockito();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void findKeyFromFirstList() throws JoseException {
|
||||||
|
refreshSequenceForSecondJwk = asList(
|
||||||
|
asList(k3.getJwk(), k4.getJwk()),
|
||||||
|
asList(k5.getJwk())).iterator();
|
||||||
|
resolver.resolveKey(k1.getJws(), null);
|
||||||
|
resolver.resolveKey(k2.getJws(), null);
|
||||||
|
resolver.resolveKey(k3.getJws(), null);
|
||||||
|
resolver.resolveKey(k4.getJws(), null);
|
||||||
|
// Key k5 is not in cache, so a refresh will be done, which
|
||||||
|
resolver.resolveKey(k5.getJws(), null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = UnresolvableKeyException.class)
|
||||||
|
public void notFoundKey() throws JoseException {
|
||||||
|
refreshSequenceForSecondJwk = asList(
|
||||||
|
asList(k3.getJwk()),
|
||||||
|
asList(k4.getJwk()),
|
||||||
|
asList(k5.getJwk())).iterator();
|
||||||
|
// Will not find key since first refresh returns k4, and we only try one refresh.
|
||||||
|
resolver.resolveKey(k5.getJws(), null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public class KeyHolder {
|
||||||
|
private final RsaJsonWebKey key;
|
||||||
|
private final String kid;
|
||||||
|
|
||||||
|
public KeyHolder(String kid) throws JoseException {
|
||||||
|
key = generateKey(kid);
|
||||||
|
this.kid = kid;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RsaJsonWebKey getRsaKey() {
|
||||||
|
return key;
|
||||||
|
}
|
||||||
|
|
||||||
|
public JsonWebKey getJwk() throws JoseException {
|
||||||
|
JsonWebKey jsonKey = JsonWebKey.Factory.newJwk(key.getRsaPublicKey());
|
||||||
|
jsonKey.setKeyId(kid);
|
||||||
|
return jsonKey;
|
||||||
|
}
|
||||||
|
|
||||||
|
public JsonWebSignature getJws() {
|
||||||
|
JsonWebSignature jws = new JsonWebSignature();
|
||||||
|
jws.setPayload(JWTAuthPluginTest.generateClaims().toJson());
|
||||||
|
jws.setKey(getRsaKey().getPrivateKey());
|
||||||
|
jws.setKeyIdHeaderValue(getRsaKey().getKeyId());
|
||||||
|
jws.setAlgorithmHeaderValue(AlgorithmIdentifiers.RSA_USING_SHA256);
|
||||||
|
return jws;
|
||||||
|
}
|
||||||
|
|
||||||
|
private RsaJsonWebKey generateKey(String kid) throws JoseException {
|
||||||
|
RsaJsonWebKey rsaJsonWebKey = RsaJwkGenerator.generateJwk(2048);
|
||||||
|
rsaJsonWebKey.setKeyId(kid);
|
||||||
|
return rsaJsonWebKey;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -46,7 +46,7 @@ wellKnownUrl ; URL to an https://openid.net/specs/openid-connect-discove
|
||||||
clientId ; Client identifier for use with OpenID Connect ; (no default value) Required to authenticate with Admin UI
|
clientId ; Client identifier for use with OpenID Connect ; (no default value) Required to authenticate with Admin UI
|
||||||
realm ; Name of the authentication realm to echo back in HTTP 401 responses. Will also be displayed in Admin UI login page ; 'solr-jwt'
|
realm ; Name of the authentication realm to echo back in HTTP 401 responses. Will also be displayed in Admin UI login page ; 'solr-jwt'
|
||||||
scope ; Whitespace separated list of valid scopes. If configured, the JWT access token MUST contain a `scope` claim with at least one of the listed scopes. Example: `solr:read solr:admin` ;
|
scope ; Whitespace separated list of valid scopes. If configured, the JWT access token MUST contain a `scope` claim with at least one of the listed scopes. Example: `solr:read solr:admin` ;
|
||||||
jwkUrl ; An https URL to a https://tools.ietf.org/html/rfc7517[JWK] keys file. ; Auto configured if `wellKnownUrl` is provided
|
jwkUrl ; A URL to a https://tools.ietf.org/html/rfc7517#section-5[JWKs] endpoint. Must use https protocol. Optionally an array of URLs in which case all public keys from all URLs will be consulted when validating signatures. ; Auto configured if `wellKnownUrl` is provided
|
||||||
jwk ; As an alternative to `jwkUrl` you may provide a JSON object here containing the public key(s) of the issuer. ;
|
jwk ; As an alternative to `jwkUrl` you may provide a JSON object here containing the public key(s) of the issuer. ;
|
||||||
iss ; Validates that the `iss` (issuer) claim equals this string ; Auto configured if `wellKnownUrl` is provided
|
iss ; Validates that the `iss` (issuer) claim equals this string ; Auto configured if `wellKnownUrl` is provided
|
||||||
aud ; Validates that the `aud` (audience) claim equals this string ; If `clientId` is configured, require `aud` to match it
|
aud ; Validates that the `aud` (audience) claim equals this string ; If `clientId` is configured, require `aud` to match it
|
||||||
|
|
Loading…
Reference in New Issue