diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 6dc6d470c9dc7..d35d0933cff25 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -143,6 +143,7 @@ + diff --git a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java index 15e23270d6f0c..2e6109648d82c 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -18,6 +18,8 @@ import org.apache.kafka.common.config.ConfigDef.Range; +import java.util.List; + public class SaslConfigs { private static final String OAUTHBEARER_NOTE = " Currently applies only to OAUTHBEARER."; @@ -129,6 +131,16 @@ public class SaslConfigs { + " authentication provider." + LOGIN_EXPONENTIAL_BACKOFF_NOTE; + + + + + + + + + + public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME = "sasl.oauthbearer.scope.claim.name"; public static final String DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME = "scope"; public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC = "The OAuth claim for the scope is often named \"" + DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME + "\", but this (optional)" @@ -141,6 +153,16 @@ public class SaslConfigs { + " setting can provide a different name to use for the subject included in the JWT payload's claims if the OAuth/OIDC provider uses a different" + " name for that claim."; + public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPE = "sasl.oauthbearer.token.endpoint.grant.type"; + public static final String DEFAULT_SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPE = "client_credentials"; + public static final List SUPPORTED_SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPES = List.of( + "client_credentials", + "urn:ietf:params:oauth:grant-type:jwt-bearer" + ); + public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPE_DOC = "The grant type used when sending the JWT token to the token endpoint. " + + "This should be set explicitly to determine which token retriever to use. The supported values are " + + SUPPORTED_SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPES; + public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL = "sasl.oauthbearer.token.endpoint.url"; public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC = "The URL for the OAuth/OIDC identity provider. If the URL is HTTP(S)-based, it is the issuer's token" + " endpoint URL to which requests will be made to login based on the configuration in " + SASL_JAAS_CONFIG + ". If the URL is file-based, it" @@ -217,6 +239,7 @@ public static void addClientSaslSupport(ConfigDef config) { .define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS, ConfigDef.Type.LONG, DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS, ConfigDef.Importance.LOW, SASL_LOGIN_RETRY_BACKOFF_MS_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, ConfigDef.Type.STRING, DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME, ConfigDef.Type.STRING, DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPE, ConfigDef.Type.STRING, DEFAULT_SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPE, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPE_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, ConfigDef.Type.LONG, DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AccessTokenRetriever.java new file mode 100644 index 0000000000000..8a3dc7199e7a9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AccessTokenRetriever.java @@ -0,0 +1,64 @@ +/* + * 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; + +import java.io.IOException; + +import javax.security.auth.spi.LoginModule; + +/** + * An implementation of AccessTokenRetriever is the means by which the login module will + * retrieve an OAuth access token that is used to authorize with a broker. The implementation may + * involve authentication to one or more remote systems, or it can be as simple as loading the contents + * from a file or configuration setting. + * + * Retrieval of a token is a separate concern from validation. + * AccessTokenRetriever implementations should not validate the integrity of the access + * token, but should rely on the companion {@link AccessTokenValidator} for that task. + * + * @see ClientCredentialsAccessTokenRetriever + * @see DefaultAccessTokenRetriever + * @see FileAccessTokenRetriever + * @see HttpAccessTokenRetriever + * @see JwtBearerAccessTokenRetriever + */ +public interface AccessTokenRetriever extends OAuthBearerConfigurable { + + /** + *

+ * 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 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 I/O during retrieval + */ + String retrieve() throws IOException; + + @Override + default void close() { + // Do nothing... + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AccessTokenValidator.java similarity index 77% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidator.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AccessTokenValidator.java index 0b107a09bc065..1ff085eea73e3 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AccessTokenValidator.java @@ -14,10 +14,7 @@ * 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; +package org.apache.kafka.common.security.oauthbearer; /** * An instance of AccessTokenValidator acts as a function object that, given an access @@ -40,13 +37,13 @@ *
  • RFC 6750, Section 2.1
  • * * - * @see LoginAccessTokenValidator A basic AccessTokenValidator used by client-side login - * authentication - * @see ValidatorAccessTokenValidator A more robust AccessTokenValidator that is used on the broker + * @see DefaultAccessTokenValidator Default validator that acts as a wrapper over one of the other validators + * @see ClientAccessTokenValidator A basic AccessTokenValidator used by client-side login + * authentication + * @see BrokerAccessTokenValidator A more robust AccessTokenValidator that is used on the broker * to validate the token's contents and verify the signature */ - -public interface AccessTokenValidator { +public interface AccessTokenValidator extends OAuthBearerConfigurable { /** * Accepts an OAuth JWT access token in base-64 encoded format, validates, and returns an @@ -56,9 +53,13 @@ public interface AccessTokenValidator { * * @return {@link OAuthBearerToken} * - * @throws ValidateException Thrown on errors performing validation of given token + * @throws InvalidJwtException Thrown on errors performing validation of given token */ - OAuthBearerToken validate(String accessToken) throws ValidateException; + OAuthBearerToken validate(String accessToken) throws InvalidJwtException; + @Override + default void close() { + // Do nothing... + } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerAccessTokenValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerAccessTokenValidator.java new file mode 100644 index 0000000000000..81d479dabdb1f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerAccessTokenValidator.java @@ -0,0 +1,318 @@ +/* + * 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; + +import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; +import org.apache.kafka.common.security.oauthbearer.internals.secured.BasicOAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClaimValidationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DelegatingVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.RefreshingHttpsJwksVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.SerializedJwt; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; + +import org.jose4j.jws.JsonWebSignature; +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.JwtConsumer; +import org.jose4j.jwt.consumer.JwtConsumerBuilder; +import org.jose4j.jwt.consumer.JwtContext; +import org.jose4j.jwx.JsonWebStructure; +import org.jose4j.lang.UnresolvableKeyException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.Key; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import javax.security.auth.login.AppConfigurationEntry; + +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; +import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE; + +/** + * 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: + * + *
      + *
    1. + * Basic structural validation of the b64token value as defined in + * RFC 6750 Section 2.1 + *
    2. + *
    3. Basic conversion of the token into an in-memory data structure
    4. + *
    5. + * Presence of scope, exp, subject, iss, and + * iat claims + *
    6. + *
    7. + * Signature matching validation against the kid and those provided by + * the OAuth/OIDC provider's JWKS + *
    8. + *
    + */ +public class BrokerAccessTokenValidator implements AccessTokenValidator { + + private static final Logger log = LoggerFactory.getLogger(BrokerAccessTokenValidator.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 final Time time; + + private CloseableVerificationKeyResolver verificationKeyResolver; + + private JwtConsumer jwtConsumer; + + private String scopeClaimName; + + private String subClaimName; + + public BrokerAccessTokenValidator() { + this(Time.SYSTEM); + } + + public BrokerAccessTokenValidator(Time time) { + this.time = time; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + CloseableVerificationKeyResolver resolver; + + // Here's the logic which keeps our VerificationKeyResolvers down to a single instance. + synchronized (VERIFICATION_KEY_RESOLVER_CACHE) { + VerificationKeyResolverKey key = new VerificationKeyResolverKey(configs, jaasConfigEntries); + resolver = VERIFICATION_KEY_RESOLVER_CACHE.computeIfAbsent( + key, + k -> new RefCountingVerificationKeyResolver(new DelegatingVerificationKeyResolver(time)) + ); + } + + configure(resolver, configs, saslMechanism, jaasConfigEntries); + } + + void configure(CloseableVerificationKeyResolver verificationKeyResolver, + Map configs, + String saslMechanism, + List jaasConfigEntries) { + this.verificationKeyResolver = verificationKeyResolver; + this.verificationKeyResolver.configure(configs, saslMechanism, jaasConfigEntries); + + 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); + + 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; + } + + @Override + public void close() { + Utils.closeQuietly(verificationKeyResolver, "verificationKeyResolver"); + } + + /** + * 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 InvalidJwtException Thrown on errors performing validation of given token + */ + @SuppressWarnings("unchecked") + public OAuthBearerToken validate(String accessToken) throws InvalidJwtException { + SerializedJwt serializedJwt = new SerializedJwt(accessToken); + + JwtContext jwt; + + try { + jwt = jwtConsumer.process(serializedJwt.getToken()); + } catch (org.jose4j.jwt.consumer.InvalidJwtException e) { + throw new InvalidJwtException(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 InvalidJwtException { + try { + T value = supplier.get(); + log.debug("getClaim - {}: {}", claimName, value); + return value; + } catch (MalformedClaimException e) { + throw new InvalidJwtException(String.format("Could not extract the '%s' claim from the access token", claimName), e); + } + } + + public interface ClaimSupplier { + + T get() throws MalformedClaimException; + + } + /** + * VerificationKeyResolverKey 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; + + // The equality of two lists cannot be determined with AppConfigurationEntry directly since + // that class does not implement hashCode() or equals(). So the JAAS options from the + // AppConfigurationEntry entries are extracted for comparison purposes. + private final List> jaasOptions; + + public VerificationKeyResolverKey(Map configs, List jaasConfigEntries) { + this.configs = configs; + this.jaasOptions = jaasConfigEntries.stream() + .map(AppConfigurationEntry::getOptions) + .collect(Collectors.toList()); + } + + @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) && jaasOptions.equals(that.jaasOptions); + } + + @Override + public int hashCode() { + return Objects.hash(configs, jaasOptions); + } + } + + /** + * 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 = Objects.requireNonNull(delegate); + } + + @Override + public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException { + return delegate.resolveKey(jws, nestingContext); + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + if (count.incrementAndGet() == 1) + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @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/LoginAccessTokenValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientAccessTokenValidator.java similarity index 70% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidator.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientAccessTokenValidator.java index 773311ff0ab18..ccbf35677cd7b 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientAccessTokenValidator.java @@ -14,10 +14,12 @@ * 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.OAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.BasicOAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClaimValidationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.SerializedJwt; import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerIllegalTokenException; import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredJws; @@ -26,11 +28,16 @@ import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Set; +import javax.security.auth.login.AppConfigurationEntry; + import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME; import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME; +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; /** * LoginAccessTokenValidator is an implementation of {@link AccessTokenValidator} that is used @@ -49,30 +56,29 @@ *
  • Presence of scope, exp, subject, and iat claims
  • * */ +public class ClientAccessTokenValidator implements AccessTokenValidator { -public class LoginAccessTokenValidator implements AccessTokenValidator { - - private static final Logger log = LoggerFactory.getLogger(LoginAccessTokenValidator.class); + private static final Logger log = LoggerFactory.getLogger(ClientAccessTokenValidator.class); public static final String EXPIRATION_CLAIM_NAME = "exp"; public static final String ISSUED_AT_CLAIM_NAME = "iat"; - private final String scopeClaimName; - - private final String subClaimName; - - /** - * Creates a new LoginAccessTokenValidator that will be used by the client for lightweight - * validation of the JWT. - * - * @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 - */ - - public LoginAccessTokenValidator(String scopeClaimName, String subClaimName) { - this.scopeClaimName = ClaimValidationUtils.validateClaimNameOverride(DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, scopeClaimName); - this.subClaimName = ClaimValidationUtils.validateClaimNameOverride(DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, subClaimName); + private String scopeClaimName; + + private String subClaimName; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + scopeClaimName = ClaimValidationUtils.validateClaimNameOverride( + DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, + cu.get(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME) + ); + subClaimName = ClaimValidationUtils.validateClaimNameOverride( + DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, + cu.get(SASL_OAUTHBEARER_SUB_CLAIM_NAME) + ); } /** @@ -81,18 +87,18 @@ public LoginAccessTokenValidator(String scopeClaimName, String subClaimName) { * * @param accessToken Non-null JWT access token * @return {@link OAuthBearerToken} - * @throws ValidateException Thrown on errors performing validation of given token + * @throws InvalidJwtException Thrown on errors performing validation of given token */ @SuppressWarnings("unchecked") - public OAuthBearerToken validate(String accessToken) throws ValidateException { + public OAuthBearerToken validate(String accessToken) throws InvalidJwtException { SerializedJwt serializedJwt = new SerializedJwt(accessToken); Map payload; try { payload = OAuthBearerUnsecuredJws.toMap(serializedJwt.getPayload()); } catch (OAuthBearerIllegalTokenException e) { - throw new ValidateException(String.format("Could not validate the access token: %s", e.getMessage()), e); + throw new InvalidJwtException(String.format("Could not validate the access token: %s", e.getMessage()), e); } Object scopeRaw = getClaim(payload, scopeClaimName); @@ -116,11 +122,13 @@ else if (scopeRaw instanceof Collection) Long issuedAt = ClaimValidationUtils.validateIssuedAt(ISSUED_AT_CLAIM_NAME, issuedAtRaw != null ? issuedAtRaw.longValue() * 1000L : null); - return new BasicOAuthBearerToken(accessToken, + return new BasicOAuthBearerToken( + accessToken, scopes, expiration, subject, - issuedAt); + issuedAt + ); } private Object getClaim(Map payload, String claimName) { @@ -128,5 +136,4 @@ private Object getClaim(Map payload, String claimName) { log.debug("getClaim - {}: {}", claimName, value); return value; } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsAccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsAccessTokenRetriever.java new file mode 100644 index 0000000000000..7b0f5e79ab75c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsAccessTokenRetriever.java @@ -0,0 +1,86 @@ +/* + * 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; + +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClientCredentialsRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import javax.security.auth.login.AppConfigurationEntry; + +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; + +/** + * HttpAccessTokenRetriever is an {@link AccessTokenRetriever} that will + * communicate with an OAuth/OIDC provider directly via HTTP to post client credentials + * ({@link #CLIENT_ID_CONFIG}/{@link #CLIENT_SECRET_CONFIG}) + * to a publicized token endpoint URL + * ({@link SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL}). + * + * @see AccessTokenRetriever + * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL + */ + +public class ClientCredentialsAccessTokenRetriever extends HttpAccessTokenRetriever { + + private static final String CLIENT_ID_CONFIG = "clientId"; + private static final String CLIENT_SECRET_CONFIG = "clientSecret"; + private static final String SCOPE_CONFIG = "scope"; + + private HttpRequestFormatter requestFormatter; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + super.configure(configs, saslMechanism, jaasConfigEntries); + + JaasOptionsUtils jou = new JaasOptionsUtils(saslMechanism, jaasConfigEntries); + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + String clientId = jou.validateString(CLIENT_ID_CONFIG); + String clientSecret = jou.validateString(CLIENT_SECRET_CONFIG); + String scope = jou.validateString(SCOPE_CONFIG, false); + boolean urlencodeHeader = validateUrlencodeHeader(cu); + requestFormatter = new ClientCredentialsRequestFormatter(clientId, clientSecret, scope, urlencodeHeader); + } + + @Override + protected HttpRequestFormatter requestFormatter() { + return requestFormatter; + } + + /** + * 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 ClientCredentialsAccessTokenRetriever} constructor. + */ + public static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) { + Boolean urlencodeHeader = configurationUtils.validateBoolean(SASL_OAUTHBEARER_HEADER_URLENCODE, false); + return Objects.requireNonNullElse(urlencodeHeader, DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenRetriever.java new file mode 100644 index 0000000000000..57392735db684 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenRetriever.java @@ -0,0 +1,72 @@ +/* + * 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; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.utils.Utils; + +import java.io.IOException; +import java.net.URL; +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_TOKEN_ENDPOINT_GRANT_TYPE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBearerRequestFormatter.GRANT_TYPE; + +public class DefaultAccessTokenRetriever implements AccessTokenRetriever { + + private AccessTokenRetriever delegate; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + URL tokenEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + + if (tokenEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file")) { + delegate = new FileAccessTokenRetriever(); + } else { + String grantType = cu.validateString(SASL_OAUTHBEARER_TOKEN_ENDPOINT_GRANT_TYPE, false); + + if (grantType != null && grantType.equalsIgnoreCase(GRANT_TYPE)) { + delegate = new JwtBearerAccessTokenRetriever(); + } else { + delegate = new ClientCredentialsAccessTokenRetriever(); + } + } + + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public String retrieve() throws IOException { + return Objects.requireNonNull(delegate).retrieve(); + } + + @Override + public void close() { + Utils.closeQuietly(delegate, "delegate"); + } + + public AccessTokenRetriever delegate() { + return delegate; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenValidator.java new file mode 100644 index 0000000000000..afad4d5794ae6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultAccessTokenValidator.java @@ -0,0 +1,87 @@ +/* + * 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; + +import org.apache.kafka.common.utils.Utils; + +import java.util.List; +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; + +/** + * 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: + * + *

      + *
    1. + * Basic structural validation of the b64token value as defined in + * RFC 6750 Section 2.1 + *
    2. + *
    3. Basic conversion of the token into an in-memory data structure
    4. + *
    5. + * Presence of scope, exp, subject, iss, and + * iat claims + *
    6. + *
    7. + * Signature matching validation against the kid and those provided by + * the OAuth/OIDC provider's JWKS + *
    8. + *
    + */ + +public class DefaultAccessTokenValidator implements AccessTokenValidator { + + private AccessTokenValidator delegate; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + AccessTokenValidator validator; + + if (configs.get(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL) != null) + validator = new BrokerAccessTokenValidator(); + else + validator = new ClientAccessTokenValidator(); + + configure(validator, configs, saslMechanism, jaasConfigEntries); + } + + void configure(AccessTokenValidator validator, + Map configs, + String saslMechanism, + List jaasConfigEntries) { + delegate = validator; + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public OAuthBearerToken validate(String accessToken) throws InvalidJwtException { + return Objects.requireNonNull(delegate).validate(accessToken); + } + + @Override + public void close() { + Utils.closeQuietly(delegate, "delegate"); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileAccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileAccessTokenRetriever.java new file mode 100644 index 0000000000000..c85ac700e5661 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileAccessTokenRetriever.java @@ -0,0 +1,61 @@ +/* + * 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; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.utils.Utils; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; + +/** + * FileAccessTokenRetriever is an {@link AccessTokenRetriever} that will load the contents of a file, + * interpreting them as a JWT access key in the serialized form. + * + * @see AccessTokenRetriever + */ +public class FileAccessTokenRetriever implements AccessTokenRetriever { + + private String accessToken; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + File accessTokenFileName = cu.validateFile(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + + try { + String fileContents = Utils.readFileAsString(accessTokenFileName.getPath()); + // always non-null; to remove any newline chars or backend will report err + accessToken = fileContents.trim(); + } catch (Exception e) { + throw new KafkaException("An error occurred reading the OAuth token from " + accessTokenFileName); + } + } + + @Override + public String retrieve() throws IOException { + return Objects.requireNonNull(accessToken, "Access token is null; please call configure() first"); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/HttpAccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/HttpAccessTokenRetriever.java new file mode 100644 index 0000000000000..e9f3f3b760433 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/HttpAccessTokenRetriever.java @@ -0,0 +1,275 @@ +/* + * 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; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ConfigurationUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.Retry; +import org.apache.kafka.common.security.oauthbearer.internals.secured.SslResource; +import org.apache.kafka.common.security.oauthbearer.internals.secured.UnretryableException; +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.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; + +import javax.security.auth.login.AppConfigurationEntry; + +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_TOKEN_ENDPOINT_URL; + +/** + * HttpAccessTokenRetriever is an {@link AccessTokenRetriever} that will + * communicate with an OAuth/OIDC provider directly via HTTP. + * + * @see AccessTokenRetriever + */ +public abstract 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; + + private String tokenEndpointUrl; + + private long retryBackoffMs; + + private long retryBackoffMaxMs; + + private HttpClient client; + + 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); + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + JaasOptionsUtils jou = new JaasOptionsUtils(saslMechanism, jaasConfigEntries); + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + + URL url = cu.validateUrl(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + tokenEndpointUrl = url.toString(); + retryBackoffMs = cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MS); + retryBackoffMaxMs = cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS); + + Optional sslResource = jou.maybeCreateSslResource(url); + Optional connectTimeoutMs = Optional.ofNullable(cu.validateInteger(SASL_LOGIN_CONNECT_TIMEOUT_MS, false)); + Optional readTimeoutMs = Optional.ofNullable(cu.validateInteger(SASL_LOGIN_READ_TIMEOUT_MS, false)); + + client = new HttpClient(tokenEndpointUrl, sslResource, connectTimeoutMs, readTimeoutMs); + } + + protected abstract HttpRequestFormatter requestFormatter(); + + /** + * 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 { + HttpRequestFormatter requestFormatter = requestFormatter(); + byte[] requestBody = Utils.utf8(requestFormatter.formatBody()); + Map headers = requestFormatter.formatHeaders(); + + Retry retry = new Retry<>(retryBackoffMs, retryBackoffMaxMs); + String responseBody; + + try { + responseBody = retry.execute(() -> { + HttpURLConnection con = null; + + try { + con = client.connect(tokenEndpointUrl, headers, Optional.of(requestBody.length)); + HttpClient.HttpResponse httpResponse = client.post(headers, requestBody); + return handleOutput(tokenEndpointUrl, httpResponse); + } 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); + } + + static String handleOutput(String url, HttpClient.HttpResponse httpResponse) throws IOException { + int responseCode = httpResponse.responseCode; + Optional responseBodyOpt = httpResponse.responseBody.map(b -> new String(b, StandardCharsets.UTF_8)); + String errorMessage = formatErrorMessage( + httpResponse.errorResponseBody.map(b -> new String(b, StandardCharsets.UTF_8)) + ); + + if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) { + if (responseBodyOpt.isPresent()) + return responseBodyOpt.get(); + + throw new IOException( + String.format( + "The token endpoint response was unexpectedly empty despite response code %d from %s and error message %s", + responseCode, + url, + errorMessage + ) + ); + } + + log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, errorMessage); + + 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, + errorMessage + ) + ) + ); + } 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, + errorMessage + ) + ); + } + } + + static String formatErrorMessage(Optional errorResponseBodyOpt) { + // See https://www.ietf.org/rfc/rfc6749.txt, section 5.2 for the format + // of this error message. + if (errorResponseBodyOpt.isEmpty()) { + return "{}"; + } + String errorResponseBody = errorResponseBodyOpt.get(); + 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)); + } + + String name = "the token endpoint response's access_token JSON attribute"; + String value = accessTokenNode.textValue(); + + 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/main/java/org/apache/kafka/common/security/oauthbearer/HttpClient.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/HttpClient.java new file mode 100644 index 0000000000000..0c4d0e866a771 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/HttpClient.java @@ -0,0 +1,188 @@ +/* + * 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; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.SslResource; +import org.apache.kafka.common.security.oauthbearer.internals.secured.UnretryableException; +import org.apache.kafka.common.utils.Utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.Map; +import java.util.Optional; + +import javax.net.ssl.HttpsURLConnection; + +/** + * HttpClient is a lightweight client that can be used by callback handlers to + * communicate with an OAuth/OIDC provider directly via HTTP. + */ +public class HttpClient implements Closeable { + + private static final Logger log = LoggerFactory.getLogger(HttpClient.class); + + private final String url; + + private final Optional sslResource; + + private final Optional connectTimeoutMs; + + private final Optional readTimeoutMs; + + public HttpClient(String url) { + this(url, Optional.empty(), Optional.empty(), Optional.empty()); + } + + public HttpClient(String url, + Optional sslResource, + Optional connectTimeoutMs, + Optional readTimeoutMs) { + this.url = url; + this.sslResource = sslResource; + this.connectTimeoutMs = connectTimeoutMs; + this.readTimeoutMs = readTimeoutMs; + } + + @Override + public void close() throws IOException { + sslResource.ifPresent(r -> Utils.closeQuietly(r, "sslResource")); + } + + public HttpURLConnection connect(String url, + Map headers, + Optional contentLength) throws IOException, UnretryableException { + log.debug("connect - starting connect for {}", url); + + HttpURLConnection con = (HttpURLConnection) new URL(url).openConnection(); + + if (sslResource.isPresent() && con instanceof HttpsURLConnection) + ((HttpsURLConnection) con).setSSLSocketFactory(sslResource.get().sslSocketFactory()); + + if (contentLength.isPresent()) { + con.setRequestMethod("POST"); + con.setDoOutput(true); + headers.put("Content-Length", String.valueOf(contentLength.get())); + } else { + con.setRequestMethod("GET"); + con.setDoOutput(false); + } + + for (Map.Entry header : headers.entrySet()) + con.setRequestProperty(header.getKey(), header.getValue()); + + headers.put("Accept", "application/json"); + headers.put("Cache-Control", "no-cache"); + + con.setUseCaches(false); + connectTimeoutMs.ifPresent(con::setConnectTimeout); + readTimeoutMs.ifPresent(con::setReadTimeout); + + log.debug("connect - preparing to connect to {}", con.getURL()); + con.connect(); + return con; + } + + public HttpResponse post(Map requestHeaders, byte[] requestBody) throws IOException { + HttpURLConnection con = null; + + try { + con = connect(url, requestHeaders, Optional.of(requestBody.length)); + write(con, requestBody); + return read(con); + } finally { + if (con != null) + con.disconnect(); + } + } + + public void write(HttpURLConnection con, byte[] requestBody) throws IOException { + try (OutputStream os = con.getOutputStream()) { + ByteArrayInputStream is = new ByteArrayInputStream(requestBody); + log.trace("write - preparing to write request body to {}", con.getURL()); + copy(is, os); + } + } + + public HttpResponse read(HttpURLConnection con) throws IOException { + int responseCode = con.getResponseCode(); + log.debug("read - responseCode: {}", responseCode); + + // NOTE: the contents of the response should not be logged so that we don't leak any + // sensitive data. + byte[] 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 + byte[] errorResponseBody = null; + + try (InputStream is = con.getInputStream()) { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + log.debug("read - preparing to read response body from {}", con.getURL()); + copy(is, os); + responseBody = os.toByteArray(); + } catch (Exception e) { + // There still can be useful error response from the servers, lets get it from the error stream. + try (InputStream is = con.getErrorStream()) { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + log.debug("read - preparing to read error response body from {}", con.getURL()); + copy(is, os); + errorResponseBody = os.toByteArray(); + } catch (Exception e2) { + log.warn("read - error retrieving error information", e2); + } + + log.warn("read - error retrieving data", e); + } + + Optional responseBodyOpt = responseBody != null && responseBody.length > 0 ? Optional.of(responseBody) : Optional.empty(); + Optional errorResponseBodyOpt = errorResponseBody != null && errorResponseBody.length > 0 ? Optional.of(errorResponseBody) : Optional.empty(); + return new HttpResponse(responseCode, responseBodyOpt, errorResponseBodyOpt); + } + + 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); + } + + public static class HttpResponse { + + public final int responseCode; + + public final Optional responseBody; + + public final Optional errorResponseBody; + + public HttpResponse(int responseCode, Optional responseBody, Optional errorResponseBody) { + this.responseCode = responseCode; + this.responseBody = responseBody; + this.errorResponseBody = errorResponseBody; + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidateException.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/InvalidJwtException.java similarity index 82% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidateException.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/InvalidJwtException.java index 430b9007830cb..7dbfb6f4c7f76 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidateException.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/InvalidJwtException.java @@ -14,8 +14,7 @@ * 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.KafkaException; @@ -29,19 +28,17 @@ * * @see AccessTokenValidator#validate(String) */ +public class InvalidJwtException extends KafkaException { -public class ValidateException extends KafkaException { - - public ValidateException(String message) { + public InvalidJwtException(String message) { super(message); } - public ValidateException(Throwable cause) { + public InvalidJwtException(Throwable cause) { super(cause); } - public ValidateException(String message, Throwable cause) { + public InvalidJwtException(String message, Throwable cause) { super(message, cause); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerAccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerAccessTokenRetriever.java new file mode 100644 index 0000000000000..015ba7957452f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerAccessTokenRetriever.java @@ -0,0 +1,95 @@ +/* + * 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; + +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpRequestFormatter; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBearerRequestFormatter; +import org.apache.kafka.common.utils.Time; + +import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; + +public class JwtBearerAccessTokenRetriever extends HttpAccessTokenRetriever { + + // The private key ID of the private key used to sign the JWT token sent to the token endpoint. This will + // be added as a header in the JWT token sent to the token endpoint. + private static final String TOKEN_ENDPOINT_PRIVATE_KEY_ID = "privateKeyId"; + + // The private key used to sign the JWT token sent to the token endpoint. This must be in PEM format without + // the header and footer. + private static final String TOKEN_ENDPOINT_PRIVATE_KEY_SECRET = "privateKeySecret"; + + // The algorithm used to sign the JWT token sent to the token endpoint. + private static final String TOKEN_ENDPOINT_SIGNING_ALGO = "tokenSigningAlgo"; + + // The subject of the JWT token sent to the token endpoint. + private static final String TOKEN_SUBJECT = "tokenSubject"; + + // The issuer of the JWT token sent to the token endpoint. + private static final String TOKEN_ISSUER = "tokenIssuer"; + + // The audience of the JWT token sent to the token endpoint. + private static final String TOKEN_AUDIENCE = "tokenAudience"; + + // The target audience of the JWT token sent to the token endpoint. + private static final String TOKEN_TARGET_AUDIENCE = "tokenTargetAudience"; + + private final Time time; + + private JwtBearerRequestFormatter requestFormatter; + + public JwtBearerAccessTokenRetriever() { + this(Time.SYSTEM); + } + + public JwtBearerAccessTokenRetriever(Time time) { + this.time = time; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + super.configure(configs, saslMechanism, jaasConfigEntries); + + JaasOptionsUtils jou = new JaasOptionsUtils(saslMechanism, jaasConfigEntries); + String privateKeyId = jou.validateString(TOKEN_ENDPOINT_PRIVATE_KEY_ID); + String privateKeySecret = jou.validateString(TOKEN_ENDPOINT_PRIVATE_KEY_SECRET); + String tokenSigningAlgo = jou.validateString(TOKEN_ENDPOINT_SIGNING_ALGO); + String tokenSubject = jou.validateString(TOKEN_SUBJECT); + String tokenIssuer = jou.validateString(TOKEN_ISSUER); + String tokenAudience = jou.validateString(TOKEN_AUDIENCE); + String tokenTargetAudience = jou.validateString(TOKEN_TARGET_AUDIENCE, false); + + requestFormatter = new JwtBearerRequestFormatter( + time, + privateKeyId, + privateKeySecret, + tokenSigningAlgo, + tokenSubject, + tokenIssuer, + tokenAudience, + tokenTargetAudience + ); + } + + @Override + protected HttpRequestFormatter requestFormatter() { + return requestFormatter; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerConfigurable.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerConfigurable.java new file mode 100644 index 0000000000000..5d0d72982fd10 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerConfigurable.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; + +import org.apache.kafka.common.Configurable; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; + +/** + * Analogue to {@link Configurable} for SASL-based authentication. + * + * Any resources created in {@link #configure(Map, String, List)} should be cleaned up and released in + * the call to {@link #close()}. + */ +public interface OAuthBearerConfigurable extends Closeable { + + /** + * Configures this object for the specified SASL mechanism. + * + * @param configs Key-value pairs containing the parsed configuration options of + * the client or broker. Note that these are the Kafka configuration options + * and not the JAAS configuration options. JAAS config options may be obtained + * from `jaasConfigEntries`. For configs that may be specified as both Kafka config + * as well as JAAS config (e.g. sasl.kerberos.service.name), the configuration + * is treated as invalid if conflicting values are provided. + * @param saslMechanism Negotiated SASL mechanism. For clients, this is the SASL + * mechanism configured for the client. For brokers, this is the mechanism + * negotiated with the client and is one of the mechanisms enabled on the broker. + * @param jaasConfigEntries JAAS configuration entries from the JAAS login context. + * This list contains a single entry for clients and may contain more than + * one entry for brokers if multiple mechanisms are enabled on a listener using + * static JAAS configuration where there is no mapping between mechanisms and + * login module entries. In this case, implementations can use the login module in + * `jaasConfigEntries` to identify the entry corresponding to `saslMechanism`. + * Alternatively, dynamic JAAS configuration option + * {@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG} may be + * configured on brokers with listener and mechanism prefix, in which case + * only the configuration entry corresponding to `saslMechanism` will be provided + * in `jaasConfigEntries`. + */ + void configure(Map configs, String saslMechanism, List jaasConfigEntries); + +// void close(); +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandler.java index fc9e689611520..75ba7ae1e7e60 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandler.java @@ -14,26 +14,24 @@ * 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.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.security.auth.AuthenticateCallbackHandler; import org.apache.kafka.common.security.auth.SaslExtensions; 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.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.JaasOptionsUtils; -import org.apache.kafka.common.security.oauthbearer.internals.secured.ValidateException; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Closeable; import java.io.IOException; import java.util.HashMap; import java.util.List; @@ -44,15 +42,20 @@ import javax.security.auth.login.AppConfigurationEntry; import javax.security.sasl.SaslException; -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; - /** *

    * OAuthBearerLoginCallbackHandler is an {@link AuthenticateCallbackHandler} that * accepts {@link OAuthBearerTokenCallback} and {@link SaslExtensionsCallback} callbacks to - * perform the steps to request a JWT from an OAuth/OIDC provider using the - * clientcredentials. This grant type is commonly used for non-interactive - * "service accounts" where there is no user available to interactively supply credentials. + * perform the steps to request a JWT from an OAuth/OIDC provider. The OAuth grant types that + * are supported include: + * + *

      + *
    • client_credentials
    • + *
    • jwt-bearer
    • + *
    + * + * These grant types are commonly used for non-interactive "service accounts" where there is + * no user available to interactively supply credentials. *

    * *

    @@ -91,24 +94,40 @@ *

    * *

    - * The Kafka configuration must also include JAAS configuration which includes the following - * OAuth-specific options: + * The Kafka configuration must also include JAAS configuration which includes OAuth-specific options. + * For client_credentials, use: * *

      *
    • clientIdOAuth client ID (required)
    • *
    • clientSecretOAuth client secret (required)
    • *
    • scopeOAuth scope (optional)
    • *
    + * + * For the jwt-bearer grant type, use: + * + *
      + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    • XXXXXXXXXXXXXXXXXXXXXXXX
    • + *
    *

    * *

    * The JAAS configuration can also include any SSL options that are needed. The configuration * options are the same as those specified by the configuration in - * {@link org.apache.kafka.common.config.SslConfigs#addClientSslSupport(ConfigDef)}. + * {@link SslConfigs#addClientSslSupport(ConfigDef)}. *

    * *

    - * Here's an example of the JAAS configuration for a Kafka client: + * Here's an example of the JAAS configuration for a Kafka client using the + * client_credentials grant type: * * * sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required \ @@ -120,15 +139,14 @@ *

    * *

    - * The configuration option - * {@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL} + * The configuration option {@link SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL} * is also required in order for the client to contact the OAuth/OIDC provider. For example: * * * sasl.oauthbearer.token.endpoint.url=https://example.com/oauth2/v1/token * * - * Please see the OAuth/OIDC providers documentation for the token endpoint URL. + * Please see the OAuth/OIDC provider's documentation for the token endpoint URL. *

    * *

    @@ -148,81 +166,55 @@ * *

    */ - -public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler { +public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler, Closeable { private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class); - public static final String CLIENT_ID_CONFIG = "clientId"; - public static final String CLIENT_SECRET_CONFIG = "clientSecret"; - public static final String SCOPE_CONFIG = "scope"; - - public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " + - "client ID to uniquely identify the service account to use for authentication for " + - "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " + - "value and is provided to the OAuth provider using the OAuth " + - "clientcredentials grant type."; - - public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " + - "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " + - "account and identifies the service account to use for authentication for " + - "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " + - "value and is provided to the OAuth provider using the OAuth " + - "clientcredentials grant type."; - - public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " + - "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL + ") may need to specify an " + - "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " + - "include with the login request."; - private static final String EXTENSION_PREFIX = "extension_"; private Map moduleOptions; - private AccessTokenRetriever accessTokenRetriever; - private AccessTokenValidator accessTokenValidator; private boolean isInitialized = false; + protected AccessTokenRetriever accessTokenRetriever; + @Override public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); - AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions); - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism); - init(accessTokenRetriever, accessTokenValidator); - } - - public void init(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator accessTokenValidator) { - this.accessTokenRetriever = accessTokenRetriever; - this.accessTokenValidator = accessTokenValidator; try { - this.accessTokenRetriever.init(); - } catch (IOException e) { - throw new KafkaException("The OAuth login configuration encountered an error when initializing the AccessTokenRetriever", e); + configure( + new DefaultAccessTokenRetriever(), + new DefaultAccessTokenValidator(), + configs, + saslMechanism, + jaasConfigEntries + ); + } catch (Throwable t) { + throw new KafkaException("The OAuth login configuration encountered an error during initialization", t); } - - isInitialized = true; } - /* - * Package-visible for testing. - */ + void configure(AccessTokenRetriever accessTokenRetriever, + AccessTokenValidator accessTokenValidator, + Map configs, + String saslMechanism, + List jaasConfigEntries) { + this.accessTokenRetriever = accessTokenRetriever; + this.accessTokenValidator = accessTokenValidator; + + this.accessTokenRetriever.configure(configs, saslMechanism, jaasConfigEntries); + this.accessTokenValidator.configure(configs, saslMechanism, jaasConfigEntries); - AccessTokenRetriever getAccessTokenRetriever() { - return accessTokenRetriever; + this.isInitialized = true; } @Override public void close() { - if (accessTokenRetriever != null) { - try { - this.accessTokenRetriever.close(); - } catch (IOException e) { - log.warn("The OAuth login configuration encountered an error when closing the AccessTokenRetriever", e); - } - } + Utils.closeQuietly(accessTokenRetriever, "accessTokenRetriever"); + Utils.closeQuietly(accessTokenValidator, "accessTokenValidator"); } @Override @@ -240,20 +232,20 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback } } - private void handleTokenCallback(OAuthBearerTokenCallback callback) throws IOException { + protected void handleTokenCallback(OAuthBearerTokenCallback callback) throws IOException { checkInitialized(); String accessToken = accessTokenRetriever.retrieve(); try { OAuthBearerToken token = accessTokenValidator.validate(accessToken); callback.token(token); - } catch (ValidateException e) { + } catch (InvalidJwtException e) { log.warn(e.getMessage(), e); callback.error("invalid_token", e.getMessage(), null); } } - private void handleExtensionsCallback(SaslExtensionsCallback callback) { + protected void handleExtensionsCallback(SaslExtensionsCallback callback) { checkInitialized(); Map 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: - * - *
        - *
      1. - * Basic structural validation of the b64token value as defined in - * RFC 6750 Section 2.1 - *
      2. - *
      3. Basic conversion of the token into an in-memory data structure
      4. - *
      5. - * Presence of scope, exp, subject, iss, and - * iat claims - *
      6. - *
      7. - * Signature matching validation against the kid and those provided by - * the OAuth/OIDC provider's JWKS - *
      8. - *
      - */ - -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 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); - } - - } - }