extensions = new HashMap<>();
@@ -286,9 +278,8 @@ private void handleExtensionsCallback(SaslExtensionsCallback callback) {
callback.extensions(saslExtensions);
}
- private void checkInitialized() {
+ protected void checkInitialized() {
if (!isInitialized)
- throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName()));
+ throw new IllegalStateException(String.format("To use %s, first call the configure method", getClass().getSimpleName()));
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
index f9422370db18b..28bc5bed5c874 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
@@ -14,32 +14,21 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer;
import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidator;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidatorFactory;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.RefreshingHttpsJwksVerificationKeyResolver;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.ValidateException;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.VerificationKeyResolverFactory;
+import org.apache.kafka.common.utils.Utils;
-import org.jose4j.jws.JsonWebSignature;
-import org.jose4j.jwx.JsonWebStructure;
-import org.jose4j.lang.UnresolvableKeyException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.Closeable;
import java.io.IOException;
-import java.security.Key;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.atomic.AtomicInteger;
import javax.security.auth.callback.Callback;
import javax.security.auth.callback.UnsupportedCallbackException;
@@ -58,8 +47,7 @@
*
*
* This {@link AuthenticateCallbackHandler} is enabled in the broker configuration by setting the
- * {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}
- * like so:
+ * {@link BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG} like so:
*
*
* listener.name..oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallbackHandler
@@ -73,7 +61,7 @@
*
*
* The configuration option
- * {@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}
+ * {@link SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}
* is also required in order to contact the OAuth/OIDC provider to retrieve the JWKS for use in
* JWT signature validation. For example:
*
@@ -103,64 +91,35 @@
*
*
*/
-
-public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler, Closeable {
private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
- /**
- * Because a {@link CloseableVerificationKeyResolver} instance can spawn threads and issue
- * HTTP(S) calls ({@link RefreshingHttpsJwksVerificationKeyResolver}), we only want to create
- * a new instance for each particular set of configuration. Because each set of configuration
- * may have multiple instances, we want to reuse the single instance.
- */
-
- private static final Map VERIFICATION_KEY_RESOLVER_CACHE = new HashMap<>();
-
- private CloseableVerificationKeyResolver verificationKeyResolver;
-
private AccessTokenValidator accessTokenValidator;
private boolean isInitialized = false;
@Override
public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
- Map moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries);
- CloseableVerificationKeyResolver verificationKeyResolver;
-
- // Here's the logic which keeps our VerificationKeyResolvers down to a single instance.
- synchronized (VERIFICATION_KEY_RESOLVER_CACHE) {
- VerificationKeyResolverKey key = new VerificationKeyResolverKey(configs, moduleOptions);
- verificationKeyResolver = VERIFICATION_KEY_RESOLVER_CACHE.computeIfAbsent(key, k ->
- new RefCountingVerificationKeyResolver(VerificationKeyResolverFactory.create(configs, saslMechanism, moduleOptions)));
+ try {
+ configure(new DefaultAccessTokenValidator(), configs, saslMechanism, jaasConfigEntries);
+ } catch (Throwable t) {
+ throw new KafkaException("The OAuth validator configuration encountered an error during initialization", t);
}
-
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
- init(verificationKeyResolver, accessTokenValidator);
}
- public void init(CloseableVerificationKeyResolver verificationKeyResolver, AccessTokenValidator accessTokenValidator) {
- this.verificationKeyResolver = verificationKeyResolver;
+ void configure(AccessTokenValidator accessTokenValidator,
+ Map configs,
+ String saslMechanism,
+ List jaasConfigEntries) {
this.accessTokenValidator = accessTokenValidator;
-
- try {
- verificationKeyResolver.init();
- } catch (Exception e) {
- throw new KafkaException("The OAuth validator configuration encountered an error when initializing the VerificationKeyResolver", e);
- }
-
- isInitialized = true;
+ this.accessTokenValidator.configure(configs, saslMechanism, jaasConfigEntries);
+ this.isInitialized = true;
}
@Override
public void close() {
- if (verificationKeyResolver != null) {
- try {
- verificationKeyResolver.close();
- } catch (Exception e) {
- log.error(e.getMessage(), e);
- }
- }
+ Utils.closeQuietly(accessTokenValidator, "accessTokenValidator");
}
@Override
@@ -181,12 +140,10 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback
private void handleValidatorCallback(OAuthBearerValidatorCallback callback) {
checkInitialized();
- OAuthBearerToken token;
-
try {
- token = accessTokenValidator.validate(callback.tokenValue());
+ OAuthBearerToken token = accessTokenValidator.validate(callback.tokenValue());
callback.token(token);
- } catch (ValidateException e) {
+ } catch (InvalidJwtException e) {
log.warn(e.getMessage(), e);
callback.error("invalid_token", null, null);
}
@@ -200,81 +157,6 @@ private void handleExtensionsValidatorCallback(OAuthBearerExtensionsValidatorCal
private void checkInitialized() {
if (!isInitialized)
- throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName()));
- }
-
- /**
- * VkrKey
is a simple structure which encapsulates the criteria for different
- * sets of configuration. This will allow us to use this object as a key in a {@link Map}
- * to keep a single instance per key.
- */
-
- private static class VerificationKeyResolverKey {
-
- private final Map configs;
-
- private final Map moduleOptions;
-
- public VerificationKeyResolverKey(Map configs, Map moduleOptions) {
- this.configs = configs;
- this.moduleOptions = moduleOptions;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- VerificationKeyResolverKey that = (VerificationKeyResolverKey) o;
- return configs.equals(that.configs) && moduleOptions.equals(that.moduleOptions);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(configs, moduleOptions);
- }
-
+ throw new IllegalStateException(String.format("To use %s, first call the configure method", getClass().getSimpleName()));
}
-
- /**
- * RefCountingVerificationKeyResolver
allows us to share a single
- * {@link CloseableVerificationKeyResolver} instance between multiple
- * {@link AuthenticateCallbackHandler} instances and perform the lifecycle methods the
- * appropriate number of times.
- */
-
- private static class RefCountingVerificationKeyResolver implements CloseableVerificationKeyResolver {
-
- private final CloseableVerificationKeyResolver delegate;
-
- private final AtomicInteger count = new AtomicInteger(0);
-
- public RefCountingVerificationKeyResolver(CloseableVerificationKeyResolver delegate) {
- this.delegate = delegate;
- }
-
- @Override
- public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException {
- return delegate.resolveKey(jws, nestingContext);
- }
-
- @Override
- public void init() throws IOException {
- if (count.incrementAndGet() == 1)
- delegate.init();
- }
-
- @Override
- public void close() throws IOException {
- if (count.decrementAndGet() == 0)
- delegate.close();
- }
-
- }
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetriever.java
deleted file mode 100644
index 080ea4515b4dd..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetriever.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.kafka.common.security.oauthbearer.internals.secured;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * An AccessTokenRetriever
is the internal API by which the login module will
- * retrieve an access token for use in authorization by the broker. The implementation may
- * involve authentication to a remote system, or it can be as simple as loading the contents
- * of a file or configuration setting.
- *
- * Retrieval is a separate concern from validation, so it isn't necessary for
- * the AccessTokenRetriever
implementation to validate the integrity of the JWT
- * access token.
- *
- * @see HttpAccessTokenRetriever
- * @see FileTokenRetriever
- */
-
-public interface AccessTokenRetriever extends Initable, Closeable {
-
- /**
- * Retrieves a JWT access token in its serialized three-part form. The implementation
- * is free to determine how it should be retrieved but should not perform validation
- * on the result.
- *
- * Note: This is a blocking function and callers should be aware that the
- * implementation may be communicating over a network, with the file system, coordinating
- * threads, etc. The facility in the {@link javax.security.auth.spi.LoginModule} from
- * which this is ultimately called does not provide an asynchronous approach.
- *
- * @return Non-null
JWT access token string
- *
- * @throws IOException Thrown on errors related to IO during retrieval
- */
-
- String retrieve() throws IOException;
-
- /**
- * Lifecycle method to perform a clean shutdown of the retriever. This must
- * be performed by the caller to ensure the correct state, freeing up and releasing any
- * resources performed in {@link #init()}.
- *
- * @throws IOException Thrown on errors related to IO during closure
- */
-
- default void close() throws IOException {
- // This method left intentionally blank.
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java
deleted file mode 100644
index 0ed4a1a230349..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.config.SaslConfigs;
-
-import java.net.URL;
-import java.util.Locale;
-import java.util.Map;
-
-import javax.net.ssl.SSLSocketFactory;
-
-import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG;
-
-public class AccessTokenRetrieverFactory {
-
- /**
- * Create an {@link AccessTokenRetriever} from the given SASL and JAAS configuration.
- *
- * Note: the returned AccessTokenRetriever
is not initialized
- * here and must be done by the caller prior to use.
- *
- * @param configs SASL configuration
- * @param jaasConfig JAAS configuration
- *
- * @return Non-null
{@link AccessTokenRetriever}
- */
-
- public static AccessTokenRetriever create(Map configs, Map jaasConfig) {
- return create(configs, null, jaasConfig);
- }
-
- public static AccessTokenRetriever create(Map configs,
- String saslMechanism,
- Map jaasConfig) {
- ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
- URL tokenEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL);
-
- if (tokenEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file")) {
- return new FileTokenRetriever(cu.validateFile(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL));
- } else {
- JaasOptionsUtils jou = new JaasOptionsUtils(jaasConfig);
- String clientId = jou.validateString(CLIENT_ID_CONFIG);
- String clientSecret = jou.validateString(CLIENT_SECRET_CONFIG);
- String scope = jou.validateString(SCOPE_CONFIG, false);
-
- SSLSocketFactory sslSocketFactory = null;
-
- if (jou.shouldCreateSSLSocketFactory(tokenEndpointUrl))
- sslSocketFactory = jou.createSSLSocketFactory();
-
- boolean urlencodeHeader = validateUrlencodeHeader(cu);
-
- return new HttpAccessTokenRetriever(clientId,
- clientSecret,
- scope,
- sslSocketFactory,
- tokenEndpointUrl.toString(),
- cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MS),
- cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS),
- cu.validateInteger(SASL_LOGIN_CONNECT_TIMEOUT_MS, false),
- cu.validateInteger(SASL_LOGIN_READ_TIMEOUT_MS, false),
- urlencodeHeader);
- }
- }
-
- /**
- * In some cases, the incoming {@link Map} doesn't contain a value for
- * {@link SaslConfigs#SASL_OAUTHBEARER_HEADER_URLENCODE}. Returning {@code null} from {@link Map#get(Object)}
- * will cause a {@link NullPointerException} when it is later unboxed.
- *
- *
- *
- * This utility method ensures that we have a non-{@code null} value to use in the
- * {@link HttpAccessTokenRetriever} constructor.
- */
- static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) {
- Boolean urlencodeHeader = configurationUtils.validateBoolean(SASL_OAUTHBEARER_HEADER_URLENCODE, false);
-
- if (urlencodeHeader != null)
- return urlencodeHeader;
- else
- return DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
- }
-
-}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactory.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactory.java
deleted file mode 100644
index e4b39e5cc53c6..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactory.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.kafka.common.security.oauthbearer.internals.secured;
-
-import org.jose4j.keys.resolvers.VerificationKeyResolver;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME;
-
-public class AccessTokenValidatorFactory {
-
- public static AccessTokenValidator create(Map configs) {
- return create(configs, (String) null);
- }
-
- public static AccessTokenValidator create(Map configs, String saslMechanism) {
- ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
- String scopeClaimName = cu.get(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME);
- String subClaimName = cu.get(SASL_OAUTHBEARER_SUB_CLAIM_NAME);
- return new LoginAccessTokenValidator(scopeClaimName, subClaimName);
- }
-
- public static AccessTokenValidator create(Map configs,
- VerificationKeyResolver verificationKeyResolver) {
- return create(configs, null, verificationKeyResolver);
- }
-
- public static AccessTokenValidator create(Map configs,
- String saslMechanism,
- VerificationKeyResolver verificationKeyResolver) {
- ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
- Set expectedAudiences = null;
- List l = cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE);
-
- if (l != null)
- expectedAudiences = Set.copyOf(l);
-
- Integer clockSkew = cu.validateInteger(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, false);
- String expectedIssuer = cu.validateString(SASL_OAUTHBEARER_EXPECTED_ISSUER, false);
- String scopeClaimName = cu.validateString(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME);
- String subClaimName = cu.validateString(SASL_OAUTHBEARER_SUB_CLAIM_NAME);
-
- return new ValidatorAccessTokenValidator(clockSkew,
- expectedAudiences,
- expectedIssuer,
- verificationKeyResolver,
- scopeClaimName,
- subClaimName);
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerToken.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerToken.java
index 48ef89844070c..1ba872c0274b3 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerToken.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerToken.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
@@ -31,7 +30,6 @@
*
* @see RFC 7515: JSON Web Signature (JWS)
*/
-
public class BasicOAuthBearerToken implements OAuthBearerToken {
private final String token;
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java
index 5bf5ef068ed0f..8893eff488aed 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java
@@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.oauthbearer.InvalidJwtException;
+
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
@@ -47,14 +48,14 @@ public class ClaimValidationUtils {
* @return Unmodifiable {@link Set} that includes the values of the original set, but with
* each value trimmed
*
- * @throws ValidateException Thrown if the value is null
, contains duplicates, or
+ * @throws InvalidJwtException Thrown if the value is null
, contains duplicates, or
* if any of the values in the set are null
, empty,
* or whitespace only
*/
- public static Set validateScopes(String scopeClaimName, Collection scopes) throws ValidateException {
+ public static Set validateScopes(String scopeClaimName, Collection scopes) throws InvalidJwtException {
if (scopes == null)
- throw new ValidateException(String.format("%s value must be non-null", scopeClaimName));
+ throw new InvalidJwtException(String.format("%s value must be non-null", scopeClaimName));
Set copy = new HashSet<>();
@@ -62,7 +63,7 @@ public static Set validateScopes(String scopeClaimName, Collection validateScopes(String scopeClaimName, Collectionnull
or negative
+ * @throws InvalidJwtException Thrown if the value is null
or negative
*/
- public static long validateExpiration(String claimName, Long claimValue) throws ValidateException {
+ public static long validateExpiration(String claimName, Long claimValue) throws InvalidJwtException {
if (claimValue == null)
- throw new ValidateException(String.format("%s value must be non-null", claimName));
+ throw new InvalidJwtException(String.format("%s value must be non-null", claimName));
if (claimValue < 0)
- throw new ValidateException(String.format("%s value must be non-negative; value given was \"%s\"", claimName, claimValue));
+ throw new InvalidJwtException(String.format("%s value must be non-negative; value given was \"%s\"", claimName, claimValue));
return claimValue;
}
@@ -112,10 +113,10 @@ public static long validateExpiration(String claimName, Long claimValue) throws
*
* @return Trimmed version of the claimValue
parameter
*
- * @throws ValidateException Thrown if the value is null
, empty, or whitespace only
+ * @throws InvalidJwtException Thrown if the value is null
, empty, or whitespace only
*/
- public static String validateSubject(String claimName, String claimValue) throws ValidateException {
+ public static String validateSubject(String claimName, String claimValue) throws InvalidJwtException {
return validateString(claimName, claimValue);
}
@@ -132,12 +133,12 @@ public static String validateSubject(String claimName, String claimValue) throws
*
* @return Input parameter, as provided
*
- * @throws ValidateException Thrown if the value is negative
+ * @throws InvalidJwtException Thrown if the value is negative
*/
- public static Long validateIssuedAt(String claimName, Long claimValue) throws ValidateException {
+ public static Long validateIssuedAt(String claimName, Long claimValue) throws InvalidJwtException {
if (claimValue != null && claimValue < 0)
- throw new ValidateException(String.format("%s value must be null or non-negative; value given was \"%s\"", claimName, claimValue));
+ throw new InvalidJwtException(String.format("%s value must be null or non-negative; value given was \"%s\"", claimName, claimValue));
return claimValue;
}
@@ -157,24 +158,24 @@ public static Long validateIssuedAt(String claimName, Long claimValue) throws Va
*
* @return Trimmed version of the value
parameter
*
- * @throws ValidateException Thrown if the value is null
, empty, or whitespace only
+ * @throws InvalidJwtException Thrown if the value is null
, empty, or whitespace only
*/
- public static String validateClaimNameOverride(String name, String value) throws ValidateException {
+ public static String validateClaimNameOverride(String name, String value) throws InvalidJwtException {
return validateString(name, value);
}
- private static String validateString(String name, String value) throws ValidateException {
+ private static String validateString(String name, String value) throws InvalidJwtException {
if (value == null)
- throw new ValidateException(String.format("%s value must be non-null", name));
+ throw new InvalidJwtException(String.format("%s value must be non-null", name));
if (value.isEmpty())
- throw new ValidateException(String.format("%s value must be non-empty", name));
+ throw new InvalidJwtException(String.format("%s value must be non-empty", name));
value = value.trim();
if (value.isEmpty())
- throw new ValidateException(String.format("%s value must not contain only whitespace", name));
+ throw new InvalidJwtException(String.format("%s value must not contain only whitespace", name));
return value;
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatter.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatter.java
new file mode 100644
index 0000000000000..eebc5ecbddc9a
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatter.java
@@ -0,0 +1,62 @@
+/*
+ * 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.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.utils.Utils;
+
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+
+public class ClientCredentialsRequestFormatter implements HttpRequestFormatter {
+
+ public static final String GRANT_TYPE = "client_credentials";
+
+ private final String clientId;
+ private final String clientSecret;
+ private final Optional scope;
+
+ public ClientCredentialsRequestFormatter(String clientId,
+ String clientSecret,
+ String scope,
+ boolean urlencodeHeader) {
+ // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1
+ this.clientId = urlencodeHeader ? URLEncoder.encode(clientId, StandardCharsets.UTF_8) : clientId;
+ this.clientSecret = urlencodeHeader ? URLEncoder.encode(clientSecret, StandardCharsets.UTF_8) : clientSecret;
+ this.scope = Utils.isBlank(scope) ? Optional.empty() : Optional.of(URLEncoder.encode(scope, StandardCharsets.UTF_8));
+ }
+
+ @Override
+ public String formatBody() {
+ StringBuilder requestParameters = new StringBuilder();
+ requestParameters.append("grant_type=").append(GRANT_TYPE);
+ scope.ifPresent(s -> requestParameters.append("&scope=").append(s));
+ return requestParameters.toString();
+ }
+
+ @Override
+ public Map formatHeaders() {
+ String s = String.format("%s:%s", clientId, clientSecret);
+ // Per RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496.
+ String encoded = Base64.getEncoder().encodeToString(Utils.utf8(s));
+ String header = String.format("Basic %s", encoded);
+ return Collections.singletonMap("Authorization", header);
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java
index bf8ca0cb82211..f6c09b4717818 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java
@@ -14,15 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerConfigurable;
import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallbackHandler;
import org.jose4j.keys.resolvers.VerificationKeyResolver;
import java.io.Closeable;
-import java.io.IOException;
/**
* The {@link OAuthBearerValidatorCallbackHandler} uses a {@link VerificationKeyResolver} as
@@ -36,18 +35,6 @@
* @see Closeable
*/
-public interface CloseableVerificationKeyResolver extends Initable, Closeable, VerificationKeyResolver {
-
- /**
- * Lifecycle method to perform a clean shutdown of the {@link VerificationKeyResolver}.
- * This must be performed by the caller to ensure the correct state, freeing up
- * and releasing any resources performed in {@link #init()}.
- *
- * @throws IOException Thrown on errors related to IO during closure
- */
-
- default void close() throws IOException {
- // This method left intentionally blank.
- }
+public interface CloseableVerificationKeyResolver extends VerificationKeyResolver, OAuthBearerConfigurable {
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java
index 10f700826c8bd..fc3a5f2b8c028 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java
@@ -14,17 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.utils.Utils;
import java.io.File;
import java.net.MalformedURLException;
import java.net.URISyntaxException;
import java.net.URL;
-import java.nio.file.Path;
import java.util.Arrays;
import java.util.Locale;
import java.util.Map;
@@ -38,21 +37,16 @@
* ConfigurationUtils
is a utility class to perform basic configuration-related
* logic and is separated out here for easier, more direct testing.
*/
-
public class ConfigurationUtils {
private final Map configs;
private final String prefix;
- public ConfigurationUtils(Map configs) {
- this(configs, null);
- }
-
public ConfigurationUtils(Map configs, String saslMechanism) {
this.configs = configs;
- if (saslMechanism != null && !saslMechanism.trim().isEmpty())
+ if (!Utils.isBlank(saslMechanism))
this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
else
this.prefix = null;
@@ -71,7 +65,7 @@ public ConfigurationUtils(Map configs, String saslMechanism) {
* ignored. Any whitespace is trimmed off of the beginning and end.
*/
- public Path validateFile(String name) {
+ public File validateFile(String name) {
URL url = validateUrl(name);
File file;
@@ -90,7 +84,7 @@ public Path validateFile(String name) {
if (file.isDirectory())
throw new ConfigException(String.format("The OAuth configuration option %s references a directory (%s), not a file", name, file));
- return file.toPath();
+ return file;
}
/**
@@ -192,11 +186,11 @@ public URL validateUrl(String name) {
return url;
}
- public String validateString(String name) throws ValidateException {
+ public String validateString(String name) {
return validateString(name, true);
}
- public String validateString(String name, boolean isRequired) throws ValidateException {
+ public String validateString(String name, boolean isRequired) {
String value = get(name);
if (value == null) {
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DelegatingVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DelegatingVerificationKeyResolver.java
new file mode 100644
index 0000000000000..735c542a5e99f
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DelegatingVerificationKeyResolver.java
@@ -0,0 +1,69 @@
+/*
+ * 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.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+
+import java.net.URL;
+import java.security.Key;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+
+import javax.security.auth.login.AppConfigurationEntry;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
+
+public class DelegatingVerificationKeyResolver implements CloseableVerificationKeyResolver {
+
+ private final Time time;
+
+ private CloseableVerificationKeyResolver delegate;
+
+ public DelegatingVerificationKeyResolver(Time time) {
+ this.time = time;
+ }
+
+ @Override
+ public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
+ ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
+ URL jwksEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
+
+ if (jwksEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file"))
+ delegate = new JwksFileVerificationKeyResolver();
+ else
+ delegate = new RefreshingHttpsJwksVerificationKeyResolver(time);
+
+ delegate.configure(configs, saslMechanism, jaasConfigEntries);
+ }
+
+ @Override
+ public void close() {
+ Utils.closeQuietly(delegate, "delegate");
+ }
+
+ @Override
+ public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException {
+ return Objects.requireNonNull(delegate).resolveKey(jws, nestingContext);
+ }
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetriever.java
deleted file mode 100644
index fdc5707278a60..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetriever.java
+++ /dev/null
@@ -1,400 +0,0 @@
-/*
- * 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.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.config.SaslConfigs;
-import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler;
-import org.apache.kafka.common.utils.Utils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.net.URLEncoder;
-import java.nio.charset.StandardCharsets;
-import java.util.Base64;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.SSLSocketFactory;
-
-/**
- * HttpAccessTokenRetriever
is an {@link AccessTokenRetriever} that will
- * communicate with an OAuth/OIDC provider directly via HTTP to post client credentials
- * ({@link OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG}/{@link OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG})
- * to a publicized token endpoint URL
- * ({@link SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL}).
- *
- * @see AccessTokenRetriever
- * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
- * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
- * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
- * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL
- */
-
-public class HttpAccessTokenRetriever implements AccessTokenRetriever {
-
- private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
-
- private static final Set UNRETRYABLE_HTTP_CODES;
-
- private static final int MAX_RESPONSE_BODY_LENGTH = 1000;
-
- public static final String AUTHORIZATION_HEADER = "Authorization";
-
- static {
- // This does not have to be an exhaustive list. There are other HTTP codes that
- // are defined in different RFCs (e.g. https://datatracker.ietf.org/doc/html/rfc6585)
- // that we won't worry about yet. The worst case if a status code is missing from
- // this set is that the request will be retried.
- UNRETRYABLE_HTTP_CODES = new HashSet<>();
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_BAD_REQUEST);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_UNAUTHORIZED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_PAYMENT_REQUIRED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_FORBIDDEN);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_NOT_FOUND);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_BAD_METHOD);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_NOT_ACCEPTABLE);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_PROXY_AUTH);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_CONFLICT);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_GONE);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_LENGTH_REQUIRED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_PRECON_FAILED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_ENTITY_TOO_LARGE);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_REQ_TOO_LONG);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_UNSUPPORTED_TYPE);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_VERSION);
- }
-
- private final String clientId;
-
- private final String clientSecret;
-
- private final String scope;
-
- private final SSLSocketFactory sslSocketFactory;
-
- private final String tokenEndpointUrl;
-
- private final long loginRetryBackoffMs;
-
- private final long loginRetryBackoffMaxMs;
-
- private final Integer loginConnectTimeoutMs;
-
- private final Integer loginReadTimeoutMs;
-
- private final boolean urlencodeHeader;
-
- public HttpAccessTokenRetriever(String clientId,
- String clientSecret,
- String scope,
- SSLSocketFactory sslSocketFactory,
- String tokenEndpointUrl,
- long loginRetryBackoffMs,
- long loginRetryBackoffMaxMs,
- Integer loginConnectTimeoutMs,
- Integer loginReadTimeoutMs,
- boolean urlencodeHeader) {
- this.clientId = Objects.requireNonNull(clientId);
- this.clientSecret = Objects.requireNonNull(clientSecret);
- this.scope = scope;
- this.sslSocketFactory = sslSocketFactory;
- this.tokenEndpointUrl = Objects.requireNonNull(tokenEndpointUrl);
- this.loginRetryBackoffMs = loginRetryBackoffMs;
- this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
- this.loginConnectTimeoutMs = loginConnectTimeoutMs;
- this.loginReadTimeoutMs = loginReadTimeoutMs;
- this.urlencodeHeader = urlencodeHeader;
- }
-
- /**
- * Retrieves a JWT access token in its serialized three-part form. The implementation
- * is free to determine how it should be retrieved but should not perform validation
- * on the result.
- *
- * Note: This is a blocking function and callers should be aware that the
- * implementation communicates over a network. The facility in the
- * {@link javax.security.auth.spi.LoginModule} from which this is ultimately called
- * does not provide an asynchronous approach.
- *
- * @return Non-null
JWT access token string
- *
- * @throws IOException Thrown on errors related to IO during retrieval
- */
-
- @Override
- public String retrieve() throws IOException {
- String authorizationHeader = formatAuthorizationHeader(clientId, clientSecret, urlencodeHeader);
- String requestBody = formatRequestBody(scope);
- Retry retry = new Retry<>(loginRetryBackoffMs, loginRetryBackoffMaxMs);
- Map headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
-
- String responseBody;
-
- try {
- responseBody = retry.execute(() -> {
- HttpURLConnection con = null;
-
- try {
- con = (HttpURLConnection) new URL(tokenEndpointUrl).openConnection();
-
- if (sslSocketFactory != null && con instanceof HttpsURLConnection)
- ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
-
- return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
- } catch (IOException e) {
- throw new ExecutionException(e);
- } finally {
- if (con != null)
- con.disconnect();
- }
- });
- } catch (ExecutionException e) {
- if (e.getCause() instanceof IOException)
- throw (IOException) e.getCause();
- else
- throw new KafkaException(e.getCause());
- }
-
- return parseAccessToken(responseBody);
- }
-
- public static String post(HttpURLConnection con,
- Map headers,
- String requestBody,
- Integer connectTimeoutMs,
- Integer readTimeoutMs)
- throws IOException, UnretryableException {
- handleInput(con, headers, requestBody, connectTimeoutMs, readTimeoutMs);
- return handleOutput(con);
- }
-
- private static void handleInput(HttpURLConnection con,
- Map headers,
- String requestBody,
- Integer connectTimeoutMs,
- Integer readTimeoutMs)
- throws IOException, UnretryableException {
- log.debug("handleInput - starting post for {}", con.getURL());
- con.setRequestMethod("POST");
- con.setRequestProperty("Accept", "application/json");
-
- if (headers != null) {
- for (Map.Entry header : headers.entrySet())
- con.setRequestProperty(header.getKey(), header.getValue());
- }
-
- con.setRequestProperty("Cache-Control", "no-cache");
-
- if (requestBody != null) {
- con.setRequestProperty("Content-Length", String.valueOf(requestBody.length()));
- con.setDoOutput(true);
- }
-
- con.setUseCaches(false);
-
- if (connectTimeoutMs != null)
- con.setConnectTimeout(connectTimeoutMs);
-
- if (readTimeoutMs != null)
- con.setReadTimeout(readTimeoutMs);
-
- log.debug("handleInput - preparing to connect to {}", con.getURL());
- con.connect();
-
- if (requestBody != null) {
- try (OutputStream os = con.getOutputStream()) {
- ByteArrayInputStream is = new ByteArrayInputStream(requestBody.getBytes(StandardCharsets.UTF_8));
- log.debug("handleInput - preparing to write request body to {}", con.getURL());
- copy(is, os);
- }
- }
- }
-
- static String handleOutput(final HttpURLConnection con) throws IOException {
- int responseCode = con.getResponseCode();
- log.debug("handleOutput - responseCode: {}", responseCode);
-
- // NOTE: the contents of the response should not be logged so that we don't leak any
- // sensitive data.
- String responseBody = null;
-
- // NOTE: It is OK to log the error response body and/or its formatted version as
- // per the OAuth spec, it doesn't include sensitive information.
- // See https://www.ietf.org/rfc/rfc6749.txt, section 5.2
- String errorResponseBody = null;
-
- try (InputStream is = con.getInputStream()) {
- ByteArrayOutputStream os = new ByteArrayOutputStream();
- log.debug("handleOutput - preparing to read response body from {}", con.getURL());
- copy(is, os);
- responseBody = os.toString(StandardCharsets.UTF_8);
- } catch (Exception e) {
- // there still can be useful error response from the servers, lets get it
- try (InputStream is = con.getErrorStream()) {
- ByteArrayOutputStream os = new ByteArrayOutputStream();
- log.debug("handleOutput - preparing to read error response body from {}", con.getURL());
- copy(is, os);
- errorResponseBody = os.toString(StandardCharsets.UTF_8);
- } catch (Exception e2) {
- log.warn("handleOutput - error retrieving error information", e2);
- }
- log.warn("handleOutput - error retrieving data", e);
- }
-
- if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
- log.debug("handleOutput - responseCode: {}, error response: {}", responseCode,
- errorResponseBody);
-
- if (responseBody == null || responseBody.isEmpty())
- throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %d from %s and error message %s",
- responseCode, con.getURL(), formatErrorMessage(errorResponseBody)));
-
- return responseBody;
- } else {
- log.warn("handleOutput - error response code: {}, error response body: {}", responseCode,
- formatErrorMessage(errorResponseBody));
-
- if (UNRETRYABLE_HTTP_CODES.contains(responseCode)) {
- // We know that this is a non-transient error, so let's not keep retrying the
- // request unnecessarily.
- throw new UnretryableException(new IOException(String.format("The response code %s and error response %s was encountered reading the token endpoint response; will not attempt further retries",
- responseCode, formatErrorMessage(errorResponseBody))));
- } else {
- // We don't know if this is a transient (retryable) error or not, so let's assume
- // it is.
- throw new IOException(String.format("The unexpected response code %s and error message %s was encountered reading the token endpoint response",
- responseCode, formatErrorMessage(errorResponseBody)));
- }
- }
- }
-
- static void copy(InputStream is, OutputStream os) throws IOException {
- byte[] buf = new byte[4096];
- int b;
-
- while ((b = is.read(buf)) != -1)
- os.write(buf, 0, b);
- }
-
- static String formatErrorMessage(String errorResponseBody) {
- // See https://www.ietf.org/rfc/rfc6749.txt, section 5.2 for the format
- // of this error message.
- if (errorResponseBody == null || errorResponseBody.trim().isEmpty()) {
- return "{}";
- }
- ObjectMapper mapper = new ObjectMapper();
- try {
- JsonNode rootNode = mapper.readTree(errorResponseBody);
- if (!rootNode.at("/error").isMissingNode()) {
- return String.format("{%s - %s}", rootNode.at("/error"), rootNode.at("/error_description"));
- } else if (!rootNode.at("/errorCode").isMissingNode()) {
- return String.format("{%s - %s}", rootNode.at("/errorCode"), rootNode.at("/errorSummary"));
- } else {
- return errorResponseBody;
- }
- } catch (Exception e) {
- log.warn("Error parsing error response", e);
- }
- return String.format("{%s}", errorResponseBody);
- }
-
- static String parseAccessToken(String responseBody) throws IOException {
- ObjectMapper mapper = new ObjectMapper();
- JsonNode rootNode = mapper.readTree(responseBody);
- JsonNode accessTokenNode = rootNode.at("/access_token");
-
- if (accessTokenNode == null) {
- // Only grab the first N characters so that if the response body is huge, we don't
- // blow up.
- String snippet = responseBody;
-
- if (snippet.length() > MAX_RESPONSE_BODY_LENGTH) {
- int actualLength = responseBody.length();
- String s = responseBody.substring(0, MAX_RESPONSE_BODY_LENGTH);
- snippet = String.format("%s (trimmed to first %d characters out of %d total)", s, MAX_RESPONSE_BODY_LENGTH, actualLength);
- }
-
- throw new IOException(String.format("The token endpoint response did not contain an access_token value. Response: (%s)", snippet));
- }
-
- return sanitizeString("the token endpoint response's access_token JSON attribute", accessTokenNode.textValue());
- }
-
- static String formatAuthorizationHeader(String clientId, String clientSecret, boolean urlencode) {
- clientId = sanitizeString("the token endpoint request client ID parameter", clientId);
- clientSecret = sanitizeString("the token endpoint request client secret parameter", clientSecret);
-
- // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1
- if (urlencode) {
- clientId = URLEncoder.encode(clientId, StandardCharsets.UTF_8);
- clientSecret = URLEncoder.encode(clientSecret, StandardCharsets.UTF_8);
- }
-
- String s = String.format("%s:%s", clientId, clientSecret);
- // Per RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496.
- String encoded = Base64.getEncoder().encodeToString(Utils.utf8(s));
- return String.format("Basic %s", encoded);
- }
-
- static String formatRequestBody(String scope) {
- StringBuilder requestParameters = new StringBuilder();
- requestParameters.append("grant_type=client_credentials");
-
- if (scope != null && !scope.trim().isEmpty()) {
- scope = scope.trim();
- String encodedScope = URLEncoder.encode(scope, StandardCharsets.UTF_8);
- requestParameters.append("&scope=").append(encodedScope);
- }
-
- return requestParameters.toString();
- }
-
- private static String sanitizeString(String name, String value) {
- if (value == null)
- throw new IllegalArgumentException(String.format("The value for %s must be non-null", name));
-
- if (value.isEmpty())
- throw new IllegalArgumentException(String.format("The value for %s must be non-empty", name));
-
- value = value.trim();
-
- if (value.isEmpty())
- throw new IllegalArgumentException(String.format("The value for %s must not contain only whitespace", name));
-
- return value;
- }
-
-}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestFormatter.java
similarity index 75%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java
rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestFormatter.java
index fc2e3d2a2e83a..17eb2e58f7a3d 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestFormatter.java
@@ -14,14 +14,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
-public class LoginAccessTokenValidatorTest extends AccessTokenValidatorTest {
+import java.util.Map;
+
+public interface HttpRequestFormatter {
- @Override
- protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) {
- return new LoginAccessTokenValidator(builder.scopeClaimName(), builder.subjectClaimName());
- }
+ String formatBody();
+ Map formatHeaders();
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java
index 3e49595dbc1b2..3cba7e7c5032b 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.config.AbstractConfig;
@@ -33,6 +32,7 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Optional;
import javax.net.ssl.SSLSocketFactory;
import javax.security.auth.login.AppConfigurationEntry;
@@ -41,7 +41,6 @@
* JaasOptionsUtils
is a utility class to perform logic for the JAAS options and
* is separated out here for easier, more direct testing.
*/
-
public class JaasOptionsUtils {
private static final Logger log = LoggerFactory.getLogger(JaasOptionsUtils.class);
@@ -52,6 +51,10 @@ public JaasOptionsUtils(Map options) {
this.options = options;
}
+ public JaasOptionsUtils(String saslMechanism, List jaasConfigEntries) {
+ this.options = getOptions(saslMechanism, jaasConfigEntries);
+ }
+
public static Map getOptions(String saslMechanism, List jaasConfigEntries) {
if (!OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(saslMechanism))
throw new IllegalArgumentException(String.format("Unexpected SASL mechanism: %s", saslMechanism));
@@ -62,8 +65,17 @@ public static Map getOptions(String saslMechanism, List maybeCreateSslResource(URL url) {
+ if (url.getProtocol().equalsIgnoreCase("https")) {
+ Map sslClientConfig = getSslClientConfig();
+ SslFactory sslFactory = new SslFactory(ConnectionMode.CLIENT);
+ sslFactory.configure(sslClientConfig);
+ SSLSocketFactory sslSocketFactory = ((DefaultSslEngineFactory) sslFactory.sslEngineFactory()).sslContext().getSocketFactory();
+ log.debug("Created SSLSocketFactory from: {}", sslClientConfig);
+ return Optional.of(new SslResource(sslFactory, sslSocketFactory));
+ } else {
+ return Optional.empty();
+ }
}
public Map getSslClientConfig() {
@@ -73,25 +85,16 @@ public boolean shouldCreateSSLSocketFactory(URL url) {
return sslClientConfig.values();
}
- public SSLSocketFactory createSSLSocketFactory() {
- Map sslClientConfig = getSslClientConfig();
- SslFactory sslFactory = new SslFactory(ConnectionMode.CLIENT);
- sslFactory.configure(sslClientConfig);
- SSLSocketFactory socketFactory = ((DefaultSslEngineFactory) sslFactory.sslEngineFactory()).sslContext().getSocketFactory();
- log.debug("Created SSLSocketFactory: {}", sslClientConfig);
- return socketFactory;
- }
-
- public String validateString(String name) throws ValidateException {
+ public String validateString(String name) {
return validateString(name, true);
}
- public String validateString(String name, boolean isRequired) throws ValidateException {
+ public String validateString(String name, boolean isRequired) {
String value = (String) options.get(name);
if (value == null) {
if (isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s value must be non-null", name));
+ throw new ConfigException(String.format("The OAuth JAAS option %s value must be non-null", name));
else
return null;
}
@@ -100,12 +103,11 @@ public String validateString(String name, boolean isRequired) throws ValidateExc
if (value.isEmpty()) {
if (isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s value must not contain only whitespace", name));
+ throw new ConfigException(String.format("The OAuth JAAS option %s value must not contain only whitespace", name));
else
return null;
}
return value;
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java
index 27cdccb286c2c..b35ac923bbacf 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java
@@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.utils.Utils;
import org.jose4j.jwk.JsonWebKeySet;
@@ -24,15 +25,18 @@
import org.jose4j.jwx.JsonWebStructure;
import org.jose4j.keys.resolvers.JwksVerificationKeyResolver;
import org.jose4j.keys.resolvers.VerificationKeyResolver;
-import org.jose4j.lang.JoseException;
import org.jose4j.lang.UnresolvableKeyException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.nio.file.Path;
+import java.io.File;
import java.security.Key;
import java.util.List;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
/**
* JwksFileVerificationKeyResolver
is a {@link VerificationKeyResolver} implementation
@@ -76,33 +80,27 @@
* Given that the JWKS is referenced by the JWT, the JWKS must be made available by the
* OAuth/OIDC provider so that a JWT can be validated.
*
- * @see org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL
* @see VerificationKeyResolver
*/
-
public class JwksFileVerificationKeyResolver implements CloseableVerificationKeyResolver {
private static final Logger log = LoggerFactory.getLogger(JwksFileVerificationKeyResolver.class);
- private final Path jwksFile;
-
private VerificationKeyResolver delegate;
- public JwksFileVerificationKeyResolver(Path jwksFile) {
- this.jwksFile = jwksFile;
- }
-
@Override
- public void init() throws IOException {
- log.debug("Starting creation of new VerificationKeyResolver from {}", jwksFile);
- String json = Utils.readFileAsString(jwksFile.toFile().getPath());
-
+ public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
+ ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
+ File jwksFile = cu.validateFile(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
+ log.debug("Starting creation of new VerificationKeyResolver from {}", jwksFile.getPath());
JsonWebKeySet jwks;
try {
+ String json = Utils.readFileAsString(jwksFile.getPath());
jwks = new JsonWebKeySet(json);
- } catch (JoseException e) {
- throw new IOException(e);
+ } catch (Exception e) {
+ throw new KafkaException(e);
}
delegate = new JwksVerificationKeyResolver(jwks.getJsonWebKeys());
@@ -116,4 +114,8 @@ public Key resolveKey(JsonWebSignature jws, List nestingContex
return delegate.resolveKey(jws, nestingContext);
}
+ @Override
+ public void close() {
+ // Do nothing...
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatter.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatter.java
new file mode 100644
index 0000000000000..82fe564c643d4
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatter.java
@@ -0,0 +1,158 @@
+/*
+ * 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.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.security.InvalidKeyException;
+import java.security.KeyFactory;
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivateKey;
+import java.security.Signature;
+import java.security.SignatureException;
+import java.security.spec.InvalidKeySpecException;
+import java.security.spec.PKCS8EncodedKeySpec;
+import java.time.Duration;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class JwtBearerRequestFormatter implements HttpRequestFormatter {
+
+ public static final String GRANT_TYPE = "urn:ietf:params:oauth:grant-type:jwt-bearer";
+
+ static final String TOKEN_SIGNING_ALGORITHM_RS256 = "RS256";
+ static final String TOKEN_SIGNING_ALGORITHM_ES256 = "ES256";
+
+ private final Time time;
+ private final String privateKeyId;
+ private final String privateKeySecret;
+ private final String tokenSigningAlgo;
+ private final String tokenSubject;
+ private final String tokenIssuer;
+ private final String tokenAudience;
+ private final String tokenTargetAudience;
+
+ public JwtBearerRequestFormatter(Time time,
+ String privateKeyId,
+ String privateKeySecret,
+ String tokenSigningAlgo,
+ String tokenSubject,
+ String tokenIssuer,
+ String tokenAudience,
+ String tokenTargetAudience) {
+ this.time = time;
+ this.privateKeyId = privateKeyId;
+ this.privateKeySecret = privateKeySecret;
+ this.tokenSigningAlgo = tokenSigningAlgo;
+ this.tokenSubject = tokenSubject;
+ this.tokenIssuer = tokenIssuer;
+ this.tokenAudience = tokenAudience;
+ this.tokenTargetAudience = tokenTargetAudience;
+ }
+
+ @Override
+ public String formatBody() {
+ String assertion;
+
+ try {
+ assertion = createAssertion();
+ } catch (Exception e) {
+ throw new KafkaException("Error signing assertion with private key", e);
+ }
+
+ String encodedGrantType = URLEncoder.encode(GRANT_TYPE, StandardCharsets.UTF_8);
+ String encodedAssertion = URLEncoder.encode(assertion, StandardCharsets.UTF_8);
+ return String.format("grant_type=%s&assertion=%s", encodedGrantType, encodedAssertion);
+ }
+
+ @Override
+ public Map formatHeaders() {
+ return Collections.singletonMap("Content-Type", "application/x-www-form-urlencoded");
+ }
+
+ String createAssertion() throws IOException, GeneralSecurityException {
+ ObjectMapper mapper = new ObjectMapper();
+ Base64.Encoder encoder = Base64.getUrlEncoder().withoutPadding();
+ String header = encodeHeader(mapper, encoder);
+ String payload = encodePayload(mapper, encoder);
+ String content = header + "." + payload;
+ PrivateKey privateKey = getPrivateKey();
+ String signedContent = sign(privateKey, content);
+ return content + "." + signedContent;
+ }
+
+ PrivateKey getPrivateKey() throws NoSuchAlgorithmException, InvalidKeySpecException {
+ byte[] pkcs8EncodedBytes = Base64.getDecoder().decode(privateKeySecret);
+ PKCS8EncodedKeySpec keySpec = new PKCS8EncodedKeySpec(pkcs8EncodedBytes);
+ KeyFactory keyFactory = KeyFactory.getInstance("RSA");
+ return keyFactory.generatePrivate(keySpec);
+ }
+
+ String encodeHeader(ObjectMapper mapper, Base64.Encoder encoder) throws IOException {
+ Map values = new HashMap<>();
+ values.put("alg", tokenSigningAlgo);
+ values.put("typ", "JWT");
+ values.put("kid", privateKeyId);
+
+ String json = mapper.writeValueAsString(values);
+ return encoder.encodeToString(Utils.utf8(json));
+ }
+
+ String encodePayload(ObjectMapper mapper, Base64.Encoder encoder) throws IOException {
+ long currentTimeSecs = time.milliseconds() / 1000L;
+ long expirationSecs = currentTimeSecs + Duration.ofMinutes(60).toSeconds();
+
+ Map values = new HashMap<>();
+ values.put("iss", tokenIssuer);
+ values.put("sub", tokenSubject);
+ values.put("aud", tokenAudience);
+ values.put("iat", currentTimeSecs);
+ values.put("exp", expirationSecs);
+ values.put("target_audience", tokenTargetAudience);
+
+ String json = mapper.writeValueAsString(values);
+ return encoder.encodeToString(Utils.utf8(json));
+ }
+
+ Signature getSignature() throws NoSuchAlgorithmException {
+ if (tokenSigningAlgo.equalsIgnoreCase(TOKEN_SIGNING_ALGORITHM_RS256)) {
+ return Signature.getInstance("SHA256withRSA");
+ } else if (tokenSigningAlgo.equalsIgnoreCase(TOKEN_SIGNING_ALGORITHM_ES256)) {
+ return Signature.getInstance("SHA256withECDSA");
+ } else {
+ throw new NoSuchAlgorithmException(String.format("Unsupported signing algorithm: %s", tokenSigningAlgo));
+ }
+ }
+
+ String sign(PrivateKey privateKey, String contentToSign) throws InvalidKeyException, SignatureException, NoSuchAlgorithmException {
+ Signature signature = getSignature();
+ signature.initSign(privateKey);
+ signature.update(contentToSign.getBytes(StandardCharsets.UTF_8));
+ byte[] signedContent = signature.sign();
+ return Base64.getUrlEncoder().withoutPadding().encodeToString(signedContent);
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java
index 62261fed58df8..b4a351da07f49 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java
@@ -14,23 +14,28 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.oauthbearer.DefaultAccessTokenValidator;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerConfigurable;
import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.jose4j.http.Get;
import org.jose4j.jwk.HttpsJwks;
import org.jose4j.jwk.JsonWebKey;
import org.jose4j.lang.JoseException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.Closeable;
import java.io.IOException;
+import java.net.URL;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
@@ -39,6 +44,13 @@
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.security.auth.login.AppConfigurationEntry;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
+
/**
* Implementation of {@link HttpsJwks} that will periodically refresh the JWKS cache to reduce or
* even prevent HTTP/HTTPS traffic in the hot path of validation. It is assumed that it's
@@ -49,15 +61,14 @@
* This instance is created and provided to the
* {@link org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver} that is used when using
* an HTTP-/HTTPS-based {@link org.jose4j.keys.resolvers.VerificationKeyResolver}, which is then
- * provided to the {@link ValidatorAccessTokenValidator} to use in validating the signature of
+ * provided to the {@link DefaultAccessTokenValidator} to use in validating the signature of
* a JWT.
*
* @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
* @see org.jose4j.keys.resolvers.VerificationKeyResolver
- * @see ValidatorAccessTokenValidator
+ * @see DefaultAccessTokenValidator
*/
-
-public final class RefreshingHttpsJwks implements Initable, Closeable {
+public class RefreshingHttpsJwks implements OAuthBearerConfigurable {
private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
@@ -71,32 +82,10 @@ public final class RefreshingHttpsJwks implements Initable, Closeable {
private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
- /**
- * {@link HttpsJwks} does the actual work of contacting the OAuth/OIDC endpoint to get the
- * JWKS. In some cases, the call to {@link HttpsJwks#getJsonWebKeys()} will trigger a call
- * to {@link HttpsJwks#refresh()} which will block the current thread in network I/O. We cache
- * the JWKS ourselves (see {@link #jsonWebKeys}) to avoid the network I/O.
- *
- * We want to be very careful where we use the {@link HttpsJwks} instance so that we don't
- * perform any operation (directly or indirectly) that could cause blocking. This is because
- * the JWKS logic is part of the larger authentication logic which operates on Kafka's network
- * thread. It's OK to execute {@link HttpsJwks#getJsonWebKeys()} (which calls
- * {@link HttpsJwks#refresh()}) from within {@link #init()} as that method is called only at
- * startup, and we can afford the blocking hit there.
- */
-
- private final HttpsJwks httpsJwks;
-
private final ScheduledExecutorService executorService;
private final Time time;
- private final long refreshMs;
-
- private final long refreshRetryBackoffMs;
-
- private final long refreshRetryBackoffMaxMs;
-
/**
* Protects {@link #missingKeyIds} and {@link #jsonWebKeys}.
*/
@@ -111,6 +100,30 @@ public final class RefreshingHttpsJwks implements Initable, Closeable {
private final AtomicBoolean refreshInProgressFlag = new AtomicBoolean(false);
+ /**
+ * {@link HttpsJwks} does the actual work of contacting the OAuth/OIDC endpoint to get the
+ * JWKS. In some cases, the call to {@link HttpsJwks#getJsonWebKeys()} will trigger a call
+ * to {@link HttpsJwks#refresh()} which will block the current thread in network I/O. We cache
+ * the JWKS ourselves (see {@link #jsonWebKeys}) to avoid the network I/O.
+ *
+ * We want to be very careful where we use the {@link HttpsJwks} instance so that we don't
+ * perform any operation (directly or indirectly) that could cause blocking. This is because
+ * the JWKS logic is part of the larger authentication logic which operates on Kafka's network
+ * thread. It's OK to execute {@link HttpsJwks#getJsonWebKeys()} (which calls
+ * {@link HttpsJwks#refresh()}) from within {@link #configure(Map, String, List)} as that method is called only at
+ * startup, and we can afford the blocking hit there.
+ */
+
+ private HttpsJwks httpsJwks;
+
+ private Optional sslResource = Optional.empty();
+
+ private long refreshMs;
+
+ private long refreshRetryBackoffMs;
+
+ private long refreshRetryBackoffMaxMs;
+
/**
* As mentioned in the comments for {@link #httpsJwks}, we cache the JWKS ourselves so that
* we can return the list immediately without any network I/O. They are only cached within
@@ -119,28 +132,14 @@ public final class RefreshingHttpsJwks implements Initable, Closeable {
private List jsonWebKeys;
- private boolean isInitialized;
-
- /**
- * Creates a RefreshingHttpsJwks
. It should only be used for testing to pass in a mock executor
- * service. Otherwise the constructor below should be used.
- */
+ protected boolean isInitialized;
- // VisibleForTesting
- RefreshingHttpsJwks(Time time,
- HttpsJwks httpsJwks,
- long refreshMs,
- long refreshRetryBackoffMs,
- long refreshRetryBackoffMaxMs,
- ScheduledExecutorService executorService) {
- if (refreshMs <= 0)
- throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+ public RefreshingHttpsJwks(Time time) {
+ this(time, Executors.newSingleThreadScheduledExecutor());
+ }
- this.httpsJwks = httpsJwks;
+ public RefreshingHttpsJwks(Time time, ScheduledExecutorService executorService) {
this.time = time;
- this.refreshMs = refreshMs;
- this.refreshRetryBackoffMs = refreshRetryBackoffMs;
- this.refreshRetryBackoffMaxMs = refreshRetryBackoffMaxMs;
this.executorService = executorService;
this.missingKeyIds = new LinkedHashMap<>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
@Override
@@ -150,55 +149,26 @@ protected boolean removeEldestEntry(Map.Entry eldest) {
};
}
- /**
- * Creates a RefreshingHttpsJwks
that will be used by the
- * {@link RefreshingHttpsJwksVerificationKeyResolver} to resolve new key IDs in JWTs.
- *
- * @param time {@link Time} instance
- * @param httpsJwks {@link HttpsJwks} instance from which to retrieve the JWKS
- * based on the OAuth/OIDC standard
- * @param refreshMs The number of milliseconds between refresh passes to connect
- * to the OAuth/OIDC JWKS endpoint to retrieve the latest set
- * @param refreshRetryBackoffMs Time for delay after initial failed attempt to retrieve JWKS
- * @param refreshRetryBackoffMaxMs Maximum time to retrieve JWKS
- */
-
- public RefreshingHttpsJwks(Time time,
- HttpsJwks httpsJwks,
- long refreshMs,
- long refreshRetryBackoffMs,
- long refreshRetryBackoffMaxMs) {
- this(time, httpsJwks, refreshMs, refreshRetryBackoffMs, refreshRetryBackoffMaxMs, Executors.newSingleThreadScheduledExecutor());
- }
-
@Override
- public void init() throws IOException {
+ public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
try {
log.debug("init started");
- List localJWKs;
+ ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
+ URL jwksEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
- try {
- localJWKs = httpsJwks.getJsonWebKeys();
- } catch (JoseException e) {
- throw new IOException("Could not refresh JWKS", e);
- }
+ JaasOptionsUtils jou = new JaasOptionsUtils(saslMechanism, jaasConfigEntries);
- try {
- refreshLock.writeLock().lock();
- jsonWebKeys = Collections.unmodifiableList(localJWKs);
- } finally {
- refreshLock.writeLock().unlock();
- }
+ HttpsJwks httpsJwks = new HttpsJwks(jwksEndpointUrl.toString());
+ sslResource = jou.maybeCreateSslResource(jwksEndpointUrl);
- // Since we just grabbed the keys (which will have invoked a HttpsJwks.refresh()
- // internally), we can delay our first invocation by refreshMs.
- //
- // Note: we refer to this as a _scheduled_ refresh.
- executorService.scheduleAtFixedRate(this::refresh,
- refreshMs,
- refreshMs,
- TimeUnit.MILLISECONDS);
+ sslResource.ifPresent(sslResource -> {
+ Get get = new Get();
+ get.setSslSocketFactory(sslResource.sslSocketFactory());
+ httpsJwks.setSimpleHttpGet(get);
+ });
+
+ configure(httpsJwks, configs, saslMechanism);
log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
} finally {
@@ -208,24 +178,56 @@ public void init() throws IOException {
}
}
+ void configure(HttpsJwks httpsJwks, Map configs, String saslMechanism) {
+ ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
+ refreshMs = cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, true, 0L);
+ refreshRetryBackoffMs = cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS);
+ refreshRetryBackoffMaxMs = cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS);
+
+ this.httpsJwks = httpsJwks;
+ this.httpsJwks.setDefaultCacheDuration(refreshMs);
+
+ List localJWKs;
+
+ try {
+ localJWKs = httpsJwks.getJsonWebKeys();
+ } catch (Exception e) {
+ throw new KafkaException("Could not refresh JWKS", e);
+ }
+
+ try {
+ refreshLock.writeLock().lock();
+ jsonWebKeys = Collections.unmodifiableList(localJWKs);
+ } finally {
+ refreshLock.writeLock().unlock();
+ }
+
+ // Since we just grabbed the keys (which will have invoked a HttpsJwks.refresh()
+ // internally), we can delay our first invocation by refreshMs.
+ //
+ // Note: we refer to this as a _scheduled_ refresh.
+ executorService.scheduleAtFixedRate(
+ this::refresh,
+ refreshMs,
+ refreshMs,
+ TimeUnit.MILLISECONDS
+ );
+ }
+
@Override
public void close() {
try {
- log.debug("close started");
+ log.debug("JWKS validation key refresh thread shutting down");
+ executorService.shutdown();
- try {
- log.debug("JWKS validation key refresh thread shutting down");
- executorService.shutdown();
-
- if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
- log.warn("JWKS validation key refresh thread termination did not end after {} {}",
- SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
- }
- } catch (InterruptedException e) {
- log.warn("JWKS validation key refresh thread error during close", e);
+ if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+ log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+ SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
}
+ } catch (Exception e) {
+ log.warn("JWKS validation key refresh thread error during close", e);
} finally {
- log.debug("close completed");
+ sslResource.ifPresent(sslResource -> Utils.closeQuietly(sslResource, "sslResource"));
}
}
@@ -257,6 +259,17 @@ public String getLocation() {
return httpsJwks.getLocation();
}
+ /**
+ * Call the actual refresh implementation that will more than likely issue HTTP(S) calls over the network.
+ */
+ private List refreshJsonWebKeys() throws JoseException, IOException {
+ log.debug("JWKS validation key calling refresh of {} starting", httpsJwks.getLocation());
+ httpsJwks.refresh();
+ List jwks = httpsJwks.getJsonWebKeys();
+ log.debug("JWKS validation key refresh of {} complete", httpsJwks.getLocation());
+ return jwks;
+ }
+
/**
*
* refresh
is an internal method that will refresh the JWKS cache and is
@@ -269,13 +282,12 @@ public String getLocation() {
*
*
*
- * The scheduled refresh is scheduled in {@link #init()} and runs every
+ * The scheduled refresh is scheduled in {@link #configure(Map, String, List)} and runs every
* {@link #refreshMs} milliseconds. An expedited refresh is performed when an
* incoming JWT refers to a key ID that isn't in our JWKS cache ({@link #jsonWebKeys})
* and we try to perform a refresh sooner than the next scheduled refresh.
*
*/
-
private void refresh() {
if (!refreshInProgressFlag.compareAndSet(false, true)) {
log.debug("OAuth JWKS refresh is already in progress; ignoring concurrent refresh");
@@ -287,13 +299,7 @@ private void refresh() {
Retry> retry = new Retry<>(refreshRetryBackoffMs, refreshRetryBackoffMaxMs);
List localJWKs = retry.execute(() -> {
try {
- log.debug("JWKS validation key calling refresh of {} starting", httpsJwks.getLocation());
- // Call the *actual* refresh implementation that will more than likely issue
- // HTTP(S) calls over the network.
- httpsJwks.refresh();
- List jwks = httpsJwks.getJsonWebKeys();
- log.debug("JWKS validation key refresh of {} complete", httpsJwks.getLocation());
- return jwks;
+ return refreshJsonWebKeys();
} catch (Exception e) {
throw new ExecutionException(e);
}
@@ -321,7 +327,7 @@ private void refresh() {
/**
*
* maybeExpediteRefresh
is a public method that will trigger a refresh of
- * the JWKS cache if all of the following conditions are met:
+ * the JWKS cache if all the following conditions are met:
*
*
* - The given
keyId
parameter is <e; the
@@ -336,7 +342,6 @@ private void refresh() {
* @param keyId JWT key ID
* @return true
if an expedited refresh was scheduled, false
otherwise
*/
-
public boolean maybeExpediteRefresh(String keyId) {
if (keyId.length() > MISSING_KEY_ID_MAX_KEY_LENGTH) {
// Although there's no limit on the length of the key ID, they're generally
@@ -375,5 +380,4 @@ public boolean maybeExpediteRefresh(String keyId) {
}
}
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java
index 52d0c6c39785f..f954436d1dcee 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java
@@ -14,9 +14,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+
import org.jose4j.jwk.HttpsJwks;
import org.jose4j.jwk.JsonWebKey;
import org.jose4j.jwk.VerificationJwkSelector;
@@ -31,6 +33,9 @@
import java.io.IOException;
import java.security.Key;
import java.util.List;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
/**
* RefreshingHttpsJwksVerificationKeyResolver
is a
@@ -80,44 +85,33 @@
* @see RefreshingHttpsJwks
* @see HttpsJwks
*/
-
public class RefreshingHttpsJwksVerificationKeyResolver implements CloseableVerificationKeyResolver {
private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwksVerificationKeyResolver.class);
- private final RefreshingHttpsJwks refreshingHttpsJwks;
+ private final Time time;
private final VerificationJwkSelector verificationJwkSelector;
+ private RefreshingHttpsJwks refreshingHttpsJwks;
+
private boolean isInitialized;
- public RefreshingHttpsJwksVerificationKeyResolver(RefreshingHttpsJwks refreshingHttpsJwks) {
- this.refreshingHttpsJwks = refreshingHttpsJwks;
+ public RefreshingHttpsJwksVerificationKeyResolver(Time time) {
+ this.time = time;
this.verificationJwkSelector = new VerificationJwkSelector();
}
@Override
- public void init() throws IOException {
- try {
- log.debug("init started");
-
- refreshingHttpsJwks.init();
- } finally {
- isInitialized = true;
-
- log.debug("init completed");
- }
+ public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
+ refreshingHttpsJwks = new RefreshingHttpsJwks(time);
+ refreshingHttpsJwks.configure(configs, saslMechanism, jaasConfigEntries);
+ isInitialized = true;
}
@Override
public void close() {
- try {
- log.debug("close started");
-
- refreshingHttpsJwks.close();
- } finally {
- log.debug("close completed");
- }
+ Utils.closeQuietly(refreshingHttpsJwks, "refreshingHttpsJwks");
}
@Override
@@ -148,5 +142,4 @@ public Key resolveKey(JsonWebSignature jws, List nestingContex
throw new UnresolvableKeyException(sb, e);
}
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retry.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retry.java
index 0da92e4fc7f72..7f219ade62a1b 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retry.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retry.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.utils.Time;
@@ -30,7 +29,6 @@
*
* @param Result type
*/
-
public class Retry {
private static final Logger log = LoggerFactory.getLogger(Retry.class);
@@ -104,5 +102,4 @@ public R execute(Retryable retryable) throws ExecutionException {
throw error;
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retryable.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retryable.java
index 46752f3ea6020..11e513c1cc4d4 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retryable.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retryable.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import java.util.concurrent.ExecutionException;
@@ -27,19 +26,16 @@
* @see Retry
* @see UnretryableException
*/
-
public interface Retryable {
/**
* Perform the operation and return the data from the response.
*
* @return Return response data, formatted in the given data type
- *
* @throws ExecutionException Thrown on errors connecting, writing, reading, timeouts, etc.
* that can likely be tried again
* @throws UnretryableException Thrown on errors that we can determine should not be tried again
*/
R call() throws ExecutionException, UnretryableException;
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java
index f45865fa63848..a1b909e156fac 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java
@@ -14,14 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.oauthbearer.InvalidJwtException;
+
/**
* SerializedJwt provides a modicum of structure and validation around a JWT's serialized form by
* splitting and making the three sections (header, payload, and signature) available to the user.
*/
-
public class SerializedJwt {
private final String token;
@@ -39,12 +39,12 @@ public SerializedJwt(String token) {
token = token.trim();
if (token.isEmpty())
- throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)");
+ throw new InvalidJwtException("Malformed JWT provided; expected three sections (header, payload, and signature)");
String[] splits = token.split("\\.");
if (splits.length != 3)
- throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)");
+ throw new InvalidJwtException("Malformed JWT provided; expected three sections (header, payload, and signature)");
this.token = token.trim();
this.header = validateSection(splits[0]);
@@ -92,13 +92,12 @@ public String getSignature() {
return signature;
}
- private String validateSection(String section) throws ValidateException {
+ private String validateSection(String section) throws InvalidJwtException {
section = section.trim();
if (section.isEmpty())
- throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)");
+ throw new InvalidJwtException("Malformed JWT provided; expected three sections (header, payload, and signature)");
return section;
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SslResource.java
similarity index 51%
rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java
rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SslResource.java
index c145cf7596959..d3fe500922649 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SslResource.java
@@ -14,44 +14,38 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.ssl.SslFactory;
import org.apache.kafka.common.utils.Utils;
+import java.io.Closeable;
import java.io.IOException;
-import java.nio.file.Path;
+
+import javax.net.ssl.SSLSocketFactory;
/**
- * FileTokenRetriever
is an {@link AccessTokenRetriever} that will load the contents,
- * interpreting them as a JWT access key in the serialized form.
- *
- * @see AccessTokenRetriever
+ * {@code SslResource} couples the {@link SslFactory} and {@link SSLSocketFactory} so that
+ * {@link #sslFactory} can be properly {@link SslFactory#close() closed} during closing of the overall
+ * OAuth login/validation module.
*/
+public class SslResource implements Closeable {
-public class FileTokenRetriever implements AccessTokenRetriever {
-
- private final Path accessTokenFile;
+ private final SslFactory sslFactory;
- private String accessToken;
+ private final SSLSocketFactory sslSocketFactory;
- public FileTokenRetriever(Path accessTokenFile) {
- this.accessTokenFile = accessTokenFile;
+ public SslResource(SslFactory sslFactory, SSLSocketFactory sslSocketFactory) {
+ this.sslFactory = sslFactory;
+ this.sslSocketFactory = sslSocketFactory;
}
- @Override
- public void init() throws IOException {
- this.accessToken = Utils.readFileAsString(accessTokenFile.toFile().getPath());
- // always non-null; to remove any newline chars or backend will report err
- this.accessToken = this.accessToken.trim();
+ public SSLSocketFactory sslSocketFactory() {
+ return sslSocketFactory;
}
@Override
- public String retrieve() throws IOException {
- if (accessToken == null)
- throw new IllegalStateException("Access token is null; please call init() first");
-
- return accessToken;
+ public void close() throws IOException {
+ Utils.closeQuietly(sslFactory, "sslFactory");
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/UnretryableException.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/UnretryableException.java
index a53ae6ec115c7..6e2494b2d168a 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/UnretryableException.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/UnretryableException.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.KafkaException;
@@ -24,5 +23,4 @@ public class UnretryableException extends KafkaException {
public UnretryableException(Throwable cause) {
super(cause);
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidator.java
deleted file mode 100644
index c7ae8edae9d93..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidator.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * 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.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
-
-import org.jose4j.jwt.JwtClaims;
-import org.jose4j.jwt.MalformedClaimException;
-import org.jose4j.jwt.NumericDate;
-import org.jose4j.jwt.ReservedClaimNames;
-import org.jose4j.jwt.consumer.InvalidJwtException;
-import org.jose4j.jwt.consumer.JwtConsumer;
-import org.jose4j.jwt.consumer.JwtConsumerBuilder;
-import org.jose4j.jwt.consumer.JwtContext;
-import org.jose4j.keys.resolvers.VerificationKeyResolver;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Set;
-
-import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
-
-/**
- * ValidatorAccessTokenValidator is an implementation of {@link AccessTokenValidator} that is used
- * by the broker to perform more extensive validation of the JWT access token that is received
- * from the client, but ultimately from posting the client credentials to the OAuth/OIDC provider's
- * token endpoint.
- *
- * The validation steps performed (primary by the jose4j library) are:
- *
- *
- * -
- * Basic structural validation of the
b64token
value as defined in
- * RFC 6750 Section 2.1
- *
- * - Basic conversion of the token into an in-memory data structure
- * -
- * Presence of scope,
exp
, subject, iss
, and
- * iat
claims
- *
- * -
- * Signature matching validation against the
kid
and those provided by
- * the OAuth/OIDC provider's JWKS
- *
- *
- */
-
-public class ValidatorAccessTokenValidator implements AccessTokenValidator {
-
- private static final Logger log = LoggerFactory.getLogger(ValidatorAccessTokenValidator.class);
-
- private final JwtConsumer jwtConsumer;
-
- private final String scopeClaimName;
-
- private final String subClaimName;
-
- /**
- * Creates a new ValidatorAccessTokenValidator that will be used by the broker for more
- * thorough validation of the JWT.
- *
- * @param clockSkew The optional value (in seconds) to allow for differences
- * between the time of the OAuth/OIDC identity provider and
- * the broker. If null
is provided, the broker
- * and the OAUth/OIDC identity provider are assumed to have
- * very close clock settings.
- * @param expectedAudiences The (optional) set the broker will use to verify that
- * the JWT was issued for one of the expected audiences.
- * The JWT will be inspected for the standard OAuth
- * aud
claim and if this value is set, the
- * broker will match the value from JWT's aud
- * claim to see if there is an exact match. If there is no
- * match, the broker will reject the JWT and authentication
- * will fail. May be null
to not perform any
- * check to verify the JWT's aud
claim matches any
- * fixed set of known/expected audiences.
- * @param expectedIssuer The (optional) value for the broker to use to verify that
- * the JWT was created by the expected issuer. The JWT will
- * be inspected for the standard OAuth iss
claim
- * and if this value is set, the broker will match it
- * exactly against what is in the JWT's iss
- * claim. If there is no match, the broker will reject the JWT
- * and authentication will fail. May be null
to not
- * perform any check to verify the JWT's iss
claim
- * matches a specific issuer.
- * @param verificationKeyResolver jose4j-based {@link VerificationKeyResolver} that is used
- * to validate the signature matches the contents of the header
- * and payload
- * @param scopeClaimName Name of the scope claim to use; must be non-null
- * @param subClaimName Name of the subject claim to use; must be
- * non-null
- *
- * @see JwtConsumerBuilder
- * @see JwtConsumer
- * @see VerificationKeyResolver
- */
-
- public ValidatorAccessTokenValidator(Integer clockSkew,
- Set expectedAudiences,
- String expectedIssuer,
- VerificationKeyResolver verificationKeyResolver,
- String scopeClaimName,
- String subClaimName) {
- final JwtConsumerBuilder jwtConsumerBuilder = new JwtConsumerBuilder();
-
- if (clockSkew != null)
- jwtConsumerBuilder.setAllowedClockSkewInSeconds(clockSkew);
-
- if (expectedAudiences != null && !expectedAudiences.isEmpty())
- jwtConsumerBuilder.setExpectedAudience(expectedAudiences.toArray(new String[0]));
-
- if (expectedIssuer != null)
- jwtConsumerBuilder.setExpectedIssuer(expectedIssuer);
-
- this.jwtConsumer = jwtConsumerBuilder
- .setJwsAlgorithmConstraints(DISALLOW_NONE)
- .setRequireExpirationTime()
- .setRequireIssuedAt()
- .setVerificationKeyResolver(verificationKeyResolver)
- .build();
- this.scopeClaimName = scopeClaimName;
- this.subClaimName = subClaimName;
- }
-
- /**
- * Accepts an OAuth JWT access token in base-64 encoded format, validates, and returns an
- * OAuthBearerToken.
- *
- * @param accessToken Non-null
JWT access token
- * @return {@link OAuthBearerToken}
- * @throws ValidateException Thrown on errors performing validation of given token
- */
-
- @SuppressWarnings("unchecked")
- public OAuthBearerToken validate(String accessToken) throws ValidateException {
- SerializedJwt serializedJwt = new SerializedJwt(accessToken);
-
- JwtContext jwt;
-
- try {
- jwt = jwtConsumer.process(serializedJwt.getToken());
- } catch (InvalidJwtException e) {
- throw new ValidateException(String.format("Could not validate the access token: %s", e.getMessage()), e);
- }
-
- JwtClaims claims = jwt.getJwtClaims();
-
- Object scopeRaw = getClaim(() -> claims.getClaimValue(scopeClaimName), scopeClaimName);
- Collection scopeRawCollection;
-
- if (scopeRaw instanceof String)
- scopeRawCollection = Collections.singletonList((String) scopeRaw);
- else if (scopeRaw instanceof Collection)
- scopeRawCollection = (Collection) scopeRaw;
- else
- scopeRawCollection = Collections.emptySet();
-
- NumericDate expirationRaw = getClaim(claims::getExpirationTime, ReservedClaimNames.EXPIRATION_TIME);
- String subRaw = getClaim(() -> claims.getStringClaimValue(subClaimName), subClaimName);
- NumericDate issuedAtRaw = getClaim(claims::getIssuedAt, ReservedClaimNames.ISSUED_AT);
-
- Set scopes = ClaimValidationUtils.validateScopes(scopeClaimName, scopeRawCollection);
- long expiration = ClaimValidationUtils.validateExpiration(ReservedClaimNames.EXPIRATION_TIME,
- expirationRaw != null ? expirationRaw.getValueInMillis() : null);
- String sub = ClaimValidationUtils.validateSubject(subClaimName, subRaw);
- Long issuedAt = ClaimValidationUtils.validateIssuedAt(ReservedClaimNames.ISSUED_AT,
- issuedAtRaw != null ? issuedAtRaw.getValueInMillis() : null);
-
- return new BasicOAuthBearerToken(accessToken,
- scopes,
- expiration,
- sub,
- issuedAt);
- }
-
- private T getClaim(ClaimSupplier supplier, String claimName) throws ValidateException {
- try {
- T value = supplier.get();
- log.debug("getClaim - {}: {}", claimName, value);
- return value;
- } catch (MalformedClaimException e) {
- throw new ValidateException(String.format("Could not extract the '%s' claim from the access token", claimName), e);
- }
- }
-
- public interface ClaimSupplier {
-
- T get() throws MalformedClaimException;
-
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java
deleted file mode 100644
index 0422045fc029d..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.utils.Time;
-
-import org.jose4j.http.Get;
-import org.jose4j.jwk.HttpsJwks;
-
-import java.net.URL;
-import java.nio.file.Path;
-import java.util.Locale;
-import java.util.Map;
-
-import javax.net.ssl.SSLSocketFactory;
-
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
-
-public class VerificationKeyResolverFactory {
-
- /**
- * Create an {@link AccessTokenRetriever} from the given
- * {@link org.apache.kafka.common.config.SaslConfigs}.
- *
- * Note: the returned CloseableVerificationKeyResolver
is not
- * initialized here and must be done by the caller.
- *
- * Primarily exposed here for unit testing.
- *
- * @param configs SASL configuration
- *
- * @return Non-null
{@link CloseableVerificationKeyResolver}
- */
- public static CloseableVerificationKeyResolver create(Map configs,
- Map jaasConfig) {
- return create(configs, null, jaasConfig);
- }
-
- public static CloseableVerificationKeyResolver create(Map configs,
- String saslMechanism,
- Map jaasConfig) {
- ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
- URL jwksEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
-
- if (jwksEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file")) {
- Path p = cu.validateFile(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
- return new JwksFileVerificationKeyResolver(p);
- } else {
- long refreshIntervalMs = cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, true, 0L);
- JaasOptionsUtils jou = new JaasOptionsUtils(jaasConfig);
- SSLSocketFactory sslSocketFactory = null;
-
- if (jou.shouldCreateSSLSocketFactory(jwksEndpointUrl))
- sslSocketFactory = jou.createSSLSocketFactory();
-
- HttpsJwks httpsJwks = new HttpsJwks(jwksEndpointUrl.toString());
- httpsJwks.setDefaultCacheDuration(refreshIntervalMs);
-
- if (sslSocketFactory != null) {
- Get get = new Get();
- get.setSslSocketFactory(sslSocketFactory);
- httpsJwks.setSimpleHttpGet(get);
- }
-
- RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(Time.SYSTEM,
- httpsJwks,
- refreshIntervalMs,
- cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS),
- cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS));
- return new RefreshingHttpsJwksVerificationKeyResolver(refreshingHttpsJwks);
- }
- }
-
-}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/AccessTokenValidatorTest.java
similarity index 70%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorTest.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/AccessTokenValidatorTest.java
index 0adaf34bbbeea..14747317f3c99 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/AccessTokenValidatorTest.java
@@ -14,15 +14,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+package org.apache.kafka.common.security.oauthbearer;
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
+import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder;
+import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest;
import org.jose4j.jws.AlgorithmIdentifiers;
import org.jose4j.jwx.HeaderParameterNames;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInstance;
import org.junit.jupiter.api.TestInstance.Lifecycle;
+import java.util.List;
+
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertThrows;
@TestInstance(Lifecycle.PER_CLASS)
@@ -32,31 +36,33 @@ public abstract class AccessTokenValidatorTest extends OAuthBearerTest {
protected AccessTokenValidator createAccessTokenValidator() throws Exception {
AccessTokenBuilder builder = new AccessTokenBuilder();
- return createAccessTokenValidator(builder);
+ AccessTokenValidator validator = createAccessTokenValidator(builder);
+ validator.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
+ return validator;
}
@Test
public void testNull() throws Exception {
AccessTokenValidator validator = createAccessTokenValidator();
- assertThrowsWithMessage(ValidateException.class, () -> validator.validate(null), "Malformed JWT provided; expected three sections (header, payload, and signature)");
+ assertThrowsWithMessage(InvalidJwtException.class, () -> validator.validate(null), "Malformed JWT provided; expected three sections (header, payload, and signature)");
}
@Test
public void testEmptyString() throws Exception {
AccessTokenValidator validator = createAccessTokenValidator();
- assertThrowsWithMessage(ValidateException.class, () -> validator.validate(""), "Malformed JWT provided; expected three sections (header, payload, and signature)");
+ assertThrowsWithMessage(InvalidJwtException.class, () -> validator.validate(""), "Malformed JWT provided; expected three sections (header, payload, and signature)");
}
@Test
public void testWhitespace() throws Exception {
AccessTokenValidator validator = createAccessTokenValidator();
- assertThrowsWithMessage(ValidateException.class, () -> validator.validate(" "), "Malformed JWT provided; expected three sections (header, payload, and signature)");
+ assertThrowsWithMessage(InvalidJwtException.class, () -> validator.validate(" "), "Malformed JWT provided; expected three sections (header, payload, and signature)");
}
@Test
public void testEmptySections() throws Exception {
AccessTokenValidator validator = createAccessTokenValidator();
- assertThrowsWithMessage(ValidateException.class, () -> validator.validate(".."), "Malformed JWT provided; expected three sections (header, payload, and signature)");
+ assertThrowsWithMessage(InvalidJwtException.class, () -> validator.validate(".."), "Malformed JWT provided; expected three sections (header, payload, and signature)");
}
@Test
@@ -66,7 +72,7 @@ public void testMissingHeader() throws Exception {
String payload = createBase64JsonJwtSection(node -> { });
String signature = "";
String accessToken = String.format("%s.%s.%s", header, payload, signature);
- assertThrows(ValidateException.class, () -> validator.validate(accessToken));
+ assertThrows(InvalidJwtException.class, () -> validator.validate(accessToken));
}
@Test
@@ -76,7 +82,7 @@ public void testMissingPayload() throws Exception {
String payload = "";
String signature = "";
String accessToken = String.format("%s.%s.%s", header, payload, signature);
- assertThrows(ValidateException.class, () -> validator.validate(accessToken));
+ assertThrows(InvalidJwtException.class, () -> validator.validate(accessToken));
}
@Test
@@ -86,7 +92,6 @@ public void testMissingSignature() throws Exception {
String payload = createBase64JsonJwtSection(node -> { });
String signature = "";
String accessToken = String.format("%s.%s.%s", header, payload, signature);
- assertThrows(ValidateException.class, () -> validator.validate(accessToken));
+ assertThrows(InvalidJwtException.class, () -> validator.validate(accessToken));
}
-
}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenRetrieverTest.java
similarity index 60%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenRetrieverTest.java
index 3e85f7b0ce4fa..304667ea2f263 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenRetrieverTest.java
@@ -14,29 +14,25 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
+package org.apache.kafka.common.security.oauthbearer;
import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.Arguments;
-import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.util.Collections;
+import java.util.List;
import java.util.Map;
-import java.util.stream.Stream;
-import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE;
import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertEquals;
-public class AccessTokenRetrieverFactoryTest extends OAuthBearerTest {
+public class DefaultAccessTokenRetrieverTest extends OAuthBearerTest {
@AfterEach
public void tearDown() throws Exception {
@@ -52,10 +48,9 @@ public void testConfigureRefreshingFileAccessTokenRetriever() throws Exception {
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.toURI().toString());
Map configs = Collections.singletonMap(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString());
- Map jaasConfig = Collections.emptyMap();
- try (AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, jaasConfig)) {
- accessTokenRetriever.init();
+ try (AccessTokenRetriever accessTokenRetriever = new DefaultAccessTokenRetriever()) {
+ accessTokenRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of());
assertEquals(expected, accessTokenRetriever.retrieve());
}
}
@@ -66,8 +61,10 @@ public void testConfigureRefreshingFileAccessTokenRetrieverWithInvalidDirectory(
String file = new File("/tmp/this-directory-does-not-exist/foo.json").toURI().toString();
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file);
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, file);
- Map jaasConfig = Collections.emptyMap();
- assertThrowsWithMessage(ConfigException.class, () -> AccessTokenRetrieverFactory.create(configs, jaasConfig), "that doesn't exist");
+
+ try (AccessTokenRetriever accessTokenRetriever = new DefaultAccessTokenRetriever()) {
+ assertThrowsWithMessage(ConfigException.class, () -> accessTokenRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), "that doesn't exist");
+ }
}
@Test
@@ -77,8 +74,10 @@ public void testConfigureRefreshingFileAccessTokenRetrieverWithInvalidFile() thr
File accessTokenFile = new File(tmpDir, "this-file-does-not-exist.json");
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.toURI().toString());
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString());
- Map jaasConfig = Collections.emptyMap();
- assertThrowsWithMessage(ConfigException.class, () -> AccessTokenRetrieverFactory.create(configs, jaasConfig), "that doesn't exist");
+
+ try (AccessTokenRetriever accessTokenRetriever = new DefaultAccessTokenRetriever()) {
+ assertThrowsWithMessage(ConfigException.class, () -> accessTokenRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), "that doesn't exist");
+ }
}
@Test
@@ -87,25 +86,9 @@ public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception {
File tmpDir = createTempDir("not_allowed");
File accessTokenFile = new File(tmpDir, "not_allowed.json");
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString());
- assertThrowsWithMessage(ConfigException.class, () -> AccessTokenRetrieverFactory.create(configs, Collections.emptyMap()),
- ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG);
- }
-
- @ParameterizedTest
- @MethodSource("urlencodeHeaderSupplier")
- public void testUrlencodeHeader(Map configs, boolean expectedValue) {
- ConfigurationUtils cu = new ConfigurationUtils(configs);
- boolean actualValue = AccessTokenRetrieverFactory.validateUrlencodeHeader(cu);
- assertEquals(expectedValue, actualValue);
- }
- private static Stream urlencodeHeaderSupplier() {
- return Stream.of(
- Arguments.of(Collections.emptyMap(), DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE),
- Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, null), DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE),
- Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, true), true),
- Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, false), false)
- );
+ try (AccessTokenRetriever accessTokenRetriever = new DefaultAccessTokenRetriever()) {
+ assertThrowsWithMessage(ConfigException.class, () -> accessTokenRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG);
+ }
}
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java
index 5b1b2976662b6..91e9fbd4619d3 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java
@@ -17,17 +17,12 @@
package org.apache.kafka.common.security.oauthbearer;
+import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenRetriever;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidator;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidatorFactory;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.FileTokenRetriever;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpAccessTokenRetriever;
import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest;
-import org.apache.kafka.common.utils.Utils;
import org.jose4j.jws.AlgorithmIdentifiers;
import org.junit.jupiter.api.AfterEach;
@@ -35,29 +30,31 @@
import java.io.File;
import java.io.IOException;
-import java.util.Base64;
-import java.util.Calendar;
import java.util.Collections;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
-import java.util.TimeZone;
import javax.security.auth.callback.Callback;
import javax.security.auth.callback.UnsupportedCallbackException;
import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest {
+
@AfterEach
public void tearDown() throws Exception {
System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG);
@@ -65,18 +62,16 @@ public void tearDown() throws Exception {
@Test
public void testHandleTokenCallback() throws Exception {
- Map configs = getSaslConfigs();
AccessTokenBuilder builder = new AccessTokenBuilder()
.jwk(createRsaJwk())
.alg(AlgorithmIdentifiers.RSA_USING_SHA256);
String accessToken = builder.build();
- AccessTokenRetriever accessTokenRetriever = () -> accessToken;
-
- OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs);
+ AccessTokenRetriever accessTokenRetriever = mock(AccessTokenRetriever.class);
+ when(accessTokenRetriever.retrieve()).thenReturn(accessToken);
- try {
+ try (OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever)) {
OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
- handler.handle(new Callback[] {callback});
+ handler.handle(new Callback[]{callback});
assertNotNull(callback.token());
OAuthBearerToken token = callback.token();
@@ -84,8 +79,6 @@ public void testHandleTokenCallback() throws Exception {
assertEquals(builder.subject(), token.principalName());
assertEquals(builder.expirationSeconds() * 1000, token.lifetimeMs());
assertEquals(builder.issuedAtSeconds() * 1000, token.startTimeMs());
- } finally {
- handler.close();
}
}
@@ -95,8 +88,8 @@ public void testHandleSaslExtensionsCallback() throws Exception {
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, "http://www.example.com");
Map jaasConfig = new HashMap<>();
- jaasConfig.put(CLIENT_ID_CONFIG, "an ID");
- jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret");
+ jaasConfig.put("clientId", "an ID");
+ jaasConfig.put("clientSecret", "a secret");
jaasConfig.put("extension_foo", "1");
jaasConfig.put("extension_bar", 2);
jaasConfig.put("EXTENSION_baz", "3");
@@ -125,8 +118,8 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() {
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, "http://www.example.com");
Map jaasConfig = new HashMap<>();
- jaasConfig.put(CLIENT_ID_CONFIG, "an ID");
- jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret");
+ jaasConfig.put("clientId", "an ID");
+ jaasConfig.put("clientSecret", "a secret");
jaasConfig.put(illegalKey, "this key isn't allowed per OAuthBearerClientInitialResponse.validateExtensions");
configureHandler(handler, configs, jaasConfig);
@@ -141,12 +134,38 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() {
}
@Test
- public void testInvalidCallbackGeneratesUnsupportedCallbackException() {
- Map configs = getSaslConfigs();
+ public void testConfigureThrowsExceptionOnAccessTokenValidatorConfigure() {
+ try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+ AccessTokenRetriever accessTokenRetriever = mock(AccessTokenRetriever.class);
+ AccessTokenValidator accessTokenValidator = mock(AccessTokenValidator.class)) {
+
+ doThrow(new KafkaException("Forced failure")).when(accessTokenValidator).configure(any(), any(), any());
+
+ assertThrows(
+ KafkaException.class,
+ () -> handler.configure(accessTokenRetriever, accessTokenValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of())
+ );
+ }
+ }
+
+ @Test
+ public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() {
+ try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+ AccessTokenRetriever accessTokenRetriever = mock(AccessTokenRetriever.class)) {
+ AccessTokenValidator accessTokenValidator = mock(AccessTokenValidator.class);
+ doThrow(new KafkaException("Forced failure")).when(accessTokenValidator).close();
+ handler.configure(accessTokenRetriever, accessTokenValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
+ assertDoesNotThrow(handler::close);
+ }
+ }
+
+ @Test
+ public void testInvalidCallbackGeneratesUnsupportedCallbackException() throws IOException {
OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- AccessTokenRetriever accessTokenRetriever = () -> "foo";
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
- handler.init(accessTokenRetriever, accessTokenValidator);
+ AccessTokenRetriever accessTokenRetriever = mock(AccessTokenRetriever.class);
+ when(accessTokenRetriever.retrieve()).thenReturn("foo");
+ AccessTokenValidator accessTokenValidator = new ClientAccessTokenValidator();
+ handler.configure(accessTokenRetriever, accessTokenValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
try {
Callback unsupportedCallback = new Callback() { };
@@ -165,53 +184,25 @@ public void testInvalidAccessToken() throws Exception {
}
@Test
- public void testMissingAccessToken() {
- AccessTokenRetriever accessTokenRetriever = () -> {
- throw new IOException("The token endpoint response access_token value must be non-null");
- };
- Map configs = getSaslConfigs();
- OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs);
+ public void testMissingAccessToken() throws IOException {
+ AccessTokenRetriever accessTokenRetriever = mock(AccessTokenRetriever.class);
+ when(accessTokenRetriever.retrieve()).thenThrow(new IOException("The token endpoint response access_token value must be non-null"));
- try {
+ try (OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever)) {
OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
- assertThrowsWithMessage(IOException.class,
+ assertThrowsWithMessage(
+ IOException.class,
() -> handler.handle(new Callback[]{callback}),
- "token endpoint response access_token value must be non-null");
- } finally {
- handler.close();
- }
- }
-
- @Test
- public void testFileTokenRetrieverHandlesNewline() throws IOException {
- Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
- long cur = cal.getTimeInMillis() / 1000;
- String exp = "" + (cur + 60 * 60); // 1 hour in future
- String iat = "" + cur;
-
- String expected = createAccessKey("{}", String.format("{\"exp\":%s, \"iat\":%s, \"sub\":\"subj\"}", exp, iat), "sign");
- String withNewline = expected + "\n";
-
- File tmpDir = createTempDir("access-token");
- File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", withNewline);
-
- Map configs = getSaslConfigs();
- OAuthBearerLoginCallbackHandler handler = createHandler(new FileTokenRetriever(accessTokenFile.toPath()), configs);
- OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
- try {
- handler.handle(new Callback[]{callback});
- assertEquals(callback.token().value(), expected);
- } catch (Exception e) {
- fail(e);
- } finally {
- handler.close();
+ "token endpoint response access_token value must be non-null"
+ );
}
}
@Test
public void testNotConfigured() {
- OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure or init method");
+ try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler()) {
+ assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure method");
+ }
}
@Test
@@ -224,9 +215,10 @@ public void testConfigureWithAccessTokenFile() throws Exception {
OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString());
- Map jaasConfigs = Collections.emptyMap();
- configureHandler(handler, configs, jaasConfigs);
- assertInstanceOf(FileTokenRetriever.class, handler.getAccessTokenRetriever());
+ Map jaasConfig = Collections.emptyMap();
+ configureHandler(handler, configs, jaasConfig);
+ assertInstanceOf(DefaultAccessTokenRetriever.class, handler.accessTokenRetriever);
+ assertInstanceOf(FileAccessTokenRetriever.class, ((DefaultAccessTokenRetriever) handler.accessTokenRetriever).delegate());
}
@Test
@@ -234,18 +226,19 @@ public void testConfigureWithAccessClientCredentials() {
OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, "http://www.example.com");
- Map jaasConfigs = new HashMap<>();
- jaasConfigs.put(CLIENT_ID_CONFIG, "an ID");
- jaasConfigs.put(CLIENT_SECRET_CONFIG, "a secret");
- configureHandler(handler, configs, jaasConfigs);
- assertInstanceOf(HttpAccessTokenRetriever.class, handler.getAccessTokenRetriever());
+ Map jaasConfig = new HashMap<>();
+ jaasConfig.put("clientId", "an ID");
+ jaasConfig.put("clientSecret", "a secret");
+ configureHandler(handler, configs, jaasConfig);
+ assertInstanceOf(DefaultAccessTokenRetriever.class, handler.accessTokenRetriever);
+ assertInstanceOf(HttpAccessTokenRetriever.class, ((DefaultAccessTokenRetriever) handler.accessTokenRetriever).delegate());
}
private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception {
- Map configs = getSaslConfigs();
- OAuthBearerLoginCallbackHandler handler = createHandler(() -> accessToken, configs);
+ AccessTokenRetriever accessTokenRetriever = mock(AccessTokenRetriever.class);
+ when(accessTokenRetriever.retrieve()).thenReturn(accessToken);
- try {
+ try (OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever)) {
OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
handler.handle(new Callback[]{callback});
@@ -255,24 +248,14 @@ private void testInvalidAccessToken(String accessToken, String expectedMessageSu
assertTrue(actualMessage.contains(expectedMessageSubstring), String.format(
"The error message \"%s\" didn't contain the expected substring \"%s\"",
actualMessage, expectedMessageSubstring));
- } finally {
- handler.close();
}
}
- private String createAccessKey(String header, String payload, String signature) {
- Base64.Encoder enc = Base64.getEncoder();
- header = enc.encodeToString(Utils.utf8(header));
- payload = enc.encodeToString(Utils.utf8(payload));
- signature = enc.encodeToString(Utils.utf8(signature));
- return String.format("%s.%s.%s", header, payload, signature);
- }
-
- private OAuthBearerLoginCallbackHandler createHandler(AccessTokenRetriever accessTokenRetriever, Map configs) {
+ protected OAuthBearerLoginCallbackHandler createHandler(AccessTokenRetriever accessTokenRetriever) {
+ Map configs = getSaslConfigs();
OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
- handler.init(accessTokenRetriever, accessTokenValidator);
+ AccessTokenValidator accessTokenValidator = new ClientAccessTokenValidator();
+ handler.configure(accessTokenRetriever, accessTokenValidator, configs, OAUTHBEARER_MECHANISM, List.of());
return handler;
}
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java
index d682a05ec11cc..d9ae8b3334b11 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java
@@ -18,23 +18,19 @@
package org.apache.kafka.common.security.oauthbearer;
import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidator;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidatorFactory;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver;
import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest;
-import org.apache.kafka.common.utils.Utils;
import org.jose4j.jws.AlgorithmIdentifiers;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
-import java.util.Base64;
import java.util.List;
import java.util.Map;
import javax.security.auth.callback.Callback;
import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@@ -53,9 +49,8 @@ public void testBasic() throws Exception {
String accessToken = builder.build();
Map configs = getSaslConfigs(SASL_OAUTHBEARER_EXPECTED_AUDIENCE, allAudiences);
- OAuthBearerValidatorCallbackHandler handler = createHandler(configs, builder);
- try {
+ try (OAuthBearerValidatorCallbackHandler handler = createHandler(configs, builder)) {
OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken);
handler.handle(new Callback[]{callback});
@@ -65,8 +60,6 @@ public void testBasic() throws Exception {
assertEquals(builder.subject(), token.principalName());
assertEquals(builder.expirationSeconds() * 1000, token.lifetimeMs());
assertEquals(builder.issuedAtSeconds() * 1000, token.startTimeMs());
- } finally {
- handler.close();
}
}
@@ -83,37 +76,22 @@ public void testInvalidAccessToken() throws Exception {
private void assertInvalidAccessTokenFails(String accessToken, String expectedMessageSubstring) throws Exception {
Map configs = getSaslConfigs();
- OAuthBearerValidatorCallbackHandler handler = createHandler(configs, new AccessTokenBuilder());
- try {
+ try (OAuthBearerValidatorCallbackHandler handler = createHandler(configs, new AccessTokenBuilder())) {
OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken);
- handler.handle(new Callback[] {callback});
+ handler.handle(new Callback[]{callback});
assertNull(callback.token());
String actualMessage = callback.errorStatus();
assertNotNull(actualMessage);
assertTrue(actualMessage.contains(expectedMessageSubstring), String.format("The error message \"%s\" didn't contain the expected substring \"%s\"", actualMessage, expectedMessageSubstring));
- } finally {
- handler.close();
}
}
- private OAuthBearerValidatorCallbackHandler createHandler(Map options,
- AccessTokenBuilder builder) {
+ private OAuthBearerValidatorCallbackHandler createHandler(Map configs, AccessTokenBuilder builder) throws Exception {
OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler();
- CloseableVerificationKeyResolver verificationKeyResolver = (jws, nestingContext) ->
- builder.jwk().getPublicKey();
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(options, verificationKeyResolver);
- handler.init(verificationKeyResolver, accessTokenValidator);
+ DefaultAccessTokenValidator accessTokenValidator = new DefaultAccessTokenValidator();
+ handler.configure(accessTokenValidator, configs, OAUTHBEARER_MECHANISM, List.of());
return handler;
}
-
- private String createAccessKey(String header, String payload, String signature) {
- Base64.Encoder enc = Base64.getEncoder();
- header = enc.encodeToString(Utils.utf8(header));
- payload = enc.encodeToString(Utils.utf8(payload));
- signature = enc.encodeToString(Utils.utf8(signature));
- return String.format("%s.%s.%s", header, payload, signature);
- }
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java
index cc910e0d16c4f..4b3ae713da1d2 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.utils.MockTime;
@@ -201,5 +200,4 @@ public String build() throws JoseException, IOException {
return jws.getCompactSerialization();
}
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactoryTest.java
deleted file mode 100644
index 2fd02e3f9a826..0000000000000
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactoryTest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler;
-
-import org.junit.jupiter.api.Test;
-
-import java.io.IOException;
-import java.util.Map;
-
-public class AccessTokenValidatorFactoryTest extends OAuthBearerTest {
-
- @Test
- public void testConfigureThrowsExceptionOnAccessTokenValidatorInit() {
- OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
- @Override
- public void init() throws IOException {
- throw new IOException("My init had an error!");
- }
- @Override
- public String retrieve() {
- return "dummy";
- }
- };
-
- Map configs = getSaslConfigs();
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
-
- assertThrowsWithMessage(
- KafkaException.class, () -> handler.init(accessTokenRetriever, accessTokenValidator), "encountered an error when initializing");
- }
-
- @Test
- public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() {
- OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
- @Override
- public void close() throws IOException {
- throw new IOException("My close had an error!");
- }
- @Override
- public String retrieve() {
- return "dummy";
- }
- };
-
- Map configs = getSaslConfigs();
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
- handler.init(accessTokenRetriever, accessTokenValidator);
-
- // Basically asserting this doesn't throw an exception :(
- handler.close();
- }
-
-}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerTokenTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerTokenTest.java
index 4e4ed591c5e39..ec49b3d385ade 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerTokenTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerTokenTest.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
@@ -88,5 +87,4 @@ public void noErrorIfModifyScope() {
// Ensure that attempting to change the token's scope set directly will not throw any error.
token.scope().clear();
}
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerAccessTokenValidatorTest.java
similarity index 67%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerAccessTokenValidatorTest.java
index 4db20e9ee10d6..0547d7ac487ff 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerAccessTokenValidatorTest.java
@@ -14,30 +14,46 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.oauthbearer.AccessTokenValidator;
+import org.apache.kafka.common.security.oauthbearer.AccessTokenValidatorTest;
import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.BrokerAccessTokenValidator;
import org.jose4j.jwk.PublicJsonWebKey;
import org.jose4j.jws.AlgorithmIdentifiers;
import org.jose4j.lang.InvalidAlgorithmException;
import org.junit.jupiter.api.Test;
+import java.security.Key;
import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
-public class ValidatorAccessTokenValidatorTest extends AccessTokenValidatorTest {
+public class BrokerAccessTokenValidatorTest extends AccessTokenValidatorTest {
@Override
- protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) {
- return new ValidatorAccessTokenValidator(30,
- Collections.emptySet(),
- null,
- (jws, nestingContext) -> builder.jwk().getKey(),
- builder.scopeClaimName(),
- builder.subjectClaimName());
+ protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) throws Exception {
+ Key key = builder.jwk() != null ? builder.jwk().getKey() : null;
+ CloseableVerificationKeyResolver keyResolver = mock(CloseableVerificationKeyResolver.class);
+ when(keyResolver.resolveKey(any(), any())).thenReturn(key);
+
+ return new BrokerAccessTokenValidator() {
+ @Override
+ public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
+ super.configure(keyResolver, configs, saslMechanism, jaasConfigEntries);
+ }
+ };
}
@Test
@@ -73,6 +89,8 @@ public void testMissingSubShouldBeValid() throws Exception {
.subjectClaimName(subClaimName)
.subject(null);
AccessTokenValidator validator = createAccessTokenValidator(tokenBuilder);
+ Map configs = Collections.singletonMap(SASL_OAUTHBEARER_SUB_CLAIM_NAME, tokenBuilder.subjectClaimName());
+ validator.configure(getSaslConfigs(configs), OAUTHBEARER_MECHANISM, List.of());
// Validation should succeed (e.g. signature verification) even if sub claim is missing
OAuthBearerToken token = validator.validate(tokenBuilder.build());
@@ -83,6 +101,7 @@ public void testMissingSubShouldBeValid() throws Exception {
private void testEncryptionAlgorithm(PublicJsonWebKey jwk, String alg) throws Exception {
AccessTokenBuilder builder = new AccessTokenBuilder().jwk(jwk).alg(alg);
AccessTokenValidator validator = createAccessTokenValidator(builder);
+ validator.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
String accessToken = builder.build();
OAuthBearerToken token = validator.validate(accessToken);
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java
index 89387797cdc30..671bd8ff4e43e 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java
@@ -14,9 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.oauthbearer.InvalidJwtException;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
@@ -42,15 +42,15 @@ public void testValidateScopes() {
@Test
public void testValidateScopesDisallowsDuplicates() {
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "b", "a")));
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "b", " a ")));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "b", "a")));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "b", " a ")));
}
@Test
public void testValidateScopesDisallowsEmptyNullAndWhitespace() {
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "")));
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", null)));
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", " ")));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", "")));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", null)));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateScopes("scope", Arrays.asList("a", " ")));
}
@Test
@@ -100,12 +100,12 @@ public void testValidateExpirationAllowsZero() {
@Test
public void testValidateExpirationDisallowsNull() {
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateExpiration("exp", null));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateExpiration("exp", null));
}
@Test
public void testValidateExpirationDisallowsNegatives() {
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateExpiration("exp", -1L));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateExpiration("exp", -1L));
}
@Test
@@ -117,9 +117,9 @@ public void testValidateSubject() {
@Test
public void testValidateSubjectDisallowsEmptyNullAndWhitespace() {
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", ""));
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", null));
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", " "));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateSubject("sub", ""));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateSubject("sub", null));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateSubject("sub", " "));
}
@Test
@@ -131,9 +131,9 @@ public void testValidateClaimNameOverride() {
@Test
public void testValidateClaimNameOverrideDisallowsEmptyNullAndWhitespace() {
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", ""));
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", null));
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", " "));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateSubject("sub", ""));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateSubject("sub", null));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateSubject("sub", " "));
}
@Test
@@ -159,7 +159,6 @@ public void testValidateIssuedAtAllowsNull() {
@Test
public void testValidateIssuedAtDisallowsNegatives() {
- assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateIssuedAt("iat", -1L));
+ assertThrows(InvalidJwtException.class, () -> ClaimValidationUtils.validateIssuedAt("iat", -1L));
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientAccessTokenValidatorTest.java
similarity index 65%
rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientAccessTokenValidatorTest.java
index 0a38f2b5094d5..078a9d458878e 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientAccessTokenValidatorTest.java
@@ -14,22 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
-import java.io.IOException;
-
-public interface Initable {
+import org.apache.kafka.common.security.oauthbearer.AccessTokenValidator;
+import org.apache.kafka.common.security.oauthbearer.AccessTokenValidatorTest;
+import org.apache.kafka.common.security.oauthbearer.ClientAccessTokenValidator;
- /**
- * Lifecycle method to perform any one-time initialization of the retriever. This must
- * be performed by the caller to ensure the correct state before methods are invoked.
- *
- * @throws IOException Thrown on errors related to IO during initialization
- */
+public class ClientAccessTokenValidatorTest extends AccessTokenValidatorTest {
- default void init() throws IOException {
- // This method left intentionally blank.
+ @Override
+ protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) {
+ return new ClientAccessTokenValidator();
}
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatterTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatterTest.java
new file mode 100644
index 0000000000000..7eaa5a0ddb89e
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestFormatterTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.security.oauthbearer.ClientCredentialsAccessTokenRetriever;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class ClientCredentialsRequestFormatterTest extends RequestFormatterTest {
+
+ @Test
+ public void testFormatAuthorizationHeader() {
+ ClientCredentialsRequestFormatter requestFormatter = new Builder()
+ .setClientId("id")
+ .setClientSecret("secret")
+ .build();
+ assertAuthorizationHeaderEquals(requestFormatter, "Basic aWQ6c2VjcmV0");
+ }
+
+ @Test
+ public void testFormatAuthorizationHeaderEncoding() {
+ ClientCredentialsRequestFormatter requestFormatter = new Builder()
+ .setClientId("SOME_RANDOM_LONG_USER_01234")
+ .setClientSecret("9Q|0`8i~ute-n9ksjLWb\\50\"AX@UUED5E")
+ .build();
+ // according to RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496.
+ assertAuthorizationHeaderEquals(requestFormatter, "Basic U09NRV9SQU5ET01fTE9OR19VU0VSXzAxMjM0OjlRfDBgOGl+dXRlLW45a3NqTFdiXDUwIkFYQFVVRUQ1RQ==");
+
+ requestFormatter = new Builder()
+ .setClientId("user!@~'")
+ .setClientSecret("secret-(*)!")
+ .setUrlencode(true)
+ .build();
+ // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1
+ assertAuthorizationHeaderEquals(requestFormatter, "Basic dXNlciUyMSU0MCU3RSUyNzpzZWNyZXQtJTI4KiUyOSUyMQ==");
+ }
+
+ @Test
+ public void testFormatRequestBody() {
+ ClientCredentialsRequestFormatter requestFormatter = new Builder()
+ .setScope("test")
+ .build();
+ assertBodyEquals(requestFormatter, "grant_type=client_credentials&scope=test");
+ }
+
+ @Test
+ public void testFormatRequestBodyWithEscaped() {
+ String questionMark = "%3F";
+ String exclamationMark = "%21";
+
+ Builder builder = new Builder()
+ .setUrlencode(false);
+
+ String expected = String.format("grant_type=client_credentials&scope=earth+is+great%s", exclamationMark);
+ assertBodyEquals(builder.setScope("earth is great!").build(), expected);
+
+ expected = String.format("grant_type=client_credentials&scope=what+on+earth%s%s%s%s%s", questionMark, exclamationMark, questionMark, exclamationMark, questionMark);
+ assertBodyEquals(builder.setScope("what on earth?!?!?").build(), expected);
+ }
+
+ @Test
+ public void testFormatRequestBodyMissingValues() {
+ Builder builder = new Builder();
+
+ String expected = "grant_type=client_credentials";
+ assertBodyEquals(builder.setScope(null).build(), expected);
+ assertBodyEquals(builder.setScope("").build(), expected);
+ assertBodyEquals(builder.setScope(" ").build(), expected);
+ }
+
+ @ParameterizedTest
+ @MethodSource("urlencodeHeaderSupplier")
+ public void testUrlencodeHeader(Map configs, boolean expectedValue) {
+ ConfigurationUtils cu = new ConfigurationUtils(configs, OAUTHBEARER_MECHANISM);
+ boolean actualValue = ClientCredentialsAccessTokenRetriever.validateUrlencodeHeader(cu);
+ assertEquals(expectedValue, actualValue);
+ }
+
+ private static Stream urlencodeHeaderSupplier() {
+ return Stream.of(
+ Arguments.of(Collections.emptyMap(), DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE),
+ Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, null), DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE),
+ Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, true), true),
+ Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, false), false)
+ );
+ }
+
+ private void assertAuthorizationHeaderEquals(ClientCredentialsRequestFormatter requestFormatter, String expected) {
+ assertHeadersEqual(requestFormatter, Collections.singletonMap("Authorization", expected));
+ }
+
+ private static class Builder {
+
+ private String clientId = "testClientId";
+ private String clientSecret = "testSecret";
+ private String scope = "testScope";
+ private boolean urlencode = false;
+
+ public Builder setClientId(String clientId) {
+ this.clientId = clientId;
+ return this;
+ }
+
+ public Builder setClientSecret(String clientSecret) {
+ this.clientSecret = clientSecret;
+ return this;
+ }
+
+ public Builder setScope(String scope) {
+ this.scope = scope;
+ return this;
+ }
+
+ public Builder setUrlencode(boolean urlencode) {
+ this.urlencode = urlencode;
+ return this;
+ }
+
+ private ClientCredentialsRequestFormatter build() {
+ return new ClientCredentialsRequestFormatter(
+ clientId,
+ clientSecret,
+ scope,
+ urlencode
+ );
+ }
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java
index 9a62f480215f7..88d5a02a05472 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.config.ConfigException;
@@ -30,6 +29,7 @@
import java.util.Map;
import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
public class ConfigurationUtilsTest extends OAuthBearerTest {
@@ -95,7 +95,7 @@ public void testUrlWhitespace() {
private void testUrl(String value) {
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, value == null ? "" : value);
Map configs = Collections.singletonMap(URL_CONFIG_NAME, value);
- ConfigurationUtils cu = new ConfigurationUtils(configs);
+ ConfigurationUtils cu = new ConfigurationUtils(configs, OAUTHBEARER_MECHANISM);
cu.validateUrl(URL_CONFIG_NAME);
}
@@ -148,7 +148,7 @@ public void testThrowIfURLIsNotAllowed() {
Map configs = new HashMap<>();
configs.put(URL_CONFIG_NAME, url);
configs.put(FILE_CONFIG_NAME, fileUrl);
- ConfigurationUtils cu = new ConfigurationUtils(configs);
+ ConfigurationUtils cu = new ConfigurationUtils(configs, OAUTHBEARER_MECHANISM);
// By default, no URL is allowed
assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(url),
@@ -171,8 +171,7 @@ public void testThrowIfURLIsNotAllowed() {
protected void testFile(String value) {
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, value == null ? "" : value);
Map configs = Collections.singletonMap(URL_CONFIG_NAME, value);
- ConfigurationUtils cu = new ConfigurationUtils(configs);
+ ConfigurationUtils cu = new ConfigurationUtils(configs, OAUTHBEARER_MECHANISM);
cu.validateFile(URL_CONFIG_NAME);
}
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DelegatingVerificationKeyResolverTest.java
similarity index 69%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactoryTest.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DelegatingVerificationKeyResolverTest.java
index c2324b9d2dac2..23564f07cff38 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactoryTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DelegatingVerificationKeyResolverTest.java
@@ -14,9 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.ConfigException;
import org.junit.jupiter.api.AfterEach;
@@ -24,12 +24,15 @@
import java.io.File;
import java.util.Collections;
+import java.util.List;
import java.util.Map;
import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
+import static org.junit.jupiter.api.Assertions.assertThrows;
-public class VerificationKeyResolverFactoryTest extends OAuthBearerTest {
+public class DelegatingVerificationKeyResolverTest extends OAuthBearerTest {
@AfterEach
public void tearDown() throws Exception {
@@ -43,20 +46,23 @@ public void testConfigureRefreshingFileVerificationKeyResolver() throws Exceptio
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, verificationKeyFile.toURI().toString());
Map configs = Collections.singletonMap(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, verificationKeyFile.toURI().toString());
- Map jaasConfig = Collections.emptyMap();
- // verify it won't throw exception
- try (CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, jaasConfig)) { }
+ // verify it throws an exception
+ try (CloseableVerificationKeyResolver verificationKeyResolver = new DelegatingVerificationKeyResolver(time)) {
+ assertThrows(KafkaException.class, () -> verificationKeyResolver.configure(configs, OAUTHBEARER_MECHANISM, List.of()));
+ }
}
@Test
- public void testConfigureRefreshingFileVerificationKeyResolverWithInvalidDirectory() {
+ public void testConfigureRefreshingFileVerificationKeyResolverWithInvalidDirectory() throws Exception {
// Should fail because the parent path doesn't exist.
String file = new File("/tmp/this-directory-does-not-exist/foo.json").toURI().toString();
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file);
Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, file);
- Map jaasConfig = Collections.emptyMap();
- assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, jaasConfig), "that doesn't exist");
+
+ try (CloseableVerificationKeyResolver verificationKeyResolver = new DelegatingVerificationKeyResolver(time)) {
+ assertThrowsWithMessage(ConfigException.class, () -> verificationKeyResolver.configure(configs, OAUTHBEARER_MECHANISM, List.of()), "that doesn't exist");
+ }
}
@Test
@@ -66,8 +72,10 @@ public void testConfigureRefreshingFileVerificationKeyResolverWithInvalidFile()
File verificationKeyFile = new File(tmpDir, "this-file-does-not-exist.json");
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, verificationKeyFile.toURI().toString());
Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, verificationKeyFile.toURI().toString());
- Map jaasConfig = Collections.emptyMap();
- assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, jaasConfig), "that doesn't exist");
+
+ try (CloseableVerificationKeyResolver verificationKeyResolver = new DelegatingVerificationKeyResolver(time)) {
+ assertThrowsWithMessage(ConfigException.class, () -> verificationKeyResolver.configure(configs, OAUTHBEARER_MECHANISM, List.of()), "that doesn't exist");
+ }
}
@Test
@@ -76,7 +84,9 @@ public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception {
File tmpDir = createTempDir("not_allowed");
File verificationKeyFile = new File(tmpDir, "not_allowed.json");
Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, verificationKeyFile.toURI().toString());
- assertThrowsWithMessage(ConfigException.class, () -> VerificationKeyResolverFactory.create(configs, Collections.emptyMap()),
- ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG);
+
+ try (CloseableVerificationKeyResolver verificationKeyResolver = new DelegatingVerificationKeyResolver(time)) {
+ assertThrowsWithMessage(ConfigException.class, () -> verificationKeyResolver.configure(configs, OAUTHBEARER_MECHANISM, List.of()), ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG);
+ }
}
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileAccessTokenRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileAccessTokenRetrieverTest.java
new file mode 100644
index 0000000000000..850fd35555461
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileAccessTokenRetrieverTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.login.AppConfigurationEntry;
+
+import static javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag.OPTIONAL;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
+import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FileAccessTokenRetrieverTest extends OAuthBearerTest {
+
+ @Test
+ public void testFileTokenRetrieverHandlesNewline() throws IOException {
+ Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+ long cur = cal.getTimeInMillis() / 1000;
+ String exp = "" + (cur + 60 * 60); // 1 hour in future
+ String iat = "" + cur;
+
+ String expected = createAccessKey("{}", String.format("{\"exp\":%s, \"iat\":%s, \"sub\":\"subj\"}", exp, iat), "sign");
+ String withNewline = expected + "\n";
+
+ File tmpDir = createTempDir("access-token");
+ File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", withNewline);
+
+ List jaasConfigEntries = new ArrayList<>();
+ jaasConfigEntries.add(new AppConfigurationEntry("dummy", OPTIONAL, Collections.emptyMap()));
+
+ System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.toURI().toString());
+ Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString());
+
+ try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler()) {
+ handler.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries);
+ OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+ handler.handle(new Callback[]{callback});
+ assertEquals(callback.token().value(), expected);
+ } catch (Exception e) {
+ fail(e);
+ }
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetrieverTest.java
index 8b1c5a370652e..d3756e07b1e4e 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetrieverTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetrieverTest.java
@@ -14,126 +14,70 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.kafka.common.security.oauthbearer.HttpAccessTokenRetriever;
+import org.apache.kafka.common.security.oauthbearer.HttpClient;
import org.junit.jupiter.api.Test;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.net.HttpURLConnection;
import java.nio.charset.StandardCharsets;
-import java.util.Random;
+import java.util.Optional;
-import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
public class HttpAccessTokenRetrieverTest extends OAuthBearerTest {
@Test
- public void test() throws IOException {
- String expectedResponse = "Hiya, buddy";
- HttpURLConnection mockedCon = createHttpURLConnection(expectedResponse);
- String response = HttpAccessTokenRetriever.post(mockedCon, null, null, null, null);
- assertEquals(expectedResponse, response);
- }
-
- @Test
- public void testEmptyResponse() throws IOException {
- HttpURLConnection mockedCon = createHttpURLConnection("");
- assertThrows(IOException.class, () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null));
- }
-
- @Test
- public void testErrorReadingResponse() throws IOException {
- HttpURLConnection mockedCon = createHttpURLConnection("dummy");
- when(mockedCon.getInputStream()).thenThrow(new IOException("Can't read"));
-
- assertThrows(IOException.class, () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null));
- }
-
- @Test
- public void testErrorResponseUnretryableCode() throws IOException {
- HttpURLConnection mockedCon = createHttpURLConnection("dummy");
- when(mockedCon.getInputStream()).thenThrow(new IOException("Can't read"));
- when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream(
- "{\"error\":\"some_arg\", \"error_description\":\"some problem with arg\"}"
- .getBytes(StandardCharsets.UTF_8)));
- when(mockedCon.getResponseCode()).thenReturn(HttpURLConnection.HTTP_BAD_REQUEST);
- UnretryableException ioe = assertThrows(UnretryableException.class,
- () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null));
- assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}"));
+ public void testErrorResponseUnretryableCode() {
+ testErrorResponse(
+ UnretryableException.class,
+ HttpURLConnection.HTTP_BAD_REQUEST,
+ "{\"error\":\"some_arg\", \"error_description\":\"some problem with arg\"}",
+ "{\"some_arg\" - \"some problem with arg\"}"
+ );
}
@Test
public void testErrorResponseRetryableCode() throws IOException {
- HttpURLConnection mockedCon = createHttpURLConnection("dummy");
- when(mockedCon.getInputStream()).thenThrow(new IOException("Can't read"));
- when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream(
- "{\"error\":\"some_arg\", \"error_description\":\"some problem with arg\"}"
- .getBytes(StandardCharsets.UTF_8)));
- when(mockedCon.getResponseCode()).thenReturn(HttpURLConnection.HTTP_INTERNAL_ERROR);
- IOException ioe = assertThrows(IOException.class,
- () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null));
- assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}"));
+ testErrorResponse(
+ IOException.class,
+ HttpURLConnection.HTTP_INTERNAL_ERROR,
+ "{\"error\":\"some_arg\", \"error_description\":\"some problem with arg\"}",
+ "{\"some_arg\" - \"some problem with arg\"}"
+ );
// error response body has different keys
- when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream(
- "{\"errorCode\":\"some_arg\", \"errorSummary\":\"some problem with arg\"}"
- .getBytes(StandardCharsets.UTF_8)));
- ioe = assertThrows(IOException.class,
- () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null));
- assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}"));
+ testErrorResponse(
+ IOException.class,
+ HttpURLConnection.HTTP_INTERNAL_ERROR,
+ "{\"errorCode\":\"some_arg\", \"errorSummary\":\"some problem with arg\"}",
+ "{\"some_arg\" - \"some problem with arg\"}"
+ );
// error response is valid json but unknown keys
- when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream(
+ testErrorResponse(
+ IOException.class,
+ HttpURLConnection.HTTP_INTERNAL_ERROR,
+ "{\"err\":\"some_arg\", \"err_des\":\"some problem with arg\"}",
"{\"err\":\"some_arg\", \"err_des\":\"some problem with arg\"}"
- .getBytes(StandardCharsets.UTF_8)));
- ioe = assertThrows(IOException.class,
- () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null));
- assertTrue(ioe.getMessage().contains("{\"err\":\"some_arg\", \"err_des\":\"some problem with arg\"}"));
+ );
}
@Test
- public void testErrorResponseIsInvalidJson() throws IOException {
- HttpURLConnection mockedCon = createHttpURLConnection("dummy");
- when(mockedCon.getInputStream()).thenThrow(new IOException("Can't read"));
- when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream(
- "non json error output".getBytes(StandardCharsets.UTF_8)));
- when(mockedCon.getResponseCode()).thenReturn(HttpURLConnection.HTTP_INTERNAL_ERROR);
- IOException ioe = assertThrows(IOException.class,
- () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null));
- assertTrue(ioe.getMessage().contains("{non json error output}"));
- }
-
- @Test
- public void testCopy() throws IOException {
- byte[] expected = new byte[4096 + 1];
- Random r = new Random();
- r.nextBytes(expected);
- InputStream in = new ByteArrayInputStream(expected);
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- HttpAccessTokenRetriever.copy(in, out);
- assertArrayEquals(expected, out.toByteArray());
- }
-
- @Test
- public void testCopyError() throws IOException {
- InputStream mockedIn = mock(InputStream.class);
- OutputStream out = new ByteArrayOutputStream();
- when(mockedIn.read(any(byte[].class))).thenThrow(new IOException());
- assertThrows(IOException.class, () -> HttpAccessTokenRetriever.copy(mockedIn, out));
+ public void testErrorResponseIsInvalidJson() {
+ testErrorResponse(
+ IOException.class,
+ HttpURLConnection.HTTP_INTERNAL_ERROR,
+ "non json error output",
+ "{non json error output}"
+ );
}
@Test
@@ -170,69 +114,19 @@ public void testParseAccessTokenInvalidJson() {
assertThrows(IOException.class, () -> HttpAccessTokenRetriever.parseAccessToken("not valid JSON"));
}
- @Test
- public void testFormatAuthorizationHeader() {
- assertAuthorizationHeader("id", "secret", false, "Basic aWQ6c2VjcmV0");
- }
-
- @Test
- public void testFormatAuthorizationHeaderEncoding() {
- // according to RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496.
- assertAuthorizationHeader("SOME_RANDOM_LONG_USER_01234", "9Q|0`8i~ute-n9ksjLWb\\50\"AX@UUED5E", false, "Basic U09NRV9SQU5ET01fTE9OR19VU0VSXzAxMjM0OjlRfDBgOGl+dXRlLW45a3NqTFdiXDUwIkFYQFVVRUQ1RQ==");
- // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1
- assertAuthorizationHeader("user!@~'", "secret-(*)!", true, "Basic dXNlciUyMSU0MCU3RSUyNzpzZWNyZXQtJTI4KiUyOSUyMQ==");
- }
-
- private void assertAuthorizationHeader(String clientId, String clientSecret, boolean urlencode, String expected) {
- String actual = HttpAccessTokenRetriever.formatAuthorizationHeader(clientId, clientSecret, urlencode);
- assertEquals(expected, actual, String.format("Expected the HTTP Authorization header generated for client ID \"%s\" and client secret \"%s\" to match", clientId, clientSecret));
+ private void testErrorResponse(Class exceptionClazz,
+ int responseCode,
+ String errorResponse,
+ String substringMatch) {
+ HttpClient.HttpResponse response = new HttpClient.HttpResponse(
+ responseCode,
+ Optional.empty(),
+ Optional.of(errorResponse.getBytes(StandardCharsets.UTF_8))
+ );
+ Exception e = assertThrows(
+ exceptionClazz,
+ () -> HttpAccessTokenRetriever.handleOutput("https://www.example.com", response)
+ );
+ assertTrue(e.getMessage().contains(substringMatch), e.getMessage());
}
-
- @Test
- public void testFormatAuthorizationHeaderMissingValues() {
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader(null, "secret", false));
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader("id", null, false));
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader(null, null, false));
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader("", "secret", false));
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader("id", "", false));
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader("", "", false));
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader(" ", "secret", false));
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader("id", " ", false));
- assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.formatAuthorizationHeader(" ", " ", false));
- }
-
- @Test
- public void testFormatRequestBody() {
- String expected = "grant_type=client_credentials&scope=scope";
- String actual = HttpAccessTokenRetriever.formatRequestBody("scope");
- assertEquals(expected, actual);
- }
-
- @Test
- public void testFormatRequestBodyWithEscaped() {
- String questionMark = "%3F";
- String exclamationMark = "%21";
-
- String expected = String.format("grant_type=client_credentials&scope=earth+is+great%s", exclamationMark);
- String actual = HttpAccessTokenRetriever.formatRequestBody("earth is great!");
- assertEquals(expected, actual);
-
- expected = String.format("grant_type=client_credentials&scope=what+on+earth%s%s%s%s%s", questionMark, exclamationMark, questionMark, exclamationMark, questionMark);
- actual = HttpAccessTokenRetriever.formatRequestBody("what on earth?!?!?");
- assertEquals(expected, actual);
- }
-
- @Test
- public void testFormatRequestBodyMissingValues() {
- String expected = "grant_type=client_credentials";
- String actual = HttpAccessTokenRetriever.formatRequestBody(null);
- assertEquals(expected, actual);
-
- actual = HttpAccessTokenRetriever.formatRequestBody("");
- assertEquals(expected, actual);
-
- actual = HttpAccessTokenRetriever.formatRequestBody(" ");
- assertEquals(expected, actual);
- }
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpClientTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpClientTest.java
new file mode 100644
index 0000000000000..15d5fae82d174
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpClientTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.security.oauthbearer.HttpClient;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.nio.charset.StandardCharsets;
+import java.util.Optional;
+import java.util.Random;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class HttpClientTest extends OAuthBearerTest {
+
+ @Test
+ public void testPost() throws IOException {
+ String expectedResponse = "Hiya, buddy";
+ HttpURLConnection mockedCon = createHttpURLConnection(expectedResponse);
+ Optional actualResponseOpt = post(mockedCon);
+ assertTrue(actualResponseOpt.isPresent());
+ assertEquals(expectedResponse, actualResponseOpt.get());
+ }
+
+ @Test
+ public void testPostWithEmptyResponse() throws IOException {
+ HttpURLConnection mockedCon = createHttpURLConnection("");
+ Optional actualResponseOpt = post(mockedCon);
+ assertFalse(actualResponseOpt.isPresent());
+ }
+
+ @Test
+ public void testPostWithErrorReadingResponse() throws IOException {
+ HttpURLConnection mockedCon = createHttpURLConnection("dummy");
+ when(mockedCon.getInputStream()).thenThrow(new IOException("Can't read"));
+ Optional actualResponseOpt = post(mockedCon);
+ assertFalse(actualResponseOpt.isPresent());
+ }
+
+ @Test
+ public void testCopy() throws IOException {
+ byte[] expected = new byte[4096 + 1];
+ Random r = new Random();
+ r.nextBytes(expected);
+ InputStream in = new ByteArrayInputStream(expected);
+ ByteArrayOutputStream out = new ByteArrayOutputStream();
+ HttpClient.copy(in, out);
+ assertArrayEquals(expected, out.toByteArray());
+ }
+
+ @Test
+ public void testCopyError() throws IOException {
+ try (InputStream mockedIn = mock(InputStream.class)) {
+ OutputStream out = new ByteArrayOutputStream();
+ when(mockedIn.read(any(byte[].class))).thenThrow(new IOException());
+ assertThrows(IOException.class, () -> HttpClient.copy(mockedIn, out));
+ }
+ }
+
+ private Optional post(HttpURLConnection con) throws IOException {
+ HttpClient client = new HttpClient("URL");
+ client.write(con, new byte[0]);
+ HttpClient.HttpResponse response = client.read(con);
+ return response.responseBody.map(bytes -> new String(bytes, StandardCharsets.UTF_8));
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtilsTest.java
index 722c734dcc7d6..7c7d891778edc 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtilsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtilsTest.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.config.SslConfigs;
@@ -54,9 +53,9 @@ public void testSSLClientConfig() {
@Test
public void testShouldUseSslClientConfig() throws Exception {
JaasOptionsUtils jou = new JaasOptionsUtils(Collections.emptyMap());
- assertFalse(jou.shouldCreateSSLSocketFactory(new URL("http://example.com")));
- assertTrue(jou.shouldCreateSSLSocketFactory(new URL("https://example.com")));
- assertFalse(jou.shouldCreateSSLSocketFactory(new URL("file:///tmp/test.txt")));
+ assertFalse(jou.maybeCreateSslResource(new URL("http://www.example.com")).isPresent());
+ assertTrue(jou.maybeCreateSslResource(new URL("https://www.example.com")).isPresent());
+ assertFalse(jou.maybeCreateSslResource(new URL("file:///tmp/test.txt")).isPresent());
}
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatterTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatterTest.java
new file mode 100644
index 0000000000000..846086d44ccae
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestFormatterTest.java
@@ -0,0 +1,240 @@
+/*
+ * 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.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+
+import org.jose4j.jwt.JwtClaims;
+import org.jose4j.jwt.consumer.InvalidJwtException;
+import org.jose4j.jwt.consumer.JwtConsumer;
+import org.jose4j.jwt.consumer.JwtConsumerBuilder;
+import org.jose4j.jwt.consumer.JwtContext;
+import org.jose4j.jwx.JsonWebStructure;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBearerRequestFormatter.GRANT_TYPE;
+import static org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBearerRequestFormatter.TOKEN_SIGNING_ALGORITHM_RS256;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class JwtBearerRequestFormatterTest extends RequestFormatterTest {
+
+ @Test
+ public void testRequestBodyParameters() throws Exception {
+ Builder builder = new Builder()
+ .setPrivateKeySecret(generatePrivateKeySecret());
+ JwtBearerRequestFormatter requestFormatter = builder.build();
+ String assertion = requestFormatter.createAssertion();
+ String requestBody = requestFormatter.formatBody();
+ String expected = "grant_type=" + URLEncoder.encode(GRANT_TYPE, StandardCharsets.UTF_8) + "&assertion=" + assertion;
+ assertEquals(
+ expected,
+ requestBody
+ );
+ }
+
+ @Test
+ public void testPrivateKeyId() throws Exception {
+ KeyPair keyPair = generateKeyPair();
+ Builder builder = new Builder()
+ .setPrivateKeySecret(generatePrivateKeySecret(keyPair.getPrivate()))
+ .setPrivateKeyId("test-id");
+ JwtBearerRequestFormatter requestFormatter = builder.build();
+ String assertion = requestFormatter.createAssertion();
+ JwtContext context = assertContext(builder, keyPair.getPublic(), assertion);
+ List joseObjects = context.getJoseObjects();
+ assertNotNull(joseObjects);
+ assertEquals(1, joseObjects.size());
+ JsonWebStructure jsonWebStructure = joseObjects.get(0);
+ assertEquals("test-id", jsonWebStructure.getKeyIdHeaderValue());
+ }
+
+ @Test
+ public void testPrivateKeySecret() throws Exception {
+ KeyPair keyPair = generateKeyPair();
+ Builder builder = new Builder()
+ .setPrivateKeySecret(generatePrivateKeySecret(keyPair.getPrivate()));
+ JwtBearerRequestFormatter requestFormatter = builder.build();
+ String assertion = requestFormatter.createAssertion();
+ assertClaims(builder, keyPair.getPublic(), assertion);
+ }
+
+ @Test
+ public void testInvalidPrivateKeySecret() throws Exception {
+ // Intentionally "mangle" the private key secret by stripping off the first character.
+ String privateKeySecret = generatePrivateKeySecret().substring(1);
+
+ JwtBearerRequestFormatter requestFormatter = new Builder()
+ .setPrivateKeySecret(privateKeySecret)
+ .build();
+ assertThrows(KafkaException.class, requestFormatter::formatBody);
+ }
+
+ @ParameterizedTest
+ @CsvSource("RS256,ES256")
+ public void testTokenSigningAlgo(String tokenSigningAlgo) throws Exception {
+ KeyPair keyPair = generateKeyPair();
+ Builder builder = new Builder()
+ .setPrivateKeySecret(generatePrivateKeySecret(keyPair.getPrivate()))
+ .setTokenSigningAlgo(tokenSigningAlgo);
+ JwtBearerRequestFormatter requestFormatter = builder.build();
+ String assertion = requestFormatter.createAssertion();
+ assertClaims(builder, keyPair.getPublic(), assertion);
+
+ JwtContext context = assertContext(builder, keyPair.getPublic(), assertion);
+ List joseObjects = context.getJoseObjects();
+ assertNotNull(joseObjects);
+ assertEquals(1, joseObjects.size());
+ JsonWebStructure jsonWebStructure = joseObjects.get(0);
+ assertEquals(tokenSigningAlgo, jsonWebStructure.getAlgorithmHeaderValue());
+ }
+
+ @Test
+ public void testInvalidTokenSigningAlgo() {
+ PrivateKey privateKey = generateKeyPair().getPrivate();
+ Builder builder = new Builder()
+ .setPrivateKeySecret(generatePrivateKeySecret(privateKey))
+ .setTokenSigningAlgo("thisisnotvalid");
+ JwtBearerRequestFormatter requestFormatter = builder.build();
+ assertThrows(NoSuchAlgorithmException.class, requestFormatter::getSignature);
+ assertThrows(
+ NoSuchAlgorithmException.class,
+ () -> requestFormatter.sign(privateKey, "dummy content"));
+ }
+
+ @Test
+ public void testContentTypeHeader() {
+ JwtBearerRequestFormatter requestFormatter = new Builder().build();
+ assertHeadersEqual(requestFormatter, Collections.singletonMap("Content-Type", "application/x-www-form-urlencoded"));
+ }
+
+ private JwtClaims assertClaims(Builder builder, PublicKey publicKey, String assertion) throws InvalidJwtException {
+ JwtConsumer jwtConsumer = jwtConsumer(builder, publicKey);
+ return jwtConsumer.processToClaims(assertion);
+ }
+
+ private JwtContext assertContext(Builder builder, PublicKey publicKey, String assertion) throws InvalidJwtException {
+ JwtConsumer jwtConsumer = jwtConsumer(builder, publicKey);
+ return jwtConsumer.process(assertion);
+ }
+
+ private JwtConsumer jwtConsumer(Builder builder, PublicKey publicKey) {
+ return new JwtConsumerBuilder()
+ .setVerificationKey(publicKey)
+ .setRequireExpirationTime()
+ .setAllowedClockSkewInSeconds(30) // Sure, let's give it some slack
+ .setExpectedSubject(builder.tokenSubject)
+ .setExpectedIssuer(builder.tokenIssuer)
+ .setExpectedAudience(builder.tokenAudience)
+ .build();
+ }
+
+ private String generatePrivateKeySecret(PrivateKey privateKey) {
+ return Base64.getEncoder().encodeToString(privateKey.getEncoded());
+ }
+
+ private String generatePrivateKeySecret() {
+ return generatePrivateKeySecret(generateKeyPair().getPrivate());
+ }
+
+ private KeyPair generateKeyPair() {
+ try {
+ KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+ keyGen.initialize(2048);
+ return keyGen.generateKeyPair();
+ } catch (NoSuchAlgorithmException e) {
+ throw new IllegalStateException("Received unexpected error during private key generation", e);
+ }
+ }
+
+ private static class Builder {
+
+ private final Time time = new MockTime();
+ private String privateKeyId = "testTokenSubject";
+ private String privateKeySecret = "testTokenSubject";
+ private String tokenSigningAlgo = TOKEN_SIGNING_ALGORITHM_RS256;
+ private String tokenSubject = "testTokenSubject";
+ private String tokenIssuer = "testTokenIssuer";
+ private String tokenAudience = "testTokenAudience";
+ private String tokenTargetAudience = "testTokenTargetAudience";
+
+ public Builder setPrivateKeyId(String privateKeyId) {
+ this.privateKeyId = privateKeyId;
+ return this;
+ }
+
+ public Builder setPrivateKeySecret(String privateKeySecret) {
+ this.privateKeySecret = privateKeySecret;
+ return this;
+ }
+
+ public Builder setTokenSigningAlgo(String tokenSigningAlgo) {
+ this.tokenSigningAlgo = tokenSigningAlgo;
+ return this;
+ }
+
+ public Builder setTokenSubject(String tokenSubject) {
+ this.tokenSubject = tokenSubject;
+ return this;
+ }
+
+ public Builder setTokenIssuer(String tokenIssuer) {
+ this.tokenIssuer = tokenIssuer;
+ return this;
+ }
+
+ public Builder setTokenAudience(String tokenAudience) {
+ this.tokenAudience = tokenAudience;
+ return this;
+ }
+
+ public Builder setTokenTargetAudience(String tokenTargetAudience) {
+ this.tokenTargetAudience = tokenTargetAudience;
+ return this;
+ }
+
+ private JwtBearerRequestFormatter build() {
+ return new JwtBearerRequestFormatter(
+ time,
+ privateKeyId,
+ privateKeySecret,
+ tokenSigningAlgo,
+ tokenSubject,
+ tokenIssuer,
+ tokenAudience,
+ tokenTargetAudience
+ );
+ }
+ }
+
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java
index 7f20b9464faea..9aa618aef7d52 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.config.AbstractConfig;
@@ -22,6 +21,8 @@
import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
import org.apache.kafka.common.security.authenticator.TestJaasConfig;
import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -54,6 +55,7 @@
import javax.security.auth.login.AppConfigurationEntry;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
@@ -65,6 +67,8 @@ public abstract class OAuthBearerTest {
protected final Logger log = LoggerFactory.getLogger(getClass());
+ protected final Time time = new MockTime();
+
protected ObjectMapper mapper = new ObjectMapper();
protected void assertThrowsWithMessage(Class extends Exception> clazz,
@@ -88,10 +92,18 @@ protected void configureHandler(AuthenticateCallbackHandler handler,
AppConfigurationEntry kafkaClient = config.getAppConfigurationEntry("KafkaClient")[0];
handler.configure(configs,
- OAuthBearerLoginModule.OAUTHBEARER_MECHANISM,
+ OAUTHBEARER_MECHANISM,
Collections.singletonList(kafkaClient));
}
+ protected String createAccessKey(String header, String payload, String signature) {
+ Base64.Encoder enc = Base64.getEncoder();
+ header = enc.encodeToString(Utils.utf8(header));
+ payload = enc.encodeToString(Utils.utf8(payload));
+ signature = enc.encodeToString(Utils.utf8(signature));
+ return String.format("%s.%s.%s", header, payload, signature);
+ }
+
protected String createBase64JsonJwtSection(Consumer c) {
String json = createJsonJwtSection(c);
@@ -211,5 +223,4 @@ protected PublicJsonWebKey createEcJwk() throws JoseException {
jwk.setKeyId("key-1");
return jwk;
}
-
}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksTest.java
index 1a77284779b36..94a796a623de1 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksTest.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.internals.KafkaFutureImpl;
@@ -30,6 +29,7 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
@@ -38,21 +38,28 @@
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
+import javax.security.auth.login.AppConfigurationEntry;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.apache.kafka.common.security.oauthbearer.internals.secured.RefreshingHttpsJwks.MISSING_KEY_ID_CACHE_IN_FLIGHT_MS;
import static org.apache.kafka.common.security.oauthbearer.internals.secured.RefreshingHttpsJwks.MISSING_KEY_ID_MAX_KEY_LENGTH;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
public class RefreshingHttpsJwksTest extends OAuthBearerTest {
- private static final int REFRESH_MS = 5000;
+ private static final long REFRESH_MS = 5000;
- private static final int RETRY_BACKOFF_MS = 50;
+ private static final long RETRY_BACKOFF_MS = 50;
- private static final int RETRY_BACKOFF_MAX_MS = 2000;
+ private static final long RETRY_BACKOFF_MAX_MS = 2000;
/**
* Test that a key not previously scheduled for refresh will be scheduled without a refresh.
@@ -64,10 +71,11 @@ public void testBasicScheduleRefresh() throws Exception {
MockTime time = new MockTime();
HttpsJwks httpsJwks = spyHttpsJwks();
+ assertNotNull(httpsJwks);
// we use mocktime here to ensure that scheduled refresh _doesn't_ run and update the invocation count
// we expect httpsJwks.refresh() to be invoked twice, once from init() and maybeExpediteRefresh() each
try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks)) {
- refreshingHttpsJwks.init();
+ refreshingHttpsJwks.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
verify(httpsJwks, times(1)).refresh();
assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId));
verify(httpsJwks, times(2)).refresh();
@@ -80,13 +88,13 @@ public void testBasicScheduleRefresh() throws Exception {
*/
@Test
- public void testMaybeExpediteRefreshNoDelay() throws Exception {
+ public void testMaybeExpediteRefreshNoDelay() {
String keyId = "abc123";
MockTime time = new MockTime();
HttpsJwks httpsJwks = spyHttpsJwks();
try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks)) {
- refreshingHttpsJwks.init();
+ refreshingHttpsJwks.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId));
assertFalse(refreshingHttpsJwks.maybeExpediteRefresh(keyId));
}
@@ -98,7 +106,7 @@ public void testMaybeExpediteRefreshNoDelay() throws Exception {
*/
@Test
- public void testMaybeExpediteRefreshDelays() throws Exception {
+ public void testMaybeExpediteRefreshDelays() {
assertMaybeExpediteRefreshWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS - 1, false);
assertMaybeExpediteRefreshWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS, true);
assertMaybeExpediteRefreshWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS + 1, true);
@@ -118,7 +126,7 @@ public void testLongKey() throws Exception {
HttpsJwks httpsJwks = spyHttpsJwks();
try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks)) {
- refreshingHttpsJwks.init();
+ refreshingHttpsJwks.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
verify(httpsJwks, times(1)).refresh();
assertFalse(refreshingHttpsJwks.maybeExpediteRefresh(keyId));
verify(httpsJwks, times(1)).refresh();
@@ -137,7 +145,7 @@ public void testSecondaryRefreshAfterElapsedDelay() throws Exception {
HttpsJwks httpsJwks = spyHttpsJwks();
try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks)) {
- refreshingHttpsJwks.init();
+ refreshingHttpsJwks.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
// We refresh once at the initialization time from getJsonWebKeys.
verify(httpsJwks, times(1)).refresh();
assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId));
@@ -173,21 +181,26 @@ private ScheduledExecutorService mockExecutorService(MockTime time) {
return executorService;
}
- private void assertMaybeExpediteRefreshWithDelay(long sleepDelay, boolean shouldBeScheduled) throws Exception {
+ private void assertMaybeExpediteRefreshWithDelay(long sleepDelay, boolean shouldBeScheduled) {
String keyId = "abc123";
MockTime time = new MockTime();
HttpsJwks httpsJwks = spyHttpsJwks();
try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks)) {
- refreshingHttpsJwks.init();
+ refreshingHttpsJwks.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId));
time.sleep(sleepDelay);
assertEquals(shouldBeScheduled, refreshingHttpsJwks.maybeExpediteRefresh(keyId));
}
}
- private RefreshingHttpsJwks getRefreshingHttpsJwks(final MockTime time, final HttpsJwks httpsJwks) {
- return new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS, RETRY_BACKOFF_MS, RETRY_BACKOFF_MAX_MS, mockExecutorService(time));
+ private RefreshingHttpsJwks getRefreshingHttpsJwks(final MockTime time, final HttpsJwks jwks) {
+ return new RefreshingHttpsJwks(time, mockExecutorService(time)) {
+ @Override
+ public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
+ super.configure(jwks, configs, saslMechanism);
+ }
+ };
}
/**
@@ -310,4 +323,13 @@ public ScheduledFuture schedule(final Callable callable, long delayMs,
}
}
+ @Override
+ protected Map getSaslConfigs() {
+ Map configs = new HashMap<>(super.getSaslConfigs());
+ configs.put(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, REFRESH_MS);
+ configs.put(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, RETRY_BACKOFF_MS);
+ configs.put(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, RETRY_BACKOFF_MAX_MS);
+ configs.put("fo", "bar");
+ return configs;
+ }
}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RequestFormatterTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RequestFormatterTest.java
new file mode 100644
index 0000000000000..0d787cc8a4a05
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RequestFormatterTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.kafka.common.security.oauthbearer.internals.secured;
+
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public abstract class RequestFormatterTest extends OAuthBearerTest {
+
+ protected void assertBodyEquals(HttpRequestFormatter requestFormatter, String expected) {
+ assertEquals(expected, requestFormatter.formatBody());
+ }
+
+ protected void assertHeadersEqual(HttpRequestFormatter requestFormatter, Map expected) {
+ assertEquals(expected, requestFormatter.formatHeaders());
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RetryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RetryTest.java
index b239f325cffa3..f2793b048b5ce 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RetryTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RetryTest.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.utils.MockTime;
diff --git a/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java b/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java
index 485146aea7ecb..9e353983e9852 100644
--- a/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java
+++ b/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java
@@ -17,372 +17,122 @@
package org.apache.kafka.tools;
-import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.config.SslConfigs;
-import org.apache.kafka.common.config.types.Password;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenRetriever;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenRetrieverFactory;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidator;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidatorFactory;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.VerificationKeyResolverFactory;
+import org.apache.kafka.common.security.JaasContext;
+import org.apache.kafka.common.security.oauthbearer.AccessTokenRetriever;
+import org.apache.kafka.common.security.oauthbearer.AccessTokenValidator;
+import org.apache.kafka.common.security.oauthbearer.DefaultAccessTokenRetriever;
+import org.apache.kafka.common.security.oauthbearer.DefaultAccessTokenValidator;
import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
import net.sourceforge.argparse4j.ArgumentParsers;
-import net.sourceforge.argparse4j.impl.Arguments;
-import net.sourceforge.argparse4j.inf.Argument;
import net.sourceforge.argparse4j.inf.ArgumentParser;
import net.sourceforge.argparse4j.inf.ArgumentParserException;
import net.sourceforge.argparse4j.inf.Namespace;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_CIPHER_SUITES_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_CIPHER_SUITES_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_ENABLED_PROTOCOLS_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_ENGINE_FACTORY_CLASS_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYMANAGER_ALGORITHM_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_KEY_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_KEY_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_LOCATION_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_PASSWORD_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_TYPE_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_TYPE_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEY_PASSWORD_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_KEY_PASSWORD_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_PROTOCOL_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_PROTOCOL_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_PROVIDER_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_PROVIDER_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_DOC;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG;
-import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_TYPE_DOC;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_DOC;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_DOC;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_DOC;
+import javax.security.auth.login.AppConfigurationEntry;
+
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
public class OAuthCompatibilityTool {
public static void main(String[] args) {
- ArgsHandler argsHandler = new ArgsHandler();
+ ArgumentParser parser = ArgumentParsers
+ .newArgumentParser("oauth-compatibility-tool")
+ .defaultHelp(true)
+ .description(
+ String.format(
+ "This tool is used to verify OAuth/OIDC provider compatibility.%n%n" +
+ "Run the following script to determine the configuration options:%n%n" +
+ " ./bin/kafka-run-class.sh %s --help",
+ OAuthCompatibilityTool.class.getName()
+ )
+ );
+ parser.addArgument("client-configuration-file")
+ .type(String.class)
+ .metavar("clientConfigurationFileName")
+ .dest("clientConfigurationFileName")
+ .help("Fully-qualified file name for the client configuration to use");
+ parser.addArgument("broker-configuration-file")
+ .type(String.class)
+ .metavar("brokerConfigurationFileName")
+ .dest("brokerConfigurationFileName")
+ .help("Fully-qualified file name for the broker configuration to use");
+
Namespace namespace;
try {
- namespace = argsHandler.parseArgs(args);
+ namespace = parser.parseArgs(args);
} catch (ArgumentParserException e) {
+ parser.handleError(e);
Exit.exit(1);
return;
}
- ConfigHandler configHandler = new ConfigHandler(namespace);
-
- Map configs = configHandler.getConfigs();
- Map jaasConfigs = configHandler.getJaasOptions();
-
try {
String accessToken;
- {
- // Client side...
- try (AccessTokenRetriever atr = AccessTokenRetrieverFactory.create(configs, jaasConfigs)) {
- atr.init();
- AccessTokenValidator atv = AccessTokenValidatorFactory.create(configs);
- System.out.println("PASSED 1/5: client configuration");
-
- accessToken = atr.retrieve();
- System.out.println("PASSED 2/5: client JWT retrieval");
-
- atv.validate(accessToken);
- System.out.println("PASSED 3/5: client JWT validation");
- }
+ // Client retrieval
+ try (AccessTokenRetriever retriever = new DefaultAccessTokenRetriever();
+ AccessTokenValidator validator = new DefaultAccessTokenValidator()) {
+ // Fill in the defaults for the values the user didn't specify.
+ ConfigDef cd = new ConfigDef();
+ SaslConfigs.addClientSaslSupport(cd);
+ SslConfigs.addClientSslSupport(cd);
+ String fileName = namespace.getString("clientConfigurationFileName");
+ Map configs = new AbstractConfig(
+ cd,
+ Utils.propsToMap(Utils.loadProps(fileName))
+ ).values();
+
+ JaasContext context = JaasContext.loadClientContext(configs);
+ List jaasConfigEntries = context.configurationEntries();
+
+ retriever.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries);
+ validator.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries);
+ System.out.println("PASSED 1/5: client configuration");
+
+ accessToken = retriever.retrieve();
+ System.out.println("PASSED 2/5: client JWT retrieval");
+
+ validator.validate(accessToken);
+ System.out.println("PASSED 3/5: client JWT validation");
}
- {
- // Broker side...
- try (CloseableVerificationKeyResolver vkr = VerificationKeyResolverFactory.create(configs, jaasConfigs)) {
- vkr.init();
- AccessTokenValidator atv = AccessTokenValidatorFactory.create(configs, vkr);
- System.out.println("PASSED 4/5: broker configuration");
+ // Broker validation
+ try (AccessTokenValidator validator = new DefaultAccessTokenValidator()) {
+ String fileName = namespace.getString("brokerConfigurationFileName");
+ Map configs = Utils.propsToMap(Utils.loadProps(fileName));
+ JaasContext context = JaasContext.loadClientContext(configs);
+ List jaasConfigEntries = context.configurationEntries();
- atv.validate(accessToken);
- System.out.println("PASSED 5/5: broker JWT validation");
- }
+ validator.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries);
+ System.out.println("PASSED 4/5: broker configuration");
+
+ validator.validate(accessToken);
+ System.out.println("PASSED 5/5: broker JWT validation");
}
System.out.println("SUCCESS");
Exit.exit(0);
} catch (Throwable t) {
System.out.println("FAILED:");
- t.printStackTrace();
+ t.printStackTrace(System.out);
if (t instanceof ConfigException) {
System.out.printf("%n");
- argsHandler.parser.printHelp();
+ parser.printHelp();
}
Exit.exit(1);
}
}
-
-
- private static class ArgsHandler {
-
- private static final String DESCRIPTION = String.format(
- "This tool is used to verify OAuth/OIDC provider compatibility.%n%n" +
- "Run the following script to determine the configuration options:%n%n" +
- " ./bin/kafka-run-class.sh %s --help",
- OAuthCompatibilityTool.class.getName());
-
- private final ArgumentParser parser;
-
- private ArgsHandler() {
- this.parser = ArgumentParsers
- .newArgumentParser("oauth-compatibility-tool")
- .defaultHelp(true)
- .description(DESCRIPTION);
- }
-
- private Namespace parseArgs(String[] args) throws ArgumentParserException {
- // SASL/OAuth
- addArgument(SASL_LOGIN_CONNECT_TIMEOUT_MS, SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC, Integer.class);
- addArgument(SASL_LOGIN_READ_TIMEOUT_MS, SASL_LOGIN_READ_TIMEOUT_MS_DOC, Integer.class);
- addArgument(SASL_LOGIN_RETRY_BACKOFF_MAX_MS, SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC, Long.class);
- addArgument(SASL_LOGIN_RETRY_BACKOFF_MS, SASL_LOGIN_RETRY_BACKOFF_MS_DOC, Long.class);
- addArgument(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC, Integer.class);
- addArgument(SASL_OAUTHBEARER_EXPECTED_AUDIENCE, SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC)
- .action(Arguments.append());
- addArgument(SASL_OAUTHBEARER_EXPECTED_ISSUER, SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC);
- addArgument(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC, Long.class);
- addArgument(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC, Long.class);
- addArgument(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC, Long.class);
- addArgument(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC);
- addArgument(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC);
- addArgument(SASL_OAUTHBEARER_SUB_CLAIM_NAME, SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC);
- addArgument(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC);
-
- // SSL
- addArgument(SSL_CIPHER_SUITES_CONFIG, SSL_CIPHER_SUITES_DOC)
- .action(Arguments.append());
- addArgument(SSL_ENABLED_PROTOCOLS_CONFIG, SSL_ENABLED_PROTOCOLS_DOC)
- .action(Arguments.append());
- addArgument(SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC);
- addArgument(SSL_ENGINE_FACTORY_CLASS_CONFIG, SSL_ENGINE_FACTORY_CLASS_DOC);
- addArgument(SSL_KEYMANAGER_ALGORITHM_CONFIG, SSL_KEYMANAGER_ALGORITHM_DOC);
- addArgument(SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG, SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC);
- addArgument(SSL_KEYSTORE_KEY_CONFIG, SSL_KEYSTORE_KEY_DOC);
- addArgument(SSL_KEYSTORE_LOCATION_CONFIG, SSL_KEYSTORE_LOCATION_DOC);
- addArgument(SSL_KEYSTORE_PASSWORD_CONFIG, SSL_KEYSTORE_PASSWORD_DOC);
- addArgument(SSL_KEYSTORE_TYPE_CONFIG, SSL_KEYSTORE_TYPE_DOC);
- addArgument(SSL_KEY_PASSWORD_CONFIG, SSL_KEY_PASSWORD_DOC);
- addArgument(SSL_PROTOCOL_CONFIG, SSL_PROTOCOL_DOC);
- addArgument(SSL_PROVIDER_CONFIG, SSL_PROVIDER_DOC);
- addArgument(SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, SSL_SECURE_RANDOM_IMPLEMENTATION_DOC);
- addArgument(SSL_TRUSTMANAGER_ALGORITHM_CONFIG, SSL_TRUSTMANAGER_ALGORITHM_DOC);
- addArgument(SSL_TRUSTSTORE_CERTIFICATES_CONFIG, SSL_TRUSTSTORE_CERTIFICATES_DOC);
- addArgument(SSL_TRUSTSTORE_LOCATION_CONFIG, SSL_TRUSTSTORE_LOCATION_DOC);
- addArgument(SSL_TRUSTSTORE_PASSWORD_CONFIG, SSL_TRUSTSTORE_PASSWORD_DOC);
- addArgument(SSL_TRUSTSTORE_TYPE_CONFIG, SSL_TRUSTSTORE_TYPE_DOC);
-
- // JAAS options...
- addArgument(CLIENT_ID_CONFIG, CLIENT_ID_DOC);
- addArgument(CLIENT_SECRET_CONFIG, CLIENT_SECRET_DOC);
- addArgument(SCOPE_CONFIG, SCOPE_DOC);
-
- try {
- return parser.parseArgs(args);
- } catch (ArgumentParserException e) {
- parser.handleError(e);
- throw e;
- }
- }
-
- private Argument addArgument(String option, String help) {
- return addArgument(option, help, String.class);
- }
-
- private Argument addArgument(String option, String help, Class> clazz) {
- // Change foo.bar into --foo.bar.
- String name = "--" + option;
-
- return parser.addArgument(name)
- .type(clazz)
- .metavar(option)
- .dest(option)
- .help(help);
- }
-
- }
-
- private static class ConfigHandler {
-
- private final Namespace namespace;
-
-
- private ConfigHandler(Namespace namespace) {
- this.namespace = namespace;
- }
-
- private Map getConfigs() {
- Map m = new HashMap<>();
-
- // SASL/OAuth
- maybeAddInt(m, SASL_LOGIN_CONNECT_TIMEOUT_MS);
- maybeAddInt(m, SASL_LOGIN_READ_TIMEOUT_MS);
- maybeAddLong(m, SASL_LOGIN_RETRY_BACKOFF_MS);
- maybeAddLong(m, SASL_LOGIN_RETRY_BACKOFF_MAX_MS);
- maybeAddString(m, SASL_OAUTHBEARER_SCOPE_CLAIM_NAME);
- maybeAddString(m, SASL_OAUTHBEARER_SUB_CLAIM_NAME);
- maybeAddString(m, SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL);
- maybeAddString(m, SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
- maybeAddLong(m, SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS);
- maybeAddLong(m, SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS);
- maybeAddLong(m, SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS);
- maybeAddInt(m, SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS);
- maybeAddStringList(m, SASL_OAUTHBEARER_EXPECTED_AUDIENCE);
- maybeAddString(m, SASL_OAUTHBEARER_EXPECTED_ISSUER);
-
- // This here is going to fill in all the defaults for the values we don't specify...
- ConfigDef cd = new ConfigDef();
- SaslConfigs.addClientSaslSupport(cd);
- SslConfigs.addClientSslSupport(cd);
- AbstractConfig config = new AbstractConfig(cd, m);
- return config.values();
- }
-
- private Map getJaasOptions() {
- Map m = new HashMap<>();
-
- // SASL/OAuth
- maybeAddString(m, CLIENT_ID_CONFIG);
- maybeAddString(m, CLIENT_SECRET_CONFIG);
- maybeAddString(m, SCOPE_CONFIG);
-
- // SSL
- maybeAddStringList(m, SSL_CIPHER_SUITES_CONFIG);
- maybeAddStringList(m, SSL_ENABLED_PROTOCOLS_CONFIG);
- maybeAddString(m, SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG);
- maybeAddClass(m, SSL_ENGINE_FACTORY_CLASS_CONFIG);
- maybeAddString(m, SSL_KEYMANAGER_ALGORITHM_CONFIG);
- maybeAddPassword(m, SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG);
- maybeAddPassword(m, SSL_KEYSTORE_KEY_CONFIG);
- maybeAddString(m, SSL_KEYSTORE_LOCATION_CONFIG);
- maybeAddPassword(m, SSL_KEYSTORE_PASSWORD_CONFIG);
- maybeAddString(m, SSL_KEYSTORE_TYPE_CONFIG);
- maybeAddPassword(m, SSL_KEY_PASSWORD_CONFIG);
- maybeAddString(m, SSL_PROTOCOL_CONFIG);
- maybeAddString(m, SSL_PROVIDER_CONFIG);
- maybeAddString(m, SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG);
- maybeAddString(m, SSL_TRUSTMANAGER_ALGORITHM_CONFIG);
- maybeAddPassword(m, SSL_TRUSTSTORE_CERTIFICATES_CONFIG);
- maybeAddString(m, SSL_TRUSTSTORE_LOCATION_CONFIG);
- maybeAddPassword(m, SSL_TRUSTSTORE_PASSWORD_CONFIG);
- maybeAddString(m, SSL_TRUSTSTORE_TYPE_CONFIG);
-
- return m;
- }
-
- private void maybeAddInt(Map m, String option) {
- Integer value = namespace.getInt(option);
-
- if (value != null)
- m.put(option, value);
- }
-
- private void maybeAddLong(Map m, String option) {
- Long value = namespace.getLong(option);
-
- if (value != null)
- m.put(option, value);
- }
-
- private void maybeAddString(Map m, String option) {
- String value = namespace.getString(option);
-
- if (value != null)
- m.put(option, value);
- }
-
- private void maybeAddPassword(Map m, String option) {
- String value = namespace.getString(option);
-
- if (value != null)
- m.put(option, new Password(value));
- }
-
- private void maybeAddClass(Map m, String option) {
- String value = namespace.getString(option);
-
- if (value != null) {
- try {
- m.put(option, Class.forName(value));
- } catch (ClassNotFoundException e) {
- throw new KafkaException("Could not find class for " + option, e);
- }
- }
- }
-
- private void maybeAddStringList(Map m, String option) {
- List value = namespace.getList(option);
-
- if (value != null)
- m.put(option, value);
- }
-
- }
-
}