diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 15ed859b8173e..07fe8fc445198 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -143,6 +143,7 @@ + diff --git a/clients/not_allowed.json b/clients/not_allowed.json new file mode 100644 index 0000000000000..7502b9a9af894 --- /dev/null +++ b/clients/not_allowed.json @@ -0,0 +1 @@ +9Bd3Qm7Wkm4l8dkZ \ No newline at end of file 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..bdad1da745685 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 @@ -129,6 +129,58 @@ public class SaslConfigs { + " authentication provider." + LOGIN_EXPONENTIAL_BACKOFF_NOTE; + + + + + + + public static final String SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS = "sasl.oauthbearer.jwt.retriever.class"; + public static final String DEFAULT_SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS = "org.apache.kafka.common.security.oauthbearer.DefaultJwtRetriever"; + public static final String SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS_DOC = "The fully-qualified class name of a JwtRetriever implementation used to request tokens from the" + + " identity provider. The default value represents a class that maintains backward compatibility with previous versions of Apache Kafka. The default" + + " implementation uses the configuration to determine which concrete implementation to create."; + + public static final String SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS = "sasl.oauthbearer.jwt.validator.class"; + public static final String DEFAULT_SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS = "org.apache.kafka.common.security.oauthbearer.DefaultJwtValidator"; + public static final String SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS_DOC = "The fully-qualified class name of a JwtValidator implementation used to validate the token from the" + + " identity provider. The default value represents a class that maintains backward compatibility with previous versions of Apache Kafka. The default" + + " implementation uses the configuration to determine which concrete implementation to create."; + + public static final String SASL_OAUTHBEARER_GRANT_TYPE = "sasl.oauthbearer.grant.type"; + public static final String DEFAULT_SASL_OAUTHBEARER_GRANT_TYPE = "client_credentials"; + public static final String SASL_OAUTHBEARER_GRANT_TYPE_DOC = "The OAuth grant type to use when communicating with the identity provider. On the whole, the OAuth layer" + + " does not rely on this value and expects it to be used and/or verified for correctness by the JwtRetriever implementation. The default value of \"client_credentials\"" + + " maintains backward compatibility. The built-in grant types are \"client_credentials\" and \"urn:ietf:params:oauth:grant-type:jwt-bearer\"." + + " The OAuth code in Kafka does not limit the values that are used. A user can write a custom JwtRetriever implementation that uses a completely different grant" + + " type, if desired."; + + public static final String SASL_OAUTHBEARER_SCOPE = "sasl.oauthbearer.scope"; + public static final String DEFAULT_SASL_OAUTHBEARER_SCOPE = "client_credentials"; + public static final String SASL_OAUTHBEARER_SCOPE_DOC = "This is the level of access a client application is granted to a resource or API which is included in" + + " the token request. If provided, it should match one or more scopes configured in the identity provider. Note: the OAuth scope was previously stored as part" + + " of the sasl.jaas.config configuration with the key \"scope\". For backward compatibility, the \"scope\" JAAS option can still be used, but if both are present," + + " this configuration value takes precedence over the value from sasl.jaas.config."; + + public static final String SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID = "sasl.oauthbearer.client.credentials.client.id"; + public static final String SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET = "sasl.oauthbearer.client.credentials.client.secret"; + public static final String SASL_OAUTHBEARER_ASSERTION_ALGORITHM = "sasl.oauthbearer.assertion.algorithm"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_AUD = "sasl.oauthbearer.assertion.claim.aud"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_EXP_MINUTES = "sasl.oauthbearer.assertion.claim.exp.minutes"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_ISS = "sasl.oauthbearer.assertion.claim.iss"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_JTI_INCLUDE = "sasl.oauthbearer.assertion.claim.jti.include"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_NBF_MINUTES = "sasl.oauthbearer.assertion.claim.nbf.minutes"; + public static final String SASL_OAUTHBEARER_ASSERTION_CLAIM_SUB = "sasl.oauthbearer.assertion.claim.sub"; + public static final String SASL_OAUTHBEARER_ASSERTION_FILE = "sasl.oauthbearer.assertion.file"; + public static final String SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE = "sasl.oauthbearer.assertion.private.key.file"; + public static final String SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_PASSPHRASE = "sasl.oauthbearer.assertion.private.key.passphrase"; + public static final String SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE = "sasl.oauthbearer.assertion.template.file"; + + + + + + 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)" @@ -217,6 +269,9 @@ 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_JWT_RETRIEVER_CLASS, ConfigDef.Type.CLASS, DEFAULT_SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, ConfigDef.Type.CLASS, DEFAULT_SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_GRANT_TYPE, ConfigDef.Type.STRING, DEFAULT_SASL_OAUTHBEARER_GRANT_TYPE, ConfigDef.Importance.MEDIUM, SASL_OAUTHBEARER_GRANT_TYPE) .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/AssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AssertionCreator.java new file mode 100644 index 0000000000000..9c6815febadb1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AssertionCreator.java @@ -0,0 +1,96 @@ +/* + * 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.Closeable; +import java.io.IOException; +import java.security.GeneralSecurityException; + +/** + * {@code AssertionCreator} is used to create a client-signed OAuth assertion that can be used with different + * grant types. See RFC 7521 for specifics. + * + *

+ * + * The assertion creator has three main steps: + * + *

    + *
  1. Create the JWT header
  2. + *
  3. Create the JWT payload
  4. + *
  5. Sign
  6. + *
+ * + *

+ * + * Step 1 is to dynamically create the JWT header. The implementation may add whatever values it needs, but + * the {@code alg} (algorithm), {@code kid} (key ID), and {@code type} (type) are usually present. Here is + * an example of the JSON version of the JWT header: + * + *

+ * {
+ *   "kid": "9d82418e64e0541066637ca8592d459c",
+ *   "alg": RS256,
+ *   "typ": "JWT",
+ * }
+ * 
+ * + *

+ * + * Step 2 is to create the JWT payload from the claims provided to {@link #create(AssertionJwtTemplate)}. Depending on the + * implementation, other claims may be dynamically generated and added to the JWT payload. Or, some of the + * claims in the incoming map could be ignored or modified. Here's an example where the implementation has + * added the {@code iat} (initialized at) and {@code exp} (expires) claims: + * + *

+ * {
+ *   "iat": 1741121401,
+ *   "exp": 1741125001,
+ *   "sub": "some-service-account",
+ *   "aud": "my_audience",
+ *   "iss": "https://example.com",
+ *   "...": "...",
+ * }
+ * 
+ * + *

+ * + * Step 3 is to use the configured private key to sign the header and payload and serialize in the compact + * JWT format. The means by which the private key (if any) is made available for use is up to the + * implementation. The private key could be loaded from a file, downloaded from a trusted resource, + * embedded in the configuration, etc. + */ +public interface AssertionCreator extends Closeable { + + /** + * Creates and signs an OAuth assertion by converting the given claims into JWT and then signing it using + * the configured algorithm. + * + *

+ * + * @param template {@link AssertionJwtTemplate} with optional header and/or claims to include in the JWT + */ + String create(AssertionJwtTemplate template) throws GeneralSecurityException, IOException; + + /** + * Closes any resources used by this implementation. The default implementation of + * this method is a no op, for convenience to implementors. + */ + @Override + default void close() { + // Do nothing... + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AssertionJwtTemplate.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AssertionJwtTemplate.java new file mode 100644 index 0000000000000..de7acd82cb314 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/AssertionJwtTemplate.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 java.io.Closeable; +import java.util.Map; + +/** + * {@code AssertionJwtTemplate} is used to provide values for use by {@link AssertionCreator}. + * The JWT header and/or payload used in the assertion likely requires headers and claims. Not all identity + * providers require the same set of headers and claims; some may require a given header or claim while + * other identity providers may prohibit it. In order to provide the most flexibility, the header + * values and claims that are to be included in the JWT can be added via a template. + * + *

+ * + * Both the {@link #header()} and {@link #payload()} APIs return a map of Objects. This because the + * JSON specification allow values to be one of the following "types": + * + *

+ * + * However, because the maps must be converted into JSON, it's important that any nested types use standard + * Java type equivalents (Map, List, String, Integer, Double, and Boolean) so that the JSON library will + * know how to serialize the entire object graph. + */ +public interface AssertionJwtTemplate extends Closeable { + + /** + * Returns a map containing zero or more header values. + * + * @return Values to include in the JWT header + */ + Map header(); + + /** + * Returns a map containing zero or more JWT payload claim values. + * + * @return Values to include in the JWT payload + */ + Map payload(); + + /** + * Closes any resources used by this implementation. The default implementation of + * this method is a no op, for convenience to implementors. + */ + @Override + default void close() { + // Do nothing... + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java new file mode 100644 index 0000000000000..1b960aed12050 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java @@ -0,0 +1,328 @@ +/* + * 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.ConfigException; +import org.apache.kafka.common.security.oauthbearer.internals.secured.BasicOAuthBearerToken; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwksFileVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerConfig; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerJaasConfig; +import org.apache.kafka.common.security.oauthbearer.internals.secured.RefCountingMap; +import org.apache.kafka.common.security.oauthbearer.internals.secured.RefreshingHttpsJwks; +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.security.oauthbearer.internals.secured.SslResource; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; + +import org.jose4j.jwk.HttpsJwks; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.function.Function; +import java.util.stream.Collectors; + +import javax.net.ssl.SSLContext; +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_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; +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.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.jaasOptions; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.maybeCreateSslResource; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.protocolMatches; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimExpiration; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimIssuedAt; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimScopes; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimSubject; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateUrl; +import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE; + +/** + * Implementation of {@link JwtValidator} 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, sub, 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 BrokerJwtValidator implements JwtValidator { + + private static final Logger log = LoggerFactory.getLogger(BrokerJwtValidator.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 RefCountingMap VERIFICATION_KEY_RESOLVER_CACHE = new RefCountingMap<>(); + + private final Time time; + + private CloseableVerificationKeyResolver verificationKeyResolver; + + private Optional sslResource = Optional.empty(); + + private JwtConsumer jwtConsumer; + + private String scopeClaimName; + + private String subClaimName; + + public BrokerJwtValidator() { + this(Time.SYSTEM); + } + + public BrokerJwtValidator(Time time) { + this.time = time; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + Function wrapResolverFn = k -> { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + URL jwksEndpoint = validateUrl(oauthConfig, SASL_OAUTHBEARER_JWKS_ENDPOINT_URL); + + if (protocolMatches(jwksEndpoint, "file")) { + return new JwksFileVerificationKeyResolver(configs, saslMechanism); + } else { + OAuthBearerJaasConfig jaasConfig = new OAuthBearerJaasConfig(jaasOptions(saslMechanism, jaasConfigEntries)); + HttpsJwks httpsJwks = new HttpsJwks(jwksEndpoint.toString()); + sslResource = maybeCreateSslResource(jwksEndpoint, jaasConfig); + Optional sslContext = sslResource.map(SslResource::sslContext); + long refreshMs = oauthConfig.getLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS); + + if (refreshMs < 0) + throw new ConfigException(String.format("The OAuth configuration option %s value must be non-negative", SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS)); + + long refreshRetryBackoffMs = oauthConfig.getLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS); + long refreshRetryBackoffMaxMs = oauthConfig.getLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS); + + RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks( + time, + httpsJwks, + sslContext, + Executors.newSingleThreadScheduledExecutor(), + refreshMs, + refreshRetryBackoffMs, + refreshRetryBackoffMaxMs + ); + return new RefreshingHttpsJwksVerificationKeyResolver(refreshingHttpsJwks); + } + }; + + // Here's the logic which keeps our VerificationKeyResolvers down to a single instance. + VerificationKeyResolverKey key = new VerificationKeyResolverKey(configs, jaasConfigEntries); + CloseableVerificationKeyResolver resolver = VERIFICATION_KEY_RESOLVER_CACHE.get( + key, + wrapResolverFn + ); + + configure(resolver, configs, saslMechanism); + } + + public void configure(CloseableVerificationKeyResolver verificationKeyResolver, + Map configs, + String saslMechanism) { + this.verificationKeyResolver = verificationKeyResolver; + + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + final JwtConsumerBuilder jwtConsumerBuilder = new JwtConsumerBuilder(); + + if (oauthConfig.containsKey(SASL_OAUTHBEARER_EXPECTED_AUDIENCE)) { + // It's a bit convoluted turning the optional list of expected audiences into an array for the jose4j API. + List list = oauthConfig.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE); + + if (!list.isEmpty()) { + Set set = Set.copyOf(list); + String[] array = set.toArray(new String[0]); + jwtConsumerBuilder.setExpectedAudience(array); + } + } + + oauthConfig.maybeGetInt(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS).ifPresent(jwtConsumerBuilder::setAllowedClockSkewInSeconds); + oauthConfig.maybeGetString(SASL_OAUTHBEARER_EXPECTED_ISSUER).ifPresent(jwtConsumerBuilder::setExpectedIssuer); + + this.jwtConsumer = jwtConsumerBuilder + .setJwsAlgorithmConstraints(DISALLOW_NONE) + .setRequireExpirationTime() + .setRequireIssuedAt() + .setVerificationKeyResolver(verificationKeyResolver) + .build(); + this.scopeClaimName = oauthConfig.getString(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME); + this.subClaimName = oauthConfig.getString(SASL_OAUTHBEARER_SUB_CLAIM_NAME); + } + + @Override + public void close() { + Utils.closeQuietly(verificationKeyResolver, "verificationKeyResolver"); + sslResource.ifPresent(r -> Utils.closeQuietly(r, "sslResource")); + } + + /** + * Accepts an OAuth JWT access token in base-64 encoded format, validates, and returns an + * OAuthBearerToken. + * + * @param jwt Non-null JWT + * @return {@link OAuthBearerToken} + * @throws JwtValidatorException Thrown on errors performing validation of given token + */ + @Override + @SuppressWarnings("unchecked") + public OAuthBearerToken validate(String jwt) throws JwtValidatorException { + SerializedJwt serializedJwt = new SerializedJwt(jwt); + + JwtContext jwtContext; + + try { + jwtContext = jwtConsumer.process(serializedJwt.getToken()); + } catch (org.jose4j.jwt.consumer.InvalidJwtException e) { + throw new JwtValidatorException(String.format("Could not validate the access token: %s", e.getMessage()), e); + } + + JwtClaims claims = jwtContext.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 = validateClaimScopes(scopeClaimName, scopeRawCollection); + long expiration = validateClaimExpiration( + ReservedClaimNames.EXPIRATION_TIME, + expirationRaw != null ? expirationRaw.getValueInMillis() : null + ); + String sub = validateClaimSubject(subClaimName, subRaw); + Long issuedAt = validateClaimIssuedAt( + ReservedClaimNames.ISSUED_AT, + issuedAtRaw != null ? issuedAtRaw.getValueInMillis() : null + ); + + return new BasicOAuthBearerToken( + jwt, + scopes, + expiration, + sub, + issuedAt + ); + } + + private T getClaim(ClaimSupplier supplier, String claimName) throws JwtValidatorException { + try { + T value = supplier.get(); + log.debug("getClaim - {}: {}", claimName, value); + return value; + } catch (MalformedClaimException e) { + throw new JwtValidatorException(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); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java new file mode 100644 index 0000000000000..907247dc5b4f3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/ClientCredentialsJwtRetriever.java @@ -0,0 +1,164 @@ +/* + * 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.ConfigException; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ClientCredentialsRequestGenerator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpRequestGenerator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtHttpClient; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtHttpResponseBodyHandler; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerConfig; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerJaasConfig; +import org.apache.kafka.common.security.oauthbearer.internals.secured.SslResource; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; + +import java.net.URL; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +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_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_CLIENT_CREDENTIALS_CLIENT_ID; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.jaasOptions; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.maybeCreateSslResource; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.urlencodeHeader; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateUrl; + +/** + * A {@link JwtRetriever} that will communicate with an OAuth/OIDC provider directly via HTTP to post client + * credentials using the client ID and client secret values to a publicized token endpoint URL + * ({@link SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL}). + */ +public class ClientCredentialsJwtRetriever implements JwtRetriever { + + private static final String CLIENT_ID_JAAS = "clientId"; + private static final String CLIENT_SECRET_JAAS = "clientSecret"; + private static final String SCOPE_JAAS = "scope"; + + private final Time time; + + private Optional sslResource = Optional.empty(); + private HttpRequestGenerator requestGenerator; + private long retryBackoffMs; + private long retryBackoffMaxMs; + private HttpClient client; + + public ClientCredentialsJwtRetriever() { + this(Time.SYSTEM); + } + + public ClientCredentialsJwtRetriever(Time time) { + this.time = time; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + OAuthBearerJaasConfig jaasConfig = new OAuthBearerJaasConfig(jaasOptions(saslMechanism, jaasConfigEntries)); + + retryBackoffMs = oauthConfig.getLong(SASL_LOGIN_RETRY_BACKOFF_MS); + retryBackoffMaxMs = oauthConfig.getLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS); + + URL tokenEndpoint = validateUrl(oauthConfig, SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + sslResource = maybeCreateSslResource(tokenEndpoint, jaasConfig); + + HttpClient.Builder clientBuilder = HttpClient.newBuilder(); + oauthConfig.maybeGetInt(SASL_LOGIN_CONNECT_TIMEOUT_MS).ifPresent(ms -> clientBuilder.connectTimeout(Duration.ofMillis(ms))); + sslResource.ifPresent(r -> clientBuilder.sslContext(r.sslContext())); + client = clientBuilder.build(); + + String clientId = configOrJaas( + oauthConfig, + jaasConfig, + SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_ID, + CLIENT_ID_JAAS, + true + ); + String clientSecret = configOrJaas( + oauthConfig, + jaasConfig, + SASL_OAUTHBEARER_CLIENT_CREDENTIALS_CLIENT_SECRET, + CLIENT_SECRET_JAAS, + true + ); + String scope = configOrJaas( + oauthConfig, + jaasConfig, + SASL_OAUTHBEARER_SCOPE, + SCOPE_JAAS, + false + ); + boolean urlencodeHeader = urlencodeHeader(oauthConfig); + + requestGenerator = new ClientCredentialsRequestGenerator( + tokenEndpoint, + clientId, + clientSecret, + scope, + urlencodeHeader + ); + } + + @Override + public String retrieve() throws JwtRetrieverException { + HttpRequest request = requestGenerator.generateRequest(); + JwtHttpClient jwtHttpClient = new JwtHttpClient(time); + HttpResponse.BodyHandler responseBodyHandler = new JwtHttpResponseBodyHandler(); + + return jwtHttpClient.request( + client, + request, + responseBodyHandler, + retryBackoffMs, + retryBackoffMaxMs + ); + } + + @Override + public void close() { + Utils.closeQuietly(requestGenerator, "requestGenerator"); + sslResource.ifPresent(r -> Utils.closeQuietly(r, "sslResource")); + } + + static String configOrJaas(OAuthBearerConfig oauthConfig, + OAuthBearerJaasConfig jaasConfig, + String configName, + String jaasName, + boolean isRequired) { + if (oauthConfig.containsKey(configName)) + return oauthConfig.getString(configName); + else if (jaasConfig.containsKey(jaasName)) + return jaasConfig.getString(jaasName); + else if (isRequired) + throw new ConfigException("Could not find OAuth configuration for " + configName + " or OAuth JAAS configuration for " + jaasName); + else + return null; + } +} 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/ClientJwtValidator.java similarity index 53% 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/ClientJwtValidator.java index 773311ff0ab18..092d7c5477fc7 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/ClientJwtValidator.java @@ -14,10 +14,11 @@ * 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.OAuthBearerConfig; +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,14 +27,24 @@ 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; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimExpiration; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimIssuedAt; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimNameOverride; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimScopes; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimSubject; /** - * LoginAccessTokenValidator is an implementation of {@link AccessTokenValidator} that is used + * {@code ClientJwtValidator} is an implementation of {@link JwtValidator} that is used * by the client to perform some rudimentary validation of the JWT access token that is received * as part of the response from posting the client credentials to the OAuth/OIDC provider's * token endpoint. @@ -46,53 +57,52 @@ * RFC 6750 Section 2.1 * *
  • Basic conversion of the token into an in-memory map
  • - *
  • Presence of scope, exp, subject, and iat claims
  • + *
  • Presence of scope, exp, sub, and iat claims
  • * */ +public class ClientJwtValidator implements JwtValidator { -public class LoginAccessTokenValidator implements AccessTokenValidator { - - private static final Logger log = LoggerFactory.getLogger(LoginAccessTokenValidator.class); + private static final Logger log = LoggerFactory.getLogger(ClientJwtValidator.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) { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + scopeClaimName = validateClaimNameOverride( + DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, + oauthConfig.get(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME) + ); + subClaimName = validateClaimNameOverride( + DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, + oauthConfig.get(SASL_OAUTHBEARER_SUB_CLAIM_NAME) + ); } /** * Accepts an OAuth JWT access token in base-64 encoded format, validates, and returns an * OAuthBearerToken. * - * @param accessToken Non-null JWT access token + * @param jwt Non-null JWT * @return {@link OAuthBearerToken} - * @throws ValidateException Thrown on errors performing validation of given token + * @throws JwtValidatorException Thrown on errors performing validation of given token */ @SuppressWarnings("unchecked") - public OAuthBearerToken validate(String accessToken) throws ValidateException { - SerializedJwt serializedJwt = new SerializedJwt(accessToken); + public OAuthBearerToken validate(String jwt) throws JwtValidatorException { + SerializedJwt serializedJwt = new SerializedJwt(jwt); 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 JwtValidatorException(String.format("Could not validate the access token: %s", e.getMessage()), e); } Object scopeRaw = getClaim(payload, scopeClaimName); @@ -109,18 +119,24 @@ else if (scopeRaw instanceof Collection) String subRaw = (String) getClaim(payload, subClaimName); Number issuedAtRaw = (Number) getClaim(payload, ISSUED_AT_CLAIM_NAME); - Set scopes = ClaimValidationUtils.validateScopes(scopeClaimName, scopeRawCollection); - long expiration = ClaimValidationUtils.validateExpiration(EXPIRATION_CLAIM_NAME, - expirationRaw != null ? expirationRaw.longValue() * 1000L : null); - String subject = ClaimValidationUtils.validateSubject(subClaimName, subRaw); - Long issuedAt = ClaimValidationUtils.validateIssuedAt(ISSUED_AT_CLAIM_NAME, - issuedAtRaw != null ? issuedAtRaw.longValue() * 1000L : null); - - return new BasicOAuthBearerToken(accessToken, + Set scopes = validateClaimScopes(scopeClaimName, scopeRawCollection); + long expiration = validateClaimExpiration( + EXPIRATION_CLAIM_NAME, + expirationRaw != null ? expirationRaw.longValue() * 1000L : null + ); + String subject = validateClaimSubject(subClaimName, subRaw); + Long issuedAt = validateClaimIssuedAt( + ISSUED_AT_CLAIM_NAME, + issuedAtRaw != null ? issuedAtRaw.longValue() * 1000L : null + ); + + return new BasicOAuthBearerToken( + jwt, scopes, expiration, subject, - issuedAt); + issuedAt + ); } private Object getClaim(Map payload, String claimName) { @@ -128,5 +144,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/DefaultJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetriever.java new file mode 100644 index 0000000000000..4c275f28bc004 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetriever.java @@ -0,0 +1,84 @@ +/* + * 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.JwtBearerRequestGenerator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerConfig; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; + +import java.net.URL; +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_GRANT_TYPE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_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.OAuthBearerUtils.protocolMatches; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateUrl; + +public class DefaultJwtRetriever implements JwtRetriever { + + private final Time time; + + private JwtRetriever delegate; + + public DefaultJwtRetriever() { + this(Time.SYSTEM); + } + + public DefaultJwtRetriever(Time time) { + this.time = time; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + URL tokenEndpoint = validateUrl(oauthConfig, SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + + if (protocolMatches(tokenEndpoint, "file")) { + delegate = new FileJwtRetriever(); + } else { + String grantType = oauthConfig.maybeGetString(SASL_OAUTHBEARER_GRANT_TYPE).orElse(DEFAULT_SASL_OAUTHBEARER_GRANT_TYPE); + + if (grantType.equalsIgnoreCase(JwtBearerRequestGenerator.GRANT_TYPE)) { + delegate = new JwtBearerJwtRetriever(time); + } else { + delegate = new ClientCredentialsJwtRetriever(time); + } + } + + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public String retrieve() throws JwtRetrieverException { + return Objects.requireNonNull(delegate).retrieve(); + } + + @Override + public void close() { + Utils.closeQuietly(delegate, "delegate"); + } + + public JwtRetriever delegate() { + return delegate; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidator.java new file mode 100644 index 0000000000000..828df5a82c249 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtValidator.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 org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerConfig; +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; + +public class DefaultJwtValidator implements JwtValidator { + + private JwtValidator delegate; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + JwtValidator validator; + + if (oauthConfig.containsKey(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL)) + validator = new BrokerJwtValidator(); + else + validator = new ClientJwtValidator(); + + configure(validator, configs, saslMechanism, jaasConfigEntries); + } + + void configure(JwtValidator validator, + Map configs, + String saslMechanism, + List jaasConfigEntries) { + delegate = validator; + delegate.configure(configs, saslMechanism, jaasConfigEntries); + } + + @Override + public OAuthBearerToken validate(String jwt) throws JwtValidatorException { + return Objects.requireNonNull(delegate).validate(jwt); + } + + @Override + public void close() { + Utils.closeQuietly(delegate, "delegate"); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileAssertionJwtTemplate.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileAssertionJwtTemplate.java new file mode 100644 index 0000000000000..de31eb4e2fea0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileAssertionJwtTemplate.java @@ -0,0 +1,163 @@ +/* + * 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.CachedFile; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.File; +import java.util.Collections; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.staticCacheRefreshPolicy; + +/** + * {@code AssertionJwtTemplateFile} is used by the user to specify a JSON file on disk that contains static values + * that can be loaded and used to construct the assertion. The file structure is a JSON containing optionally a + * header and/or payload top-level attribute. + * + *

    + * + * Here is a minimally viable JSON structure: + * + *

    + * {
    + * }
    + * 
    + * + * OK, at that point it doesn't make sense for the user to build that file. + * + *

    + * + * Here is another, slightly less minimal JSON template: + * + *

    + * {
    + *    "header": {
    + *     "foo": 1
    + *   },
    + *    "payload": {
    + *     "bar": 2
    + *   }
    + * }
    + * 
    + * + * This provides a single header value and a single payload claim. + * + *

    + * + * A more realistic example template looks like so: + * + *

    + * {
    + *   "header": {
    + *     "kid": "f829d41b06f14f9e",
    + *     "some-random-header": 123456
    + *   },
    + *   "payload": {
    + *     "sub": "some-service-account",
    + *     "aud": "my_audience",
    + *     "iss": "https://example.com",
    + *     "useSomeResource": false,
    + *     "allowedAnimals": [
    + *       "cat",
    + *       "dog",
    + *       "hamster"
    + *     ]
    + *   }
    + * }
    + * 
    + * + * The AssertionCreator would accept the AssertionJwtTemplate and augment the template header and/or payload + * with dynamic values. For example, the above header would be augmented with the {@code alg} (algorithm) and + * {@code typ} (type) values per the OAuth RFC: + * + *
    + * {
    + *   "kid": "f829d41b06f14f9e",
    + *   "some-random-header": 123456,
    + *   "alg": "RS256",
    + *   "typ": "JWT"
    + * }
    + * 
    + * + * And the payload would also be augmented to add the {@code iat} (issued at) and {@code exp} (expiration) timestamps: + * + *
    + * {
    + *   "iat": 1741121401,
    + *   "exp": 1741125001,
    + *   "sub": "some-service-account",
    + *   "aud": "my_audience",
    + *   "iss": "https://example.com",
    + *   "useSomeResource": false,
    + *   "allowedAnimals": [
    + *     "cat",
    + *     "dog",
    + *     "hamster"
    + *   ]
    + * }
    + * 
    + */ +public class FileAssertionJwtTemplate implements AssertionJwtTemplate { + + @SuppressWarnings("unchecked") + private static final BiFunction JSON_TRANSFORMER = (file, json) -> { + try { + ObjectMapper mapper = new ObjectMapper(); + Map map = (Map) mapper.readValue(json, Map.class); + + Map header = (Map) map.computeIfAbsent("header", k -> Map.of()); + Map payload = (Map) map.computeIfAbsent("payload", k -> Map.of()); + + return new CachedTemplate(header, payload); + } catch (Exception e) { + throw new KafkaException("An error occurred parsing the OAuth assertion template file from " + file.getPath(), e); + } + }; + + private final CachedFile jsonFile; + + public FileAssertionJwtTemplate(File jsonFile) { + this.jsonFile = new CachedFile<>(jsonFile, JSON_TRANSFORMER, staticCacheRefreshPolicy()); + } + + @Override + public Map header() { + return jsonFile.transformed().header; + } + + @Override + public Map payload() { + return jsonFile.transformed().payload; + } + + private static class CachedTemplate { + + private final Map header; + + private final Map payload; + + private CachedTemplate(Map header, Map payload) { + this.header = Collections.unmodifiableMap(header); + this.payload = Collections.unmodifiableMap(payload); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetriever.java new file mode 100644 index 0000000000000..eb934c8c9d854 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetriever.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 org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerConfig; + +import java.io.File; +import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.staticCacheRefreshPolicy; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateFileUrl; + +/** + * A {@link JwtRetriever} that will load the contents of a file, interpreting them as a JWT in serialized form. + */ +public class FileJwtRetriever implements JwtRetriever { + + private CachedFile jwtFile; + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + File fileName = validateFileUrl(oauthConfig, SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + + try { + // always non-null; to remove any newline chars or backend will report err + jwtFile = new CachedFile<>( + fileName, + (file, contents) -> contents.trim(), + staticCacheRefreshPolicy() + ); + } catch (Exception e) { + throw new KafkaException("An error occurred reading the OAuth JWT from " + fileName); + } + } + + @Override + public String retrieve() throws JwtRetrieverException { + if (jwtFile == null) + throw new JwtRetrieverException("JWT is null; please call configure() first"); + + return jwtFile.transformed(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetriever.java new file mode 100644 index 0000000000000..5181ccfcff7b8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtBearerJwtRetriever.java @@ -0,0 +1,133 @@ +/* + * 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.DefaultAssertionCreator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.FileAssertionCreator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpRequestGenerator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBearerRequestGenerator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtHttpClient; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtHttpResponseBodyHandler; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerConfig; +import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerJaasConfig; +import org.apache.kafka.common.security.oauthbearer.internals.secured.SslResource; +import org.apache.kafka.common.security.oauthbearer.internals.secured.StaticAssertionJwtTemplate; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; + +import java.io.File; +import java.net.URL; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +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_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_ASSERTION_ALGORITHM; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_FILE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.jaasOptions; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.maybeCreateSslResource; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateFile; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateUrl; + +public class JwtBearerJwtRetriever implements JwtRetriever { + + private final Time time; + + private Optional sslResource = Optional.empty(); + private HttpRequestGenerator requestGenerator; + private long retryBackoffMs; + private long retryBackoffMaxMs; + private HttpClient client; + + public JwtBearerJwtRetriever() { + this(Time.SYSTEM); + } + + public JwtBearerJwtRetriever(Time time) { + this.time = time; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + OAuthBearerJaasConfig jaasConfig = new OAuthBearerJaasConfig(jaasOptions(saslMechanism, jaasConfigEntries)); + + URL tokenEndpoint = validateUrl(oauthConfig, SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + retryBackoffMs = oauthConfig.getLong(SASL_LOGIN_RETRY_BACKOFF_MS); + retryBackoffMaxMs = oauthConfig.getLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS); + sslResource = maybeCreateSslResource(tokenEndpoint, jaasConfig); + + HttpClient.Builder clientBuilder = HttpClient.newBuilder(); + oauthConfig.maybeGetInt(SASL_LOGIN_CONNECT_TIMEOUT_MS).ifPresent(ms -> clientBuilder.connectTimeout(Duration.ofMillis(ms))); + sslResource.ifPresent(r -> clientBuilder.sslContext(r.sslContext())); + client = clientBuilder.build(); + + AssertionCreator assertionCreator; + AssertionJwtTemplate assertionJwtTemplate; + + if (oauthConfig.containsKey(SASL_OAUTHBEARER_ASSERTION_FILE)) { + File assertionFile = validateFile(oauthConfig, SASL_OAUTHBEARER_ASSERTION_FILE); + assertionCreator = new FileAssertionCreator(assertionFile); + assertionJwtTemplate = new StaticAssertionJwtTemplate(Map.of()); + } else { + String algorithm = oauthConfig.getString(SASL_OAUTHBEARER_ASSERTION_ALGORITHM); + File privateKeyFile = validateFile(oauthConfig, SASL_OAUTHBEARER_ASSERTION_PRIVATE_KEY_FILE); + assertionCreator = new DefaultAssertionCreator(time, algorithm, privateKeyFile); + + File assertionTemplateFile = validateFile(oauthConfig, SASL_OAUTHBEARER_ASSERTION_TEMPLATE_FILE); + assertionJwtTemplate = new FileAssertionJwtTemplate(assertionTemplateFile); + } + + requestGenerator = new JwtBearerRequestGenerator( + tokenEndpoint, + assertionCreator, + assertionJwtTemplate + ); + } + + @Override + public String retrieve() throws JwtRetrieverException { + HttpRequest request = requestGenerator.generateRequest(); + JwtHttpClient jwtHttpClient = new JwtHttpClient(time); + HttpResponse.BodyHandler responseBodyHandler = new JwtHttpResponseBodyHandler(); + + return jwtHttpClient.request( + client, + request, + responseBodyHandler, + retryBackoffMs, + retryBackoffMaxMs + ); + } + + @Override + public void close() { + Utils.closeQuietly(requestGenerator, "requestGenerator"); + sslResource.ifPresent(r -> Utils.closeQuietly(r, "sslResource")); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetriever.java new file mode 100644 index 0000000000000..f9a48b0b765bd --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetriever.java @@ -0,0 +1,68 @@ +/* + * 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.JwtHttpClient; + +import javax.security.auth.spi.LoginModule; + +/** + * An implementation of JwtRetriever is the means by which the login module will + * retrieve an OAuth JWT 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. JwtRetriever + * implementations should not validate the integrity of the JWT, but should rely on the companion + * {@link JwtValidator} for that task. + * + * @see ClientCredentialsJwtRetriever + * @see DefaultJwtRetriever + * @see FileJwtRetriever + * @see JwtHttpClient + * @see JwtBearerJwtRetriever + */ +public interface JwtRetriever 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 JwtRetrieverException Thrown on errors related to retrieval + */ + String retrieve() throws JwtRetrieverException; + + /** + * Closes any resources used by this implementation. The default implementation of + * this method is a no op, for convenience to implementors. + */ + @Override + default void close() { + // Do nothing... + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetrieverException.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetrieverException.java new file mode 100644 index 0000000000000..b2264a2f82d06 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/JwtRetrieverException.java @@ -0,0 +1,39 @@ +/* + * 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; + +/** + * A {@code JwtRetrieverException} is thrown in cases where the JWT cannot be retrieved. + * + * @see JwtRetriever#retrieve() + */ +public class JwtRetrieverException extends KafkaException { + + public JwtRetrieverException(String message) { + super(message); + } + + public JwtRetrieverException(Throwable cause) { + super(cause); + } + + public JwtRetrieverException(String message, Throwable cause) { + super(message, cause); + } +} 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/JwtValidator.java similarity index 64% 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/JwtValidator.java index 0b107a09bc065..d26b317a2ecad 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/JwtValidator.java @@ -14,13 +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.OAuthBearerToken; +package org.apache.kafka.common.security.oauthbearer; /** - * An instance of AccessTokenValidator acts as a function object that, given an access + * An instance of JwtValidator acts as a function object that, given an access * token in base-64 encoded JWT format, can parse the data, perform validation, and construct an * {@link OAuthBearerToken} for use by the caller. * @@ -40,25 +37,31 @@ *
  • 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 - * to validate the token's contents and verify the signature + * @see DefaultJwtValidator Default validator that acts as a wrapper over one of the other validators + * @see ClientJwtValidator A basic validator used by client-side login authentication + * @see BrokerJwtValidator A more robust validator that is used on the broker to validate the token's contents + * and verify the signature */ - -public interface AccessTokenValidator { +public interface JwtValidator extends OAuthBearerConfigurable { /** - * Accepts an OAuth JWT access token in base-64 encoded format, validates, and returns an - * OAuthBearerToken. + * Accepts an OAuth JWT in base-64 encoded format, validates, and returns an OAuthBearerToken. * - * @param accessToken Non-null JWT access token + * @param jwt Non-null JWT to validate * * @return {@link OAuthBearerToken} * - * @throws ValidateException Thrown on errors performing validation of given token + * @throws JwtValidatorException Thrown on errors performing validation of given token */ - OAuthBearerToken validate(String accessToken) throws ValidateException; + OAuthBearerToken validate(String jwt) throws JwtValidatorException; + /** + * Closes any resources used by this implementation. The default implementation of + * this method is a no op, for convenience to implementors. + */ + @Override + default void close() { + // Do nothing... + } } 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/JwtValidatorException.java similarity index 59% 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/JwtValidatorException.java index 430b9007830cb..e6ba8406e72e2 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/JwtValidatorException.java @@ -14,34 +14,31 @@ * 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; import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; /** - * ValidateException is thrown in cases where a JWT access token cannot be determined to be - * valid for one reason or another. It is intended to be used when errors arise within the - * processing of a {@link javax.security.auth.callback.CallbackHandler#handle(Callback[])}. - * This error, however, is not thrown from that method directly. + * A {@code JwtValidatorException} is thrown in cases where the validity of a JWT access token + * cannot be determined. It is intended to be used when errors arise within the processing of a + * {@link CallbackHandler#handle(Callback[])}. This error, however, is not thrown from that method directly. * - * @see AccessTokenValidator#validate(String) + * @see JwtValidator#validate(String) */ +public class JwtValidatorException extends KafkaException { -public class ValidateException extends KafkaException { - - public ValidateException(String message) { + public JwtValidatorException(String message) { super(message); } - public ValidateException(Throwable cause) { + public JwtValidatorException(Throwable cause) { super(cause); } - public ValidateException(String message, Throwable cause) { + public JwtValidatorException(String message, Throwable cause) { super(message, cause); } - } 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..8a7eec371aa9b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerConfigurable.java @@ -0,0 +1,60 @@ +/* + * 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 OAuth-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); +} 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..413bb2e7c7c2d 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,22 @@ * 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 +40,25 @@ import javax.security.auth.login.AppConfigurationEntry; import javax.security.sasl.SaslException; -import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.getConfiguredInstanceOrDefault; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.jaasOptions; /** *

    * 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 +97,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 +142,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 +169,61 @@ * *

    */ - -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 JwtValidator jwtValidator; private boolean isInitialized = false; + protected JwtRetriever jwtRetriever; + @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); + moduleOptions = jaasOptions(saslMechanism, jaasConfigEntries); + + this.jwtRetriever = getConfiguredInstanceOrDefault( + configs, + saslMechanism, + jaasConfigEntries, + SASL_OAUTHBEARER_JWT_RETRIEVER_CLASS, + JwtRetriever.class + ); + + this.jwtValidator = getConfiguredInstanceOrDefault( + configs, + saslMechanism, + jaasConfigEntries, + SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, + JwtValidator.class + ); + + this.isInitialized = true; } - 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); - } - - isInitialized = true; - } + void configure(JwtRetriever jwtRetriever, + JwtValidator jwtValidator, + Map configs, + String saslMechanism, + List jaasConfigEntries) { + this.jwtRetriever = jwtRetriever; + this.jwtValidator = jwtValidator; - /* - * Package-visible for testing. - */ + this.jwtRetriever.configure(configs, saslMechanism, jaasConfigEntries); + this.jwtValidator.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(jwtRetriever, "jwtRetriever"); + Utils.closeQuietly(jwtValidator, "jwtValidator"); } @Override @@ -240,20 +241,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(); + String jwt = jwtRetriever.retrieve(); try { - OAuthBearerToken token = accessTokenValidator.validate(accessToken); + OAuthBearerToken token = jwtValidator.validate(jwt); callback.token(token); - } catch (ValidateException e) { + } catch (JwtValidatorException 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 +287,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..fb94057f57652 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,37 +14,28 @@ * 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; import javax.security.auth.login.AppConfigurationEntry; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.getConfiguredInstanceOrDefault; + /** *

    * OAuthBearerValidatorCallbackHandler is an {@link AuthenticateCallbackHandler} that @@ -58,8 +49,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 +63,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 +93,39 @@ * *

    */ - -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 JwtValidator jwtValidator; 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))); - } - - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver); - init(verificationKeyResolver, accessTokenValidator); + this.jwtValidator = getConfiguredInstanceOrDefault( + configs, + saslMechanism, + jaasConfigEntries, + SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS, + JwtValidator.class + ); + + this.isInitialized = true; } - public void init(CloseableVerificationKeyResolver verificationKeyResolver, AccessTokenValidator accessTokenValidator) { - this.verificationKeyResolver = verificationKeyResolver; - 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; + void configure(JwtValidator jwtValidator, + Map configs, + String saslMechanism, + List jaasConfigEntries) { + this.jwtValidator = jwtValidator; + this.jwtValidator.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(jwtValidator, "jwtValidator"); } @Override @@ -181,12 +146,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 = jwtValidator.validate(callback.tokenValue()); callback.token(token); - } catch (ValidateException e) { + } catch (JwtValidatorException e) { log.warn(e.getMessage(), e); callback.error("invalid_token", null, null); } @@ -200,81 +163,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())); + throw new IllegalStateException(String.format("To use %s, first call the configure 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); - } - - } - - /** - * 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/CachedFile.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java new file mode 100644 index 0000000000000..0c20909f98695 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java @@ -0,0 +1,119 @@ +/* + * 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 java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.function.BiFunction; + +public class CachedFile { + + public static final BiFunction NOOP_TRANSFORMER = (file, contents) -> contents; + + protected final File file; + protected final BiFunction transformer; + protected final BiFunction, Boolean> cacheRefreshPolicy; + protected CachedFileInfo cachedFileInfo; + + public CachedFile(File file, + BiFunction transformer, + BiFunction, Boolean> cacheRefreshPolicy) { + this.file = file; + this.transformer = transformer; + this.cacheRefreshPolicy = cacheRefreshPolicy; + } + + public static BiFunction, Boolean> staticCacheRefreshPolicy() { + return (file, cachedFileInfo) -> cachedFileInfo == null; + } + + public static BiFunction, Boolean> lastModifiedCacheRefreshPolicy() { + return (file, cachedFileInfo) -> cachedFileInfo == null || cachedFileInfo.lastModified() != file.lastModified(); + } + + public long size() { + return cachedFileInfo().size(); + } + + public long lastModified() { + return cachedFileInfo().lastModified(); + } + + public String contents() { + return cachedFileInfo().contents(); + } + + public T transformed() { + return cachedFileInfo().transformed(); + } + + protected CachedFileInfo cachedFileInfo() { + if (cacheRefreshPolicy.apply(file, cachedFileInfo)) { + long size = file.length(); + long lastModified = file.lastModified(); + String contents; + + try { + contents = Files.readString(file.toPath()); + } catch (IOException e) { + throw new KafkaException("Error reading the file contents of OAuth resource " + file.getPath() + " for caching"); + } + + T transformed = transformer.apply(file, contents); + cachedFileInfo = new CachedFileInfo<>(size, lastModified, contents, transformed); + } + + return cachedFileInfo; + } + + public static class CachedFileInfo { + + private final long size; + + private final long lastModified; + + private final String contents; + + private final T transformed; + + public CachedFileInfo(long size, long lastModified, String contents, T transformed) { + this.size = size; + this.lastModified = lastModified; + this.contents = contents; + this.transformed = transformed; + } + + public long size() { + return size; + } + + public long lastModified() { + return lastModified; + } + + public String contents() { + return contents; + } + + public T transformed() { + return transformed; + } + } +} 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 deleted file mode 100644 index 5bf5ef068ed0f..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java +++ /dev/null @@ -1,182 +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.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -/** - * Simple utility class to perform basic cleaning and validation on input values so that they're - * performed consistently throughout the code base. - */ - -public class ClaimValidationUtils { - - /** - * Validates that the scopes are valid, where invalid means any of - * the following: - * - *

      - *
    • Collection is null
    • - *
    • Collection has duplicates
    • - *
    • Any of the elements in the collection are null
    • - *
    • Any of the elements in the collection are zero length
    • - *
    • Any of the elements in the collection are whitespace only
    • - *
    - * - * @param scopeClaimName Name of the claim used for the scope values - * @param scopes Collection of String scopes - * - * @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 - * if any of the values in the set are null, empty, - * or whitespace only - */ - - public static Set validateScopes(String scopeClaimName, Collection scopes) throws ValidateException { - if (scopes == null) - throw new ValidateException(String.format("%s value must be non-null", scopeClaimName)); - - Set copy = new HashSet<>(); - - for (String scope : scopes) { - scope = validateString(scopeClaimName, scope); - - if (copy.contains(scope)) - throw new ValidateException(String.format("%s value must not contain duplicates - %s already present", scopeClaimName, scope)); - - copy.add(scope); - } - - return Collections.unmodifiableSet(copy); - } - - /** - * Validates that the given lifetime is valid, where invalid means any of - * the following: - * - *
      - *
    • null
    • - *
    • Negative
    • - *
    - * - * @param claimName Name of the claim - * @param claimValue Expiration time (in milliseconds) - * - * @return Input parameter, as provided - * - * @throws ValidateException Thrown if the value is null or negative - */ - - public static long validateExpiration(String claimName, Long claimValue) throws ValidateException { - if (claimValue == null) - throw new ValidateException(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)); - - return claimValue; - } - - /** - * Validates that the given claim value is valid, where invalid means any of - * the following: - * - *
      - *
    • null
    • - *
    • Zero length
    • - *
    • Whitespace only
    • - *
    - * - * @param claimName Name of the claim - * @param claimValue Name of the subject - * - * @return Trimmed version of the claimValue parameter - * - * @throws ValidateException Thrown if the value is null, empty, or whitespace only - */ - - public static String validateSubject(String claimName, String claimValue) throws ValidateException { - return validateString(claimName, claimValue); - } - - /** - * Validates that the given issued at claim name is valid, where invalid means any of - * the following: - * - *
      - *
    • Negative
    • - *
    - * - * @param claimName Name of the claim - * @param claimValue Start time (in milliseconds) or null if not used - * - * @return Input parameter, as provided - * - * @throws ValidateException Thrown if the value is negative - */ - - public static Long validateIssuedAt(String claimName, Long claimValue) throws ValidateException { - if (claimValue != null && claimValue < 0) - throw new ValidateException(String.format("%s value must be null or non-negative; value given was \"%s\"", claimName, claimValue)); - - return claimValue; - } - - /** - * Validates that the given claim name override is valid, where invalid means - * any of the following: - * - *
      - *
    • null
    • - *
    • Zero length
    • - *
    • Whitespace only
    • - *
    - * - * @param name "Standard" name of the claim, e.g. sub - * @param value "Override" name of the claim, e.g. email - * - * @return Trimmed version of the value parameter - * - * @throws ValidateException Thrown if the value is null, empty, or whitespace only - */ - - public static String validateClaimNameOverride(String name, String value) throws ValidateException { - return validateString(name, value); - } - - private static String validateString(String name, String value) throws ValidateException { - if (value == null) - throw new ValidateException(String.format("%s value must be non-null", name)); - - if (value.isEmpty()) - throw new ValidateException(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)); - - return value; - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGenerator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGenerator.java new file mode 100644 index 0000000000000..d338f18ef1967 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGenerator.java @@ -0,0 +1,99 @@ +/* + * 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.Utils; + +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLEncoder; +import java.net.http.HttpRequest; +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class ClientCredentialsRequestGenerator implements HttpRequestGenerator { + + public static final String GRANT_TYPE = "client_credentials"; + + private final URL tokenEndpoint; + private final String clientId; + private final String clientSecret; + private final Optional scope; + + public ClientCredentialsRequestGenerator(URL tokenEndpoint, + String clientId, + String clientSecret, + String scope, + boolean urlencodeHeader) { + this.tokenEndpoint = tokenEndpoint; + + // 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 generateBody() { + 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 generateHeaders() { + 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); + + Map headers = new HashMap<>(); + headers.put("Accept", "application/json"); + headers.put("Authorization", header); + headers.put("Cache-Control", "no-cache"); + headers.put("Content-Type", "application/x-www-form-urlencoded"); + return headers; + } + + @Override + public HttpRequest generateRequest() { + HttpRequest.BodyPublisher bodyPublisher = HttpRequest.BodyPublishers.ofString(generateBody()); + + URI uri; + + try { + uri = tokenEndpoint.toURI(); + } catch (URISyntaxException e) { + throw new KafkaException("An error occurred formatting the OAuth token retrieval request", e); + } + + HttpRequest.Builder builder = HttpRequest.newBuilder() + .uri(uri) + .POST(bodyPublisher); + + for (Map.Entry header : generateHeaders().entrySet()) + builder = builder.header(header.getKey(), header.getValue()); + + return builder.build(); + } +} 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..278f80f2fd835 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,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.OAuthBearerValidatorCallbackHandler; @@ -22,7 +21,6 @@ import org.jose4j.keys.resolvers.VerificationKeyResolver; import java.io.Closeable; -import java.io.IOException; /** * The {@link OAuthBearerValidatorCallbackHandler} uses a {@link VerificationKeyResolver} as @@ -36,18 +34,9 @@ * @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 - */ +public interface CloseableVerificationKeyResolver extends VerificationKeyResolver, Closeable { - default void close() throws IOException { - // This method left intentionally blank. + default void close() { + // Do nothing... } - } 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 deleted file mode 100644 index 10f700826c8bd..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java +++ /dev/null @@ -1,252 +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.ConfigException; -import org.apache.kafka.common.network.ListenerName; - -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; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; -import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT; - -/** - * 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()) - this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim()); - else - this.prefix = null; - } - - /** - * Validates that, if a value is supplied, is a file that: - * - *
  • - *
      exists
    - *
      has read permission
    - *
      points to a file
    - *
  • - * - * If the value is null or an empty string, it is assumed to be an "empty" value and thus. - * ignored. Any whitespace is trimmed off of the beginning and end. - */ - - public Path validateFile(String name) { - URL url = validateUrl(name); - File file; - - try { - file = new File(url.toURI().getRawPath()).getAbsoluteFile(); - } catch (URISyntaxException e) { - throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, url, e.getMessage())); - } - - if (!file.exists()) - throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", name, file)); - - if (!file.canRead()) - throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't have read permission", name, file)); - - 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(); - } - - /** - * Validates that, if a value is supplied, is a value that: - * - *
  • - *
      is an Integer
    - *
      has a value that is not less than the provided minimum value
    - *
  • - * - * If the value is null or an empty string, it is assumed to be an "empty" value and thus - * ignored. Any whitespace is trimmed off of the beginning and end. - */ - - public Integer validateInteger(String name, boolean isRequired) { - Integer value = get(name); - - if (value == null) { - if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); - else - return null; - } - - return value; - } - - /** - * Validates that, if a value is supplied, is a value that: - * - *
  • - *
      is an Integer
    - *
      has a value that is not less than the provided minimum value
    - *
  • - * - * If the value is null or an empty string, it is assumed to be an "empty" value and thus - * ignored. Any whitespace is trimmed off of the beginning and end. - */ - - public Long validateLong(String name) { - return validateLong(name, true); - } - - public Long validateLong(String name, boolean isRequired) { - return validateLong(name, isRequired, null); - } - - public Long validateLong(String name, boolean isRequired, Long min) { - Long value = get(name); - - if (value == null) { - if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); - else - return null; - } - - if (min != null && value < min) - throw new ConfigException(String.format("The OAuth configuration option %s value must be at least %s", name, min)); - - return value; - } - - /** - * Validates that the configured URL that: - * - *
      - *
    • is well-formed
    • - *
    • contains a scheme
    • - *
    • uses either HTTP, HTTPS, or file protocols
    • - *
    • is in the allow-list
    • - *
    - * - * No effort is made to connect to the URL in the validation step. - */ - - public URL validateUrl(String name) { - String value = validateString(name); - URL url; - - try { - url = new URL(value); - } catch (MalformedURLException e) { - throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, value, e.getMessage())); - } - - String protocol = url.getProtocol(); - - if (protocol == null || protocol.trim().isEmpty()) - throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is missing the protocol", name, value)); - - protocol = protocol.toLowerCase(Locale.ROOT); - - if (!(protocol.equals("http") || protocol.equals("https") || protocol.equals("file"))) - throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that contains an invalid protocol (%s); only \"http\", \"https\", and \"file\" protocol are supported", name, value, protocol)); - - throwIfURLIsNotAllowed(value); - - return url; - } - - public String validateString(String name) throws ValidateException { - return validateString(name, true); - } - - public String validateString(String name, boolean isRequired) throws ValidateException { - String value = get(name); - - if (value == null) { - if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s value must be non-null", name)); - else - return null; - } - - value = value.trim(); - - if (value.isEmpty()) { - if (isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s value must not contain only whitespace", name)); - else - return null; - } - - return value; - } - - public Boolean validateBoolean(String name, boolean isRequired) { - Boolean value = get(name); - - if (value == null && isRequired) - throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); - - return value; - } - - @SuppressWarnings("unchecked") - public T get(String name) { - T value = (T) configs.get(prefix + name); - - if (value != null) - return value; - - return (T) configs.get(name); - } - - // visible for testing - // make sure the url is in the "org.apache.kafka.sasl.oauthbearer.allowed.urls" system property - void throwIfURLIsNotAllowed(String value) { - Set allowedUrls = Arrays.stream( - System.getProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT).split(",")) - .map(String::trim) - .collect(Collectors.toSet()); - if (!allowedUrls.contains(value)) { - throw new ConfigException(value + " is not allowed. Update system property '" - + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG + "' to allow " + value); - } - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreator.java new file mode 100644 index 0000000000000..1302bd7c66a0c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreator.java @@ -0,0 +1,128 @@ +/* + * 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.AssertionCreator; +import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.security.GeneralSecurityException; +import java.security.KeyFactory; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.security.Signature; +import java.security.spec.PKCS8EncodedKeySpec; +import java.time.Duration; +import java.util.Base64; +import java.util.HashMap; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.staticCacheRefreshPolicy; + +public class DefaultAssertionCreator implements AssertionCreator { + + static final String TOKEN_SIGNING_ALGORITHM_RS256 = "RS256"; + static final String TOKEN_SIGNING_ALGORITHM_ES256 = "ES256"; + + private static final BiFunction PRIVATE_KEY_TRANSFORMER = (file, privateKeySecret) -> { + try { + byte[] pkcs8EncodedBytes = Base64.getDecoder().decode(privateKeySecret); + PKCS8EncodedKeySpec keySpec = new PKCS8EncodedKeySpec(pkcs8EncodedBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + return keyFactory.generatePrivate(keySpec); + } catch (GeneralSecurityException e) { + throw new KafkaException("An error occurred generating the OAuth assertion private key from " + file.getPath(), e); + } + }; + + private final Time time; + private final String algorithm; + private final CachedFile privateKeyFile; + + public DefaultAssertionCreator(Time time, String algorithm, File privateKeyFile) { + this.time = time; + this.algorithm = algorithm; + this.privateKeyFile = new CachedFile<>( + privateKeyFile, + PRIVATE_KEY_TRANSFORMER, + staticCacheRefreshPolicy() + ); + } + + @Override + public String create(AssertionJwtTemplate template) throws GeneralSecurityException, IOException { + ObjectMapper mapper = new ObjectMapper(); + Base64.Encoder encoder = Base64.getUrlEncoder().withoutPadding(); + String header = encodeHeader(template, mapper, encoder); + String payload = encodePayload(template, mapper, encoder); + String content = header + "." + payload; + PrivateKey privateKey = privateKeyFile.transformed(); + String signedContent = sign(privateKey, content); + return content + "." + signedContent; + } + + String encodeHeader(AssertionJwtTemplate template, + ObjectMapper mapper, + Base64.Encoder encoder) throws IOException { + Map values = new HashMap<>(template.header()); + values.put("alg", algorithm); + values.put("typ", "JWT"); + + String json = mapper.writeValueAsString(values); + return encoder.encodeToString(Utils.utf8(json)); + } + + String encodePayload(AssertionJwtTemplate template, + ObjectMapper mapper, + Base64.Encoder encoder) throws IOException { + long currentTimeSecs = time.milliseconds() / 1000L; + long expirationSecs = currentTimeSecs + Duration.ofMinutes(60).toSeconds(); + + Map values = new HashMap<>(template.payload()); + values.put("iat", currentTimeSecs); + values.put("exp", expirationSecs); + + String json = mapper.writeValueAsString(values); + return encoder.encodeToString(Utils.utf8(json)); + } + + Signature getSignature() throws GeneralSecurityException { + if (algorithm.equalsIgnoreCase(TOKEN_SIGNING_ALGORITHM_RS256)) { + return Signature.getInstance("SHA256withRSA"); + } else if (algorithm.equalsIgnoreCase(TOKEN_SIGNING_ALGORITHM_ES256)) { + return Signature.getInstance("SHA256withECDSA"); + } else { + throw new NoSuchAlgorithmException(String.format("Unsupported signing algorithm: %s", algorithm)); + } + } + + String sign(PrivateKey privateKey, String contentToSign) throws GeneralSecurityException { + 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/FileAssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileAssertionCreator.java new file mode 100644 index 0000000000000..692e3b7550d43 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileAssertionCreator.java @@ -0,0 +1,46 @@ +/* + * 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.AssertionCreator; +import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate; + +import java.io.File; +import java.io.IOException; +import java.security.GeneralSecurityException; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.NOOP_TRANSFORMER; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.staticCacheRefreshPolicy; + +public class FileAssertionCreator implements AssertionCreator { + + private final CachedFile assertionFile; + + public FileAssertionCreator(File assertionFile) { + try { + this.assertionFile = new CachedFile<>(assertionFile, NOOP_TRANSFORMER, staticCacheRefreshPolicy()); + } catch (Exception e) { + throw new KafkaException("An error occurred reading the OAuth JWT from " + assertionFile); + } + } + + @Override + public String create(AssertionJwtTemplate template) throws GeneralSecurityException, IOException { + return assertionFile.transformed(); + } +} 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/FileTokenRetriever.java deleted file mode 100644 index c145cf7596959..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java +++ /dev/null @@ -1,57 +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.Utils; - -import java.io.IOException; -import java.nio.file.Path; - -/** - * FileTokenRetriever is an {@link AccessTokenRetriever} that will load the contents, - * interpreting them as a JWT access key in the serialized form. - * - * @see AccessTokenRetriever - */ - -public class FileTokenRetriever implements AccessTokenRetriever { - - private final Path accessTokenFile; - - private String accessToken; - - public FileTokenRetriever(Path accessTokenFile) { - this.accessTokenFile = accessTokenFile; - } - - @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(); - } - - @Override - public String retrieve() throws IOException { - if (accessToken == null) - throw new IllegalStateException("Access token is null; please call init() first"); - - return accessToken; - } - -} 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/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/HttpRequestGenerator.java similarity index 74% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/UnretryableException.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestGenerator.java index a53ae6ec115c7..b5a7c34985969 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/HttpRequestGenerator.java @@ -14,15 +14,21 @@ * 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 java.io.Closeable; +import java.net.http.HttpRequest; +import java.util.Map; -public class UnretryableException extends KafkaException { +public interface HttpRequestGenerator extends Closeable { - public UnretryableException(Throwable cause) { - super(cause); - } + String generateBody(); + + Map generateHeaders(); + HttpRequest generateRequest(); + + default void close() { + // Do nothing... + } } 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 deleted file mode 100644 index 3e49595dbc1b2..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java +++ /dev/null @@ -1,111 +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.AbstractConfig; -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.network.ConnectionMode; -import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; -import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory; -import org.apache.kafka.common.security.ssl.SslFactory; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.net.URL; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -import javax.net.ssl.SSLSocketFactory; -import javax.security.auth.login.AppConfigurationEntry; - -/** - * 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); - - private final Map options; - - public JaasOptionsUtils(Map options) { - this.options = options; - } - - public static Map getOptions(String saslMechanism, List jaasConfigEntries) { - if (!OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(saslMechanism)) - throw new IllegalArgumentException(String.format("Unexpected SASL mechanism: %s", saslMechanism)); - - if (Objects.requireNonNull(jaasConfigEntries).size() != 1 || jaasConfigEntries.get(0) == null) - throw new IllegalArgumentException(String.format("Must supply exactly 1 non-null JAAS mechanism configuration (size was %d)", jaasConfigEntries.size())); - - return Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions()); - } - - public boolean shouldCreateSSLSocketFactory(URL url) { - return url.getProtocol().equalsIgnoreCase("https"); - } - - public Map getSslClientConfig() { - ConfigDef sslConfigDef = new ConfigDef(); - sslConfigDef.withClientSslSupport(); - AbstractConfig sslClientConfig = new AbstractConfig(sslConfigDef, options); - 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 { - return validateString(name, true); - } - - public String validateString(String name, boolean isRequired) throws ValidateException { - 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)); - else - return null; - } - - value = value.trim(); - - if (value.isEmpty()) { - if (isRequired) - throw new ConfigException(String.format("The OAuth configuration 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..e143a3e6beb8c 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.File; import java.io.IOException; -import java.nio.file.Path; import java.security.Key; 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.security.oauthbearer.internals.secured.OAuthBearerUtils.validateFileUrl; /** * JwksFileVerificationKeyResolver is a {@link VerificationKeyResolver} implementation @@ -76,33 +80,28 @@ * 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()); + private final VerificationKeyResolver delegate; + public JwksFileVerificationKeyResolver(Map configs, String saslMechanism) { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism); + File jwksFile = validateFileUrl(oauthConfig, 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 (IOException e) { + throw new KafkaException("An error occurred loading JWKS data from " + jwksFile.getPath(), e); + } catch (Exception e) { + throw new KafkaException(e); } delegate = new JwksVerificationKeyResolver(jwks.getJsonWebKeys()); @@ -110,10 +109,6 @@ public void init() throws IOException { @Override public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException { - if (delegate == null) - throw new UnresolvableKeyException("VerificationKeyResolver delegate is null; please call init() first"); - return delegate.resolveKey(jws, nestingContext); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGenerator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGenerator.java new file mode 100644 index 0000000000000..ba51287a3eecd --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGenerator.java @@ -0,0 +1,101 @@ +/* + * 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.AssertionCreator; +import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate; +import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException; +import org.apache.kafka.common.utils.Utils; + +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLEncoder; +import java.net.http.HttpRequest; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; + +public class JwtBearerRequestGenerator implements HttpRequestGenerator { + + public static final String GRANT_TYPE = "urn:ietf:params:oauth:grant-type:jwt-bearer"; + + private final URL tokenEndpoint; + private final AssertionCreator assertionCreator; + private final AssertionJwtTemplate assertionJwtTemplate; + + public JwtBearerRequestGenerator(URL tokenEndpoint, + AssertionCreator assertionCreator, + AssertionJwtTemplate assertionJwtTemplate) { + this.tokenEndpoint = tokenEndpoint; + this.assertionCreator = assertionCreator; + this.assertionJwtTemplate = assertionJwtTemplate; + } + + @Override + public String generateBody() { + String assertion; + + try { + assertion = assertionCreator.create(assertionJwtTemplate); + } catch (Exception e) { + throw new JwtRetrieverException("Error signing OAuth 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 generateHeaders() { + Map headers = new HashMap<>(); + headers.put("Accept", "application/json"); + headers.put("Cache-Control", "no-cache"); + headers.put("Content-Type", "application/x-www-form-urlencoded"); + return headers; + } + + @Override + public HttpRequest generateRequest() { + HttpRequest.BodyPublisher bodyPublisher = HttpRequest.BodyPublishers.ofString(generateBody()); + + URI uri; + + try { + uri = tokenEndpoint.toURI(); + } catch (URISyntaxException e) { + throw new KafkaException("An error occurred formatting the OAuth token retrieval request", e); + } + + HttpRequest.Builder builder = HttpRequest.newBuilder() + .uri(uri) + .POST(bodyPublisher); + + for (Map.Entry header : generateHeaders().entrySet()) + builder = builder.header(header.getKey(), header.getValue()); + + return builder.build(); + } + + @Override + public void close() { + Utils.closeQuietly(assertionCreator, "assertionCreator"); + Utils.closeQuietly(assertionJwtTemplate, "assertionJwtTemplate"); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpClient.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpClient.java new file mode 100644 index 0000000000000..0bd855c02f917 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpClient.java @@ -0,0 +1,149 @@ +/* + * 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.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException; +import org.apache.kafka.common.utils.Time; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.util.Collections; +import java.util.Set; + +/** + * A {@link JwtRetriever} that will communicate with an OAuth/OIDC provider directly via HTTP. + * + * @see JwtRetriever + */ +public final class JwtHttpClient { + + private static final Logger log = LoggerFactory.getLogger(JwtHttpClient.class); + + public static final Set DEFAULT_SUCCESS_STATUS_CODES; + + public static final Set DEFAULT_FATAL_STATUS_CODES; + + static { + DEFAULT_SUCCESS_STATUS_CODES = Set.of( + HttpURLConnection.HTTP_OK, + HttpURLConnection.HTTP_CREATED + ); + + // 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. + DEFAULT_FATAL_STATUS_CODES = Set.of( + HttpURLConnection.HTTP_BAD_REQUEST, + HttpURLConnection.HTTP_UNAUTHORIZED, + HttpURLConnection.HTTP_PAYMENT_REQUIRED, + HttpURLConnection.HTTP_FORBIDDEN, + HttpURLConnection.HTTP_NOT_FOUND, + HttpURLConnection.HTTP_BAD_METHOD, + HttpURLConnection.HTTP_NOT_ACCEPTABLE, + HttpURLConnection.HTTP_PROXY_AUTH, + HttpURLConnection.HTTP_CONFLICT, + HttpURLConnection.HTTP_GONE, + HttpURLConnection.HTTP_LENGTH_REQUIRED, + HttpURLConnection.HTTP_PRECON_FAILED, + HttpURLConnection.HTTP_ENTITY_TOO_LARGE, + HttpURLConnection.HTTP_REQ_TOO_LONG, + HttpURLConnection.HTTP_UNSUPPORTED_TYPE, + HttpURLConnection.HTTP_NOT_IMPLEMENTED, + HttpURLConnection.HTTP_VERSION + ); + } + + private final Time time; + private final Set successStatusCodes; + private final Set fatalStatusCodes; + + public JwtHttpClient(Time time) { + this(time, DEFAULT_SUCCESS_STATUS_CODES, DEFAULT_FATAL_STATUS_CODES); + } + + public JwtHttpClient(Time time, + Set successStatusCodes, + Set fatalStatusCodes) { + this.time = time; + this.successStatusCodes = Collections.unmodifiableSet(successStatusCodes); + this.fatalStatusCodes = Collections.unmodifiableSet(fatalStatusCodes); + } + + public String request(HttpClient client, + HttpRequest request, + HttpResponse.BodyHandler responseHandler, + long retryBackoffMs, + long retryBackoffMaxMs) throws JwtRetrieverException { + long endMs = time.milliseconds() + retryBackoffMaxMs; + int currAttempt = 0; + + while (time.milliseconds() <= endMs) { + currAttempt++; + + try { + HttpResponse response = client.send(request, responseHandler); + int statusCode = response.statusCode(); + + if (successStatusCodes.contains(statusCode)) { + return response.body(); + } else if (fatalStatusCodes.contains(statusCode)) { + // This is a non-transient error, so don't bother retrying the request unnecessarily. + throw new JwtRetrieverException( + String.format( + "The fatal status code %s was encountered on attempt %s to retrieve the JWT from the OAuth token endpoint", + statusCode, + currAttempt + ) + ); + } else { + long waitMs = retryBackoffMs * (long) Math.pow(2, currAttempt - 1); + long diff = endMs - time.milliseconds(); + waitMs = Math.min(waitMs, diff); + + if (waitMs <= 0) + break; + + log.warn( + "The status code {} was encountered on attempt {} to retrieve the JWT from the OAuth token endpoint; sleeping {} ms before attempting again", + statusCode, + currAttempt, + waitMs + ); + + time.sleep(waitMs); + } + } catch (IOException | InterruptedException e) { + throw new JwtRetrieverException(e); + } + } + + throw new JwtRetrieverException( + String.format( + "%s failed attempts were made to retrieve the JWT from the OAuth token endpoint; will not attempt further", + currAttempt + ) + ); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpResponseBodyHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpResponseBodyHandler.java new file mode 100644 index 0000000000000..687784290c635 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpResponseBodyHandler.java @@ -0,0 +1,97 @@ +/* + * 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.JwtRetrieverException; +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.net.http.HttpResponse; +import java.util.Collections; +import java.util.List; + +public class JwtHttpResponseBodyHandler implements HttpResponse.BodyHandler { + + public static final List DEFAULT_JSON_PATHS = List.of("/id_token", "/access_token"); + + private static final Logger log = LoggerFactory.getLogger(JwtHttpResponseBodyHandler.class); + private static final int MAX_RESPONSE_BODY_LENGTH = 1000; + + private final List jsonPaths; + + public JwtHttpResponseBodyHandler() { + this(DEFAULT_JSON_PATHS); + } + + public JwtHttpResponseBodyHandler(List jsonPaths) { + this.jsonPaths = Collections.unmodifiableList(jsonPaths); + } + + @Override + public HttpResponse.BodySubscriber apply(HttpResponse.ResponseInfo responseInfo) { + return HttpResponse.BodySubscribers.mapping( + HttpResponse.BodyHandlers.ofString().apply(responseInfo), + this::extractJwt + ); + } + + public String extractJwt(String responseBody) throws JwtRetrieverException { + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode; + + try { + rootNode = mapper.readTree(responseBody); + } catch (Exception e) { + throw new JwtRetrieverException("An unexpected error occurred parsing the JSON from the OAuth token retrieval response", e); + } + + for (String jsonPath : jsonPaths) { + JsonNode node = rootNode.at(jsonPath); + + if (node == null) { + log.debug("The JSON path {} did not yield a node in the OAuth token retrieval response", jsonPath); + continue; + } + + String jwt = node.textValue(); + + if (Utils.isBlank(jwt)) { + log.debug("The JSON path {} yielded a node in the OAuth token retrieval response, but the value was null, blank, or whitespace", jsonPath); + continue; + } + + return jwt.trim(); + } + + // 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 JwtRetrieverException(String.format("The token endpoint response did not contain a JWT value. Response: (%s)", snippet)); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerAbstractConfig.java similarity index 66% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerAbstractConfig.java index 0a38f2b5094d5..3be0fb2830dad 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerAbstractConfig.java @@ -14,22 +14,18 @@ * 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 { +/** + * ConfigurationUtils is a utility class to perform basic configuration-related + * logic and is separated out here for easier, more direct testing. + */ +public abstract class OAuthBearerAbstractConfig { - /** - * 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 abstract String getString(String key); - default void init() throws IOException { - // This method left intentionally blank. - } + public abstract boolean containsKey(String key); + public abstract T get(String key); } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfig.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfig.java new file mode 100644 index 0000000000000..8b972ab10f50b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfig.java @@ -0,0 +1,123 @@ +/* + * 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.ConfigException; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.utils.Utils; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * OAuthBearerConfig is a utility class to perform basic configuration-related + * logic and is separated out here for easier, more direct testing. + */ +public class OAuthBearerConfig extends OAuthBearerAbstractConfig { + + private final Map configs; + + private final String prefix; + + public OAuthBearerConfig(Map configs, String saslMechanism) { + this.configs = configs; + + if (!Utils.isBlank(saslMechanism)) + this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim()); + else + this.prefix = null; + } + + public Short getShort(String key) { + return (Short) get(key); + } + + public Integer getInt(String key) { + return (Integer) get(key); + } + + public Optional maybeGetInt(String key) { + if (containsKey(key)) + return Optional.of(getInt(key)); + + return Optional.empty(); + } + + public Long getLong(String key) { + return (Long) get(key); + } + + public Double getDouble(String key) { + return (Double) get(key); + } + + @SuppressWarnings("unchecked") + public List getList(String key) { + return (List) get(key); + } + + public Boolean getBoolean(String key) { + return (Boolean) get(key); + } + + public Optional maybeGetBoolean(String key) { + if (containsKey(key)) + return Optional.of(getBoolean(key)); + + return Optional.empty(); + } + + @Override + public String getString(String key) { + String s = get(key); + + if (Utils.isBlank(s)) { + throw new ConfigException("No value was found for the OAuth configuration " + key); + } else { + return s.trim(); + } + } + + public Optional maybeGetString(String key) { + if (containsKey(key)) + return Optional.of(getString(key)); + + return Optional.empty(); + } + + @Override + public boolean containsKey(String key) { + return configs.get(key) != null || configs.get(prefix + key) != null; + } + + @Override + @SuppressWarnings("unchecked") + public T get(String key) { + T value = (T) configs.get(prefix + key); + + if (value != null) + return value; + + value = (T) configs.get(key); + + if (value != null) + return value; + + throw new ConfigException("No value was found for the OAuth configuration " + key); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerJaasConfig.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerJaasConfig.java new file mode 100644 index 0000000000000..8b07bb073c5a9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerJaasConfig.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.config.ConfigException; +import org.apache.kafka.common.utils.Utils; + +import java.util.Collections; +import java.util.Map; + +import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG; + +/** + * OAuthBearerJaasConfig is a utility class to perform logic for the JAAS options and + * is separated out here for easier, more direct testing. + */ +public class OAuthBearerJaasConfig extends OAuthBearerAbstractConfig { + + private final Map options; + + public OAuthBearerJaasConfig(Map options) { + this.options = Collections.unmodifiableMap(options); + } + + public Map options() { + return options; + } + + @Override + public String getString(String key) { + String s = get(key); + + if (Utils.isBlank(s)) { + throw new ConfigException("No value was found for the OAuth option " + key + " in " + SASL_JAAS_CONFIG); + } else { + return s.trim(); + } + } + + @Override + public boolean containsKey(String key) { + return options.get(key) != null; + } + + @Override + @SuppressWarnings("unchecked") + public T get(String key) { + T value = (T) options.get(key); + + if (value != null) + return value; + + throw new ConfigException("No value was found for the OAuth option " + key + " in " + SASL_JAAS_CONFIG); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtils.java new file mode 100644 index 0000000000000..399050e06e4ea --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtils.java @@ -0,0 +1,385 @@ +/* + * 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.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.security.oauthbearer.ClientCredentialsJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.JwtValidatorException; +import org.apache.kafka.common.security.oauthbearer.OAuthBearerConfigurable; +import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; +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.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +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; +import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; +import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT; + +public class OAuthBearerUtils { + + public static Map jaasOptions(String saslMechanism, List jaasConfigEntries) { + if (!OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(saslMechanism)) + throw new ConfigException(String.format("Unexpected SASL mechanism: %s", saslMechanism)); + + if (Objects.requireNonNull(jaasConfigEntries).size() != 1 || jaasConfigEntries.get(0) == null) + throw new ConfigException(String.format("Must supply exactly 1 non-null JAAS mechanism configuration (size was %d)", jaasConfigEntries.size())); + + return Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions()); + } + + public static boolean protocolMatches(URL url, String protocol) { + return protocol.equalsIgnoreCase(url.getProtocol()); + } + + public static Optional maybeCreateSslResource(URL url, OAuthBearerJaasConfig jaasConfig) { + if (protocolMatches(url, "https")) { + Map sslClientConfig = getSslClientConfig(jaasConfig); + return Optional.of(SslResource.create(sslClientConfig)); + } else { + return Optional.empty(); + } + } + + public static Map getSslClientConfig(OAuthBearerJaasConfig jaasConfig) { + ConfigDef sslConfigDef = new ConfigDef(); + sslConfigDef.withClientSslSupport(); + AbstractConfig sslClientConfig = new AbstractConfig(sslConfigDef, jaasConfig.options()); + return sslClientConfig.values(); + } + + // visible for testing + // make sure the url is in the "org.apache.kafka.sasl.oauthbearer.allowed.urls" system property + public static void throwIfURLIsNotAllowed(String value) { + Set allowedUrls = Arrays.stream( + System.getProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT).split(",")) + .map(String::trim) + .collect(Collectors.toSet()); + if (!allowedUrls.contains(value)) { + throw new ConfigException(value + " is not allowed. Update system property '" + + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG + "' to allow " + value); + } + } + + public static T getConfiguredInstanceOrDefault(Map configs, + String saslMechanism, + List jaasConfigEntries, + String configName, + Class clazz) { + Object classOrClassName = configs.get(configName); + Object o; + + if (classOrClassName instanceof String) { + try { + o = Utils.newInstance((String) classOrClassName, clazz); + } catch (ClassNotFoundException e) { + throw new KafkaException("Class " + classOrClassName + " cannot be found", e); + } + } else if (classOrClassName instanceof Class) { + o = Utils.newInstance((Class) classOrClassName); + } else { + throw new KafkaException("Unexpected element of type " + classOrClassName.getClass().getName() + ", expected String or Class"); + } + + if (!clazz.isInstance(o)) + throw new KafkaException(classOrClassName + " is not an instance of " + clazz.getName()); + + try { + if (o instanceof OAuthBearerConfigurable) + ((OAuthBearerConfigurable) o).configure(configs, saslMechanism, jaasConfigEntries); + } catch (Exception e) { + Utils.closeQuietly((AutoCloseable) o, "AutoCloseable object constructed and configured during failed call to configure()"); + throw e; + } + + return clazz.cast(o); + } + + /** + * 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 ClientCredentialsJwtRetriever} constructor. + */ + public static boolean urlencodeHeader(OAuthBearerConfig oauthConfig) { + if (oauthConfig.containsKey(SASL_OAUTHBEARER_HEADER_URLENCODE)) + return oauthConfig.getBoolean(SASL_OAUTHBEARER_HEADER_URLENCODE); + else + return DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; + } + + /** + * Validates that the scopes are valid, where invalid means any of + * the following: + * + *

      + *
    • Collection is null
    • + *
    • Collection has duplicates
    • + *
    • Any of the elements in the collection are null
    • + *
    • Any of the elements in the collection are zero length
    • + *
    • Any of the elements in the collection are whitespace only
    • + *
    + * + * @param scopeClaimName Name of the claim used for the scope values + * @param scopes Collection of String scopes + * + * @return Unmodifiable {@link Set} that includes the values of the original set, but with + * each value trimmed + * + * @throws JwtValidatorException 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 validateClaimScopes(String scopeClaimName, Collection scopes) throws JwtValidatorException { + if (scopes == null) + throw new JwtValidatorException(String.format("%s value must be non-null", scopeClaimName)); + + Set copy = new HashSet<>(); + + for (String scope : scopes) { + scope = validateClaimValue(scopeClaimName, scope); + + if (copy.contains(scope)) + throw new JwtValidatorException(String.format("%s value must not contain duplicates - %s already present", scopeClaimName, scope)); + + copy.add(scope); + } + + return Collections.unmodifiableSet(copy); + } + + /** + * Validates that the given lifetime is valid, where invalid means any of + * the following: + * + *
      + *
    • null
    • + *
    • Negative
    • + *
    + * + * @param claimName Name of the claim + * @param claimValue Expiration time (in milliseconds) + * + * @return Input parameter, as provided + * + * @throws JwtValidatorException Thrown if the value is null or negative + */ + public static long validateClaimExpiration(String claimName, Long claimValue) throws JwtValidatorException { + if (claimValue == null) + throw new JwtValidatorException(String.format("%s value must be non-null", claimName)); + + if (claimValue < 0) + throw new JwtValidatorException(String.format("%s value must be non-negative; value given was \"%s\"", claimName, claimValue)); + + return claimValue; + } + + /** + * Validates that the given claim value is valid, where invalid means any of + * the following: + * + *
      + *
    • null
    • + *
    • Zero length
    • + *
    • Whitespace only
    • + *
    + * + * @param claimName Name of the claim + * @param claimValue Name of the subject + * + * @return Trimmed version of the claimValue parameter + * + * @throws JwtValidatorException Thrown if the value is null, empty, or whitespace only + */ + + public static String validateClaimSubject(String claimName, String claimValue) throws JwtValidatorException { + return validateClaimValue(claimName, claimValue); + } + + /** + * Validates that the given issued at claim name is valid, where invalid means any of + * the following: + * + *
      + *
    • Negative
    • + *
    + * + * @param claimName Name of the claim + * @param claimValue Start time (in milliseconds) or null if not used + * + * @return Input parameter, as provided + * + * @throws JwtValidatorException Thrown if the value is negative + */ + + public static Long validateClaimIssuedAt(String claimName, Long claimValue) throws JwtValidatorException { + if (claimValue != null && claimValue < 0) + throw new JwtValidatorException(String.format("%s value must be null or non-negative; value given was \"%s\"", claimName, claimValue)); + + return claimValue; + } + + /** + * Validates that the given claim name override is valid, where invalid means + * any of the following: + * + *
      + *
    • null
    • + *
    • Zero length
    • + *
    • Whitespace only
    • + *
    + * + * @param name "Standard" name of the claim, e.g. sub + * @param value "Override" name of the claim, e.g. email + * + * @return Trimmed version of the value parameter + * + * @throws JwtValidatorException Thrown if the value is null, empty, or whitespace only + */ + + public static String validateClaimNameOverride(String name, String value) throws JwtValidatorException { + return validateClaimValue(name, value); + } + + public static String validateClaimValue(String name, String value) throws JwtValidatorException { + if (Utils.isBlank(value)) + throw new JwtValidatorException(String.format("The value of the OAuth claim %s must be non-null, non-empty, and non-whitespace", name)); + + return value.trim(); + } + + /** + * Validates that, if a value is supplied, is a file that: + * + *
  • + *
      exists
    + *
      has read permission
    + *
      points to a file
    + *
  • + * + * If the value is null or an empty string, it is assumed to be an "empty" value and thus. + * ignored. Any whitespace is trimmed off of the beginning and end. + */ + public static File validateFile(OAuthBearerAbstractConfig config, String key) { + String fileName = config.getString(key); + File file = new File(fileName); + return validateFile(key, file); + } + + /** + * Validates that, if a value is supplied, is a file that: + * + *
  • + *
      exists
    + *
      has read permission
    + *
      points to a file
    + *
  • + * + * If the value is null or an empty string, it is assumed to be an "empty" value and thus. + * ignored. Any whitespace is trimmed off of the beginning and end. + */ + public static File validateFileUrl(OAuthBearerAbstractConfig config, String key) { + URL url = validateUrl(config, key); + File file; + + try { + file = new File(url.toURI().getRawPath()).getAbsoluteFile(); + } catch (URISyntaxException e) { + throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", key, url, e.getMessage())); + } + + return validateFile(key, file); + } + + /** + * Validates that the file: + * + *
  • + *
      exists
    + *
      has read permission
    + *
      points to a file
    + *
  • + */ + public static File validateFile(String key, File file) { + if (!file.exists()) + throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", key, file)); + + if (!file.canRead()) + throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't have read permission", key, file)); + + if (file.isDirectory()) + throw new ConfigException(String.format("The OAuth configuration option %s references a directory (%s), not a file", key, file)); + + return file; + } + + /** + * Validates that the configured URL that: + * + *
      + *
    • is well-formed
    • + *
    • contains a scheme
    • + *
    • uses either HTTP, HTTPS, or file protocols
    • + *
    • is in the allow-list
    • + *
    + * + * No effort is made to connect to the URL in the validation step. + */ + public static URL validateUrl(OAuthBearerAbstractConfig config, String key) { + String value = config.getString(key); + URL url; + + try { + url = new URL(value); + } catch (MalformedURLException e) { + throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", key, value, e.getMessage())); + } + + String protocol = url.getProtocol(); + + if (!protocolMatches(url, "https") && !protocolMatches(url, "http") && !protocolMatches(url, "file")) + throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that contains an invalid protocol (%s); only \"http\", \"https\", and \"file\" protocol are supported", key, value, protocol)); + + throwIfURLIsNotAllowed(value); + + return url; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefCountingMap.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefCountingMap.java new file mode 100644 index 0000000000000..20aab1851f53f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefCountingMap.java @@ -0,0 +1,48 @@ +/* + * 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.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +public class RefCountingMap { + + private final Map values = new HashMap<>(); + + private final Map counters = new HashMap<>(); + + public synchronized V get(K key, Function mappingFunction) { + V value = values.computeIfAbsent(key, mappingFunction); + counters.computeIfAbsent(key, k -> new AtomicInteger()).incrementAndGet(); + return value; + } + + public synchronized void release(K key) { + int currentCount = counters.computeIfAbsent(key, k -> new AtomicInteger()).decrementAndGet(); + + if (currentCount == 0) { + V value = values.remove(key); + + // maybeCloseQuietly includes checks for AutoCloseable and null. + Utils.maybeCloseQuietly(value, "value for " + key); + } + } +} 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..4306829813799 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,11 +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; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.oauthbearer.DefaultJwtValidator; import org.apache.kafka.common.utils.Time; +import org.jose4j.http.Get; import org.jose4j.jwk.HttpsJwks; import org.jose4j.jwk.JsonWebKey; import org.jose4j.lang.JoseException; @@ -31,14 +33,15 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; +import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.net.ssl.SSLContext; + /** * 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 +52,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 DefaultJwtValidator} to use in validating the signature of * a JWT. * * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver * @see org.jose4j.keys.resolvers.VerificationKeyResolver - * @see ValidatorAccessTokenValidator + * @see DefaultJwtValidator */ - -public final class RefreshingHttpsJwks implements Initable, Closeable { +public class RefreshingHttpsJwks implements Closeable { private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class); @@ -71,6 +73,8 @@ public final class RefreshingHttpsJwks implements Initable, Closeable { private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS; + private final Time time; + /** * {@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 @@ -81,7 +85,7 @@ public final class RefreshingHttpsJwks implements Initable, Closeable { * 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 + * {@link HttpsJwks#refresh()}) from within the constructor as it's called only at * startup, and we can afford the blocking hit there. */ @@ -89,143 +93,97 @@ public final class RefreshingHttpsJwks implements Initable, Closeable { 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}. - */ - - private final ReadWriteLock refreshLock = new ReentrantReadWriteLock(); - private final Map missingKeyIds; /** * Flag to prevent concurrent refresh invocations. */ - private final AtomicBoolean refreshInProgressFlag = new AtomicBoolean(false); + /** + * Protects {@link #missingKeyIds} and {@link #jsonWebKeys}. + */ + private final ReadWriteLock refreshLock = new ReentrantReadWriteLock(); + /** * 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 * calls to {@link #refresh()}. */ - 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. - */ - - // 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"); - - this.httpsJwks = httpsJwks; + public RefreshingHttpsJwks(Time time, + HttpsJwks httpsJwks, + Optional sslContext, + ScheduledExecutorService executorService, + long refreshMs, + long refreshRetryBackoffMs, + long refreshRetryBackoffMaxMs) { this.time = time; - this.refreshMs = refreshMs; + this.httpsJwks = httpsJwks; + this.executorService = executorService; this.refreshRetryBackoffMs = refreshRetryBackoffMs; this.refreshRetryBackoffMaxMs = refreshRetryBackoffMaxMs; - this.executorService = executorService; this.missingKeyIds = new LinkedHashMap<>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) { @Override protected boolean removeEldestEntry(Map.Entry eldest) { return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES; } }; - } - /** - * 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 - */ + sslContext.ifPresent(c -> { + Get get = new Get(); + get.setSslSocketFactory(c.getSocketFactory()); + httpsJwks.setSimpleHttpGet(get); + }); - 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 { - try { - log.debug("init started"); + this.httpsJwks.setDefaultCacheDuration(refreshMs); - List localJWKs; + List localJWKs; - try { - localJWKs = httpsJwks.getJsonWebKeys(); - } catch (JoseException e) { - throw new IOException("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); + try { + localJWKs = httpsJwks.getJsonWebKeys(); + } catch (Exception e) { + throw new KafkaException("Could not refresh JWKS", e); + } - log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs); + try { + refreshLock.writeLock().lock(); + jsonWebKeys = Collections.unmodifiableList(localJWKs); } finally { - isInitialized = true; - - log.debug("init completed"); + 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 + ); + + log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs); } @Override public void close() { try { - log.debug("close started"); - - try { - log.debug("JWKS validation key refresh thread shutting down"); - executorService.shutdown(); + 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); } - } finally { - log.debug("close completed"); + } catch (Exception e) { + log.warn("JWKS validation key refresh thread error during close", e); } } @@ -234,7 +192,7 @@ public void close() { * sometimes called internal to {@link HttpsJwks#getJsonWebKeys()}. We want to avoid any * blocking I/O as this code is running in the authentication path on the Kafka network thread. *

    - * The list may be stale up to {@link #refreshMs}. + * The list may be stale up to the refresh interval. * * @return {@link List} of {@link JsonWebKey} instances * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs @@ -242,9 +200,6 @@ public void close() { */ public List getJsonWebKeys() throws JoseException, IOException { - if (!isInitialized) - throw new IllegalStateException("Please call init() first"); - try { refreshLock.readLock().lock(); return jsonWebKeys; @@ -257,6 +212,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 +235,12 @@ public String getLocation() { *

    * *

    - * The scheduled refresh is scheduled in {@link #init()} and runs every - * {@link #refreshMs} milliseconds. An expedited refresh is performed when an + * The scheduled refresh is scheduled in the constructor and runs at interval + * defined by the refresh configuration. 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"); @@ -284,20 +249,46 @@ private void refresh() { try { log.info("OAuth JWKS refresh of {} starting", httpsJwks.getLocation()); - Retry> retry = new Retry<>(refreshRetryBackoffMs, refreshRetryBackoffMaxMs); - List localJWKs = retry.execute(() -> { + + List localJWKs = null; + long endMs = time.milliseconds() + refreshRetryBackoffMaxMs; + int currAttempt = 0; + + while (time.milliseconds() <= endMs) { + currAttempt++; + 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; - } catch (Exception e) { - throw new ExecutionException(e); + localJWKs = refreshJsonWebKeys(); + break; + } catch (IOException | JoseException e) { + long waitMs = refreshRetryBackoffMs * (long) Math.pow(2, currAttempt - 1); + long diff = endMs - time.milliseconds(); + waitMs = Math.min(waitMs, diff); + + if (waitMs <= 0) + break; + + log.warn( + "An error was encountered on attempt {} to retrieve the OAuth JWKS from {}; sleeping {} ms before attempting again", + currAttempt, + httpsJwks.getLocation(), + waitMs, + e + ); + + time.sleep(waitMs); } - }); + } + + if (localJWKs == null) { + log.warn( + "{} attempts were made to refresh the OAuth JWKS from {}, but none were successful; not updating local JWKS cache", + currAttempt, + httpsJwks.getLocation() + ); + + return; + } try { refreshLock.writeLock().lock(); @@ -311,8 +302,6 @@ private void refresh() { } log.info("OAuth JWKS refresh of {} complete", httpsJwks.getLocation()); - } catch (ExecutionException e) { - log.warn("OAuth JWKS refresh of {} encountered an error; not updating local JWKS cache", httpsJwks.getLocation(), e); } finally { refreshInProgressFlag.set(false); } @@ -321,7 +310,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 +325,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 +363,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..b9a4c518cdec5 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,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.utils.Utils; + import org.jose4j.jwk.HttpsJwks; import org.jose4j.jwk.JsonWebKey; import org.jose4j.jwk.VerificationJwkSelector; @@ -80,51 +81,26 @@ * @see RefreshingHttpsJwks * @see HttpsJwks */ - public class RefreshingHttpsJwksVerificationKeyResolver implements CloseableVerificationKeyResolver { private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwksVerificationKeyResolver.class); - private final RefreshingHttpsJwks refreshingHttpsJwks; - private final VerificationJwkSelector verificationJwkSelector; - private boolean isInitialized; + private final RefreshingHttpsJwks refreshingHttpsJwks; public RefreshingHttpsJwksVerificationKeyResolver(RefreshingHttpsJwks refreshingHttpsJwks) { this.refreshingHttpsJwks = refreshingHttpsJwks; this.verificationJwkSelector = new VerificationJwkSelector(); } - @Override - public void init() throws IOException { - try { - log.debug("init started"); - - refreshingHttpsJwks.init(); - } finally { - isInitialized = true; - - log.debug("init completed"); - } - } - @Override public void close() { - try { - log.debug("close started"); - - refreshingHttpsJwks.close(); - } finally { - log.debug("close completed"); - } + Utils.closeQuietly(refreshingHttpsJwks, "refreshingHttpsJwks"); } @Override public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException { - if (!isInitialized) - throw new IllegalStateException("Please call init() first"); - try { List jwks = refreshingHttpsJwks.getJsonWebKeys(); JsonWebKey jwk = verificationJwkSelector.select(jws, jwks); @@ -148,5 +124,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 deleted file mode 100644 index 0da92e4fc7f72..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retry.java +++ /dev/null @@ -1,108 +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.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.ExecutionException; - -/** - * Retry encapsulates the mechanism to perform a retry and then exponential - * backoff using provided wait times between attempts. - * - * @param Result type - */ - -public class Retry { - - private static final Logger log = LoggerFactory.getLogger(Retry.class); - - private final Time time; - - private final long retryBackoffMs; - - private final long retryBackoffMaxMs; - - public Retry(long retryBackoffMs, long retryBackoffMaxMs) { - this(Time.SYSTEM, retryBackoffMs, retryBackoffMaxMs); - } - - public Retry(Time time, long retryBackoffMs, long retryBackoffMaxMs) { - this.time = time; - this.retryBackoffMs = retryBackoffMs; - this.retryBackoffMaxMs = retryBackoffMaxMs; - - if (this.retryBackoffMs < 0) - throw new IllegalArgumentException(String.format("retryBackoffMs value (%d) must be non-negative", retryBackoffMs)); - - if (this.retryBackoffMaxMs < 0) - throw new IllegalArgumentException(String.format("retryBackoffMaxMs value (%d) must be non-negative", retryBackoffMaxMs)); - - if (this.retryBackoffMaxMs < this.retryBackoffMs) - throw new IllegalArgumentException(String.format("retryBackoffMaxMs value (%d) is less than retryBackoffMs value (%d)", retryBackoffMaxMs, retryBackoffMs)); - } - - public R execute(Retryable retryable) throws ExecutionException { - long endMs = time.milliseconds() + retryBackoffMaxMs; - int currAttempt = 0; - ExecutionException error = null; - - while (time.milliseconds() <= endMs) { - currAttempt++; - - try { - return retryable.call(); - } catch (UnretryableException e) { - // We've deemed this error to not be worth retrying, so collect the error and - // fail immediately. - if (error == null) - error = new ExecutionException(e); - - break; - } catch (ExecutionException e) { - log.warn("Error during retry attempt {}", currAttempt, e); - - if (error == null) - error = e; - - long waitMs = retryBackoffMs * (long) Math.pow(2, currAttempt - 1); - long diff = endMs - time.milliseconds(); - waitMs = Math.min(waitMs, diff); - - if (waitMs <= 0) - break; - - String message = String.format("Attempt %d to make call resulted in an error; sleeping %d ms before retrying", - currAttempt, waitMs); - log.warn(message, e); - - time.sleep(waitMs); - } - } - - if (error == null) - // Really shouldn't ever get to here, but... - error = new ExecutionException(new IllegalStateException("Exhausted all retry attempts but no attempt returned value or encountered exception")); - - throw error; - } - -} 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..2cc8b8ae69940 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.JwtValidatorException; + /** * 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 JwtValidatorException("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 JwtValidatorException("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 JwtValidatorException { section = section.trim(); if (section.isEmpty()) - throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)"); + throw new JwtValidatorException("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/SslResource.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SslResource.java new file mode 100644 index 0000000000000..4de532bf5937b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SslResource.java @@ -0,0 +1,75 @@ +/* + * 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.ConfigException; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.network.ConnectionMode; +import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory; +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.util.Map; + +import javax.net.ssl.SSLContext; + +/** + * {@code SslResource} couples the {@link SslFactory} and {@link SSLContext} so that + * {@link #sslFactory} can be properly {@link SslFactory#close() closed} during closing of the overall + * OAuth login/validation module. The {@link SSLContext} API is what the HTTP clients use, so the two + * need to be kept closely together. + */ +public class SslResource implements Closeable { + + private final SslFactory sslFactory; + + private final SSLContext sslContext; + + public SslResource(SslFactory sslFactory, SSLContext sslContext) { + this.sslFactory = sslFactory; + this.sslContext = sslContext; + } + + public static SslResource create(Map configs) { + SslFactory sslFactory = new SslFactory(ConnectionMode.CLIENT); + sslFactory.configure(configs); + + if (!((sslFactory.sslEngineFactory()) instanceof DefaultSslEngineFactory)) { + String message = String.format( + "The OAuth %s configuration includes a custom SSL factory class (%s) which is not a supported JAAS option for OAuth", + SaslConfigs.SASL_JAAS_CONFIG, + SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG + ); + throw new ConfigException(message); + } + + SSLContext sslContext = ((DefaultSslEngineFactory) sslFactory.sslEngineFactory()).sslContext(); + return new SslResource(sslFactory, sslContext); + } + + public SSLContext sslContext() { + return sslContext; + } + + @Override + public void close() throws IOException { + Utils.closeQuietly(sslFactory, "sslFactory"); + } +} 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/StaticAssertionJwtTemplate.java similarity index 54% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retryable.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/StaticAssertionJwtTemplate.java index 46752f3ea6020..9dede2ad32c3f 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/StaticAssertionJwtTemplate.java @@ -14,32 +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 java.util.concurrent.ExecutionException; +import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate; -/** - * Simple interface to abstract out the call that is made so that it can be retried. - * - * @param Result type - * - * @see Retry - * @see UnretryableException - */ +import java.util.Collections; +import java.util.Map; + +public class StaticAssertionJwtTemplate implements AssertionJwtTemplate { -public interface Retryable { + private final Map payload; - /** - * 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 - */ + public StaticAssertionJwtTemplate(Map payload) { + this.payload = Collections.unmodifiableMap(payload); + } - R call() throws ExecutionException, UnretryableException; + @Override + public Map header() { + return Collections.emptyMap(); + } + @Override + public Map payload() { + return payload; + } } 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/ValidatorAccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidatorTest.java similarity index 58% 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/BrokerJwtValidatorTest.java index 4db20e9ee10d6..7f2ce543e9922 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/BrokerJwtValidatorTest.java @@ -14,30 +14,44 @@ * 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.CloseableVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder; 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 BrokerJwtValidatorTest extends JwtValidatorTest { @Override - protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) { - return new ValidatorAccessTokenValidator(30, - Collections.emptySet(), - null, - (jws, nestingContext) -> builder.jwk().getKey(), - builder.scopeClaimName(), - builder.subjectClaimName()); + protected JwtValidator createValidator(JwtBuilder 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 BrokerJwtValidator() { + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + super.configure(keyResolver, configs, saslMechanism); + } + }; } @Test @@ -66,13 +80,15 @@ public void testMissingSubShouldBeValid() throws Exception { String subClaimName = "client_id"; String subject = "otherSub"; PublicJsonWebKey jwk = createRsaJwk(); - AccessTokenBuilder tokenBuilder = new AccessTokenBuilder() + JwtBuilder tokenBuilder = new JwtBuilder() .jwk(jwk) .alg(AlgorithmIdentifiers.RSA_USING_SHA256) .addCustomClaim(subClaimName, subject) .subjectClaimName(subClaimName) .subject(null); - AccessTokenValidator validator = createAccessTokenValidator(tokenBuilder); + JwtValidator validator = createValidator(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()); @@ -81,10 +97,11 @@ 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); - String accessToken = builder.build(); - OAuthBearerToken token = validator.validate(accessToken); + JwtBuilder builder = new JwtBuilder().jwk(jwk).alg(alg); + JwtValidator validator = createValidator(builder); + validator.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of()); + String jwt = builder.build(); + OAuthBearerToken token = validator.validate(jwt); assertEquals(builder.subject(), token.principalName()); assertEquals(builder.issuedAtSeconds() * 1000, token.startTimeMs()); diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.java similarity index 70% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.java index fc2e3d2a2e83a..72c925fe2f9c0 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.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; -package org.apache.kafka.common.security.oauthbearer.internals.secured; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder; -public class LoginAccessTokenValidatorTest extends AccessTokenValidatorTest { +public class ClientJwtValidatorTest extends JwtValidatorTest { @Override - protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) { - return new LoginAccessTokenValidator(builder.scopeClaimName(), builder.subjectClaimName()); + protected JwtValidator createValidator(JwtBuilder builder) { + return new ClientJwtValidator(); } - } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java new file mode 100644 index 0000000000000..4d971af44a20e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java @@ -0,0 +1,91 @@ +/* + * 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.ConfigException; +import org.apache.kafka.test.TestUtils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +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_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 DefaultJwtRetrieverTest extends OAuthBearerTest { + + @AfterEach + public void tearDown() throws Exception { + System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + } + + @Test + public void testConfigureRefreshingFileJwtRetriever() throws Exception { + String expected = "{}"; + + File jwtFile = createTempFile("jwt-", ".json", expected); + + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, jwtFile.toURI().toString()); + Map configs = Collections.singletonMap(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString()); + + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()); + assertEquals(expected, jwtRetriever.retrieve()); + } + } + + @Test + public void testConfigureRefreshingFileJwtRetrieverWithInvalidDirectory() { + // 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_TOKEN_ENDPOINT_URL, file); + + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + assertThrowsWithMessage(ConfigException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), "that doesn't exist"); + } + } + + @Test + public void testConfigureRefreshingFileJwtRetrieverWithInvalidFile() throws Exception { + // Should fail because while the parent path exists, the file itself doesn't. + File jwtFile = new File(TestUtils.tempDirectory(), "this-file-does-not-exist.json"); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, jwtFile.toURI().toString()); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString()); + + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + assertThrowsWithMessage(ConfigException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), "that doesn't exist"); + } + } + + @Test + public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception { + // Should fail if the URL is not allowed + File jwtFile = new File("not_allowed.json"); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString()); + + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) { + assertThrowsWithMessage(ConfigException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetrieverTest.java new file mode 100644 index 0000000000000..3df6155f6f12e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetrieverTest.java @@ -0,0 +1,65 @@ +/* + * 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.junit.jupiter.api.Test; + +import java.io.File; +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; + +public class FileJwtRetrieverTest extends OAuthBearerTest { + + @Test + public void testFileTokenRetrieverHandlesNewline() throws Exception { + 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 jwtFile = createTempFile("jwt-", ".json", withNewline); + + List jaasConfigEntries = new ArrayList<>(); + jaasConfigEntries.add(new AppConfigurationEntry("dummy", OPTIONAL, Collections.emptyMap())); + + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, jwtFile.toURI().toString()); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.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); + } + } +} 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/JwtValidatorTest.java similarity index 50% 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/JwtValidatorTest.java index 0adaf34bbbeea..0d67a4e81d73d 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/JwtValidatorTest.java @@ -14,8 +14,9 @@ * 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.JwtBuilder; import org.jose4j.jws.AlgorithmIdentifiers; import org.jose4j.jwx.HeaderParameterNames; @@ -23,70 +24,74 @@ 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) -public abstract class AccessTokenValidatorTest extends OAuthBearerTest { +public abstract class JwtValidatorTest extends OAuthBearerTest { - protected abstract AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder accessTokenBuilder) throws Exception; + protected abstract JwtValidator createValidator(JwtBuilder jwtBuilder) throws Exception; - protected AccessTokenValidator createAccessTokenValidator() throws Exception { - AccessTokenBuilder builder = new AccessTokenBuilder(); - return createAccessTokenValidator(builder); + protected JwtValidator createValidator() throws Exception { + JwtBuilder builder = new JwtBuilder(); + JwtValidator validator = createValidator(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)"); + JwtValidator validator = createValidator(); + assertThrowsWithMessage(JwtValidatorException.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)"); + JwtValidator validator = createValidator(); + assertThrowsWithMessage(JwtValidatorException.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)"); + JwtValidator validator = createValidator(); + assertThrowsWithMessage(JwtValidatorException.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)"); + JwtValidator validator = createValidator(); + assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(".."), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test public void testMissingHeader() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createValidator(); String header = ""; String payload = createBase64JsonJwtSection(node -> { }); String signature = ""; - String accessToken = String.format("%s.%s.%s", header, payload, signature); - assertThrows(ValidateException.class, () -> validator.validate(accessToken)); + String jwt = String.format("%s.%s.%s", header, payload, signature); + assertThrows(JwtValidatorException.class, () -> validator.validate(jwt)); } @Test public void testMissingPayload() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createValidator(); String header = createBase64JsonJwtSection(node -> node.put(HeaderParameterNames.ALGORITHM, AlgorithmIdentifiers.NONE)); String payload = ""; String signature = ""; - String accessToken = String.format("%s.%s.%s", header, payload, signature); - assertThrows(ValidateException.class, () -> validator.validate(accessToken)); + String jwt = String.format("%s.%s.%s", header, payload, signature); + assertThrows(JwtValidatorException.class, () -> validator.validate(jwt)); } @Test public void testMissingSignature() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createValidator(); String header = createBase64JsonJwtSection(node -> node.put(HeaderParameterNames.ALGORITHM, AlgorithmIdentifiers.NONE)); String payload = createBase64JsonJwtSection(node -> { }); String signature = ""; - String accessToken = String.format("%s.%s.%s", header, payload, signature); - assertThrows(ValidateException.class, () -> validator.validate(accessToken)); + String jwt = String.format("%s.%s.%s", header, payload, signature); + assertThrows(JwtValidatorException.class, () -> validator.validate(jwt)); } - } \ No newline at end of file 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..4d1848ca2b2da 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,11 @@ 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.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder; import org.jose4j.jws.AlgorithmIdentifiers; import org.junit.jupiter.api.AfterEach; @@ -35,29 +29,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,27 +61,23 @@ public void tearDown() throws Exception { @Test public void testHandleTokenCallback() throws Exception { - Map configs = getSaslConfigs(); - AccessTokenBuilder builder = new AccessTokenBuilder() + JwtBuilder builder = new JwtBuilder() .jwk(createRsaJwk()) .alg(AlgorithmIdentifiers.RSA_USING_SHA256); - String accessToken = builder.build(); - AccessTokenRetriever accessTokenRetriever = () -> accessToken; - - OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs); + String jwt = builder.build(); + JwtRetriever jwtRetriever = mock(JwtRetriever.class); + when(jwtRetriever.retrieve()).thenReturn(jwt); - try { + try (OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever)) { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); - handler.handle(new Callback[] {callback}); + handler.handle(new Callback[]{callback}); assertNotNull(callback.token()); OAuthBearerToken token = callback.token(); - assertEquals(accessToken, token.value()); + assertEquals(jwt, token.value()); assertEquals(builder.subject(), token.principalName()); assertEquals(builder.expirationSeconds() * 1000, token.lifetimeMs()); assertEquals(builder.issuedAtSeconds() * 1000, token.startTimeMs()); - } finally { - handler.close(); } } @@ -95,8 +87,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 +117,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,92 +133,90 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() { } @Test - public void testInvalidCallbackGeneratesUnsupportedCallbackException() { - Map configs = getSaslConfigs(); - OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - AccessTokenRetriever accessTokenRetriever = () -> "foo"; - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs); - handler.init(accessTokenRetriever, accessTokenValidator); + public void testConfigureThrowsExceptionOnJwtValidatorConfigure() { + try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + JwtRetriever jwtRetriever = mock(JwtRetriever.class); + JwtValidator jwtValidator = mock(JwtValidator.class)) { - try { - Callback unsupportedCallback = new Callback() { }; - assertThrows(UnsupportedCallbackException.class, () -> handler.handle(new Callback[]{unsupportedCallback})); - } finally { - handler.close(); + doThrow(new KafkaException("Forced failure")).when(jwtValidator).configure(any(), any(), any()); + + assertThrows( + KafkaException.class, + () -> handler.configure(jwtRetriever, jwtValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of()) + ); } } @Test - public void testInvalidAccessToken() throws Exception { - testInvalidAccessToken("this isn't valid", "Malformed JWT provided"); - testInvalidAccessToken("this.isn't.valid", "malformed Base64 URL encoded value"); - testInvalidAccessToken(createAccessKey("this", "isn't", "valid"), "malformed JSON"); - testInvalidAccessToken(createAccessKey("{}", "{}", "{}"), "exp value must be non-null"); + public void testConfigureThrowsExceptionOnJwtValidatorClose() { + try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + JwtRetriever jwtRetriever = mock(JwtRetriever.class)) { + JwtValidator jwtValidator = mock(JwtValidator.class); + doThrow(new KafkaException("Forced failure")).when(jwtValidator).close(); + handler.configure(jwtRetriever, jwtValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of()); + assertDoesNotThrow(handler::close); + } } @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 testInvalidCallbackGeneratesUnsupportedCallbackException() throws IOException { + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + JwtRetriever jwtRetriever = mock(JwtRetriever.class); + when(jwtRetriever.retrieve()).thenReturn("foo"); + JwtValidator jwtValidator = new ClientJwtValidator(); + handler.configure(jwtRetriever, jwtValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of()); try { - OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); - assertThrowsWithMessage(IOException.class, - () -> handler.handle(new Callback[]{callback}), - "token endpoint response access_token value must be non-null"); + Callback unsupportedCallback = new Callback() { }; + assertThrows(UnsupportedCallbackException.class, () -> handler.handle(new Callback[]{unsupportedCallback})); } 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"; + public void testInvalidJwt() throws Exception { + testInvalidJwt("this isn't valid", "Malformed JWT provided"); + testInvalidJwt("this.isn't.valid", "malformed Base64 URL encoded value"); + testInvalidJwt(createAccessKey("this", "isn't", "valid"), "malformed JSON"); + testInvalidJwt(createAccessKey("{}", "{}", "{}"), "exp value must be non-null"); + } - File tmpDir = createTempDir("access-token"); - File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", withNewline); + @Test + public void testMissingJwt() throws IOException { + JwtRetriever jwtRetriever = mock(JwtRetriever.class); + when(jwtRetriever.retrieve()).thenThrow(new JwtRetrieverException("The token endpoint response id_token value must be non-null")); - 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(); + try (OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever)) { + OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); + assertThrowsWithMessage( + JwtRetrieverException.class, + () -> handler.handle(new Callback[]{callback}), + "token endpoint response id_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 - public void testConfigureWithAccessTokenFile() throws Exception { + public void testConfigureWithJwtFile() throws Exception { String expected = "{}"; - File tmpDir = createTempDir("access-token"); - File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", expected); - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.toURI().toString()); + File jwtFile = createTempFile("jwt-", ".json", expected); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, jwtFile.toURI().toString()); 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 configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString()); + Map jaasConfig = Collections.emptyMap(); + configureHandler(handler, configs, jaasConfig); + assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever); + assertInstanceOf(FileJwtRetriever.class, ((DefaultJwtRetriever) handler.jwtRetriever).delegate()); } @Test @@ -234,18 +224,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(DefaultJwtRetriever.class, handler.jwtRetriever); + assertInstanceOf(ClientCredentialsJwtRetriever.class, ((DefaultJwtRetriever) handler.jwtRetriever).delegate()); } - private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception { - Map configs = getSaslConfigs(); - OAuthBearerLoginCallbackHandler handler = createHandler(() -> accessToken, configs); + private void testInvalidJwt(String jwt, String expectedMessageSubstring) throws Exception { + JwtRetriever jwtRetriever = mock(JwtRetriever.class); + when(jwtRetriever.retrieve()).thenReturn(jwt); - try { + try (OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever)) { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); handler.handle(new Callback[]{callback}); @@ -255,24 +246,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(JwtRetriever jwtRetriever) { + Map configs = getSaslConfigs(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs); - handler.init(accessTokenRetriever, accessTokenValidator); + JwtValidator jwtValidator = new ClientJwtValidator(); + handler.configure(jwtRetriever, jwtValidator, configs, OAUTHBEARER_MECHANISM, List.of()); return handler; } - } 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/OAuthBearerTest.java similarity index 68% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerTest.java index 7f20b9464faea..1b101ba0a9b80 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/OAuthBearerTest.java @@ -14,15 +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; +package org.apache.kafka.common.security.oauthbearer; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; 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 org.apache.kafka.test.TestUtils; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -37,34 +38,28 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.File; -import java.io.FileWriter; import java.io.IOException; -import java.net.HttpURLConnection; -import java.net.URL; -import java.util.Arrays; +import java.nio.file.Files; import java.util.Base64; import java.util.Collections; -import java.util.Iterator; import java.util.Map; -import java.util.concurrent.ExecutionException; import java.util.function.Consumer; 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; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; @TestInstance(Lifecycle.PER_CLASS) 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 +83,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); @@ -119,61 +122,10 @@ protected String createJsonJwtSection(Consumer c) { } } - protected Retryable createRetryable(Exception[] attempts) { - Iterator i = Arrays.asList(attempts).iterator(); - - return () -> { - Exception e = i.hasNext() ? i.next() : null; - - if (e == null) { - return "success!"; - } else { - if (e instanceof IOException) - throw new ExecutionException(e); - else if (e instanceof RuntimeException) - throw (RuntimeException) e; - else - throw new RuntimeException(e); - } - }; - } - - protected HttpURLConnection createHttpURLConnection(String response) throws IOException { - HttpURLConnection mockedCon = mock(HttpURLConnection.class); - when(mockedCon.getURL()).thenReturn(new URL("https://www.example.com")); - when(mockedCon.getResponseCode()).thenReturn(200); - when(mockedCon.getOutputStream()).thenReturn(new ByteArrayOutputStream()); - when(mockedCon.getInputStream()).thenReturn(new ByteArrayInputStream(Utils.utf8(response))); - return mockedCon; - } - - protected File createTempDir(String directory) throws IOException { - File tmpDir = new File(System.getProperty("java.io.tmpdir")); - - if (directory != null) - tmpDir = new File(tmpDir, directory); - - if (!tmpDir.exists() && !tmpDir.mkdirs()) - throw new IOException("Could not create " + tmpDir); - - tmpDir.deleteOnExit(); - log.debug("Created temp directory {}", tmpDir); - return tmpDir; - } - - protected File createTempFile(File tmpDir, - String prefix, - String suffix, - String contents) - throws IOException { - File file = File.createTempFile(prefix, suffix, tmpDir); + protected File createTempFile(String prefix, String suffix, String contents) throws IOException { + File file = TestUtils.tempFile(prefix, suffix); + Files.writeString(file.toPath(), contents); log.debug("Created new temp file {}", file); - file.deleteOnExit(); - - try (FileWriter writer = new FileWriter(file)) { - writer.write(contents); - } - return file; } @@ -211,5 +163,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/OAuthBearerValidatorCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java index d682a05ec11cc..1d074c9f2760e 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 @@ -17,24 +17,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.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder; 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; @@ -46,74 +41,56 @@ public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest { public void testBasic() throws Exception { String expectedAudience = "a"; List allAudiences = Arrays.asList(expectedAudience, "b", "c"); - AccessTokenBuilder builder = new AccessTokenBuilder() + JwtBuilder builder = new JwtBuilder() .audience(expectedAudience) .jwk(createRsaJwk()) .alg(AlgorithmIdentifiers.RSA_USING_SHA256); - String accessToken = builder.build(); + String jwt = builder.build(); Map configs = getSaslConfigs(SASL_OAUTHBEARER_EXPECTED_AUDIENCE, allAudiences); - OAuthBearerValidatorCallbackHandler handler = createHandler(configs, builder); - try { - OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); + try (OAuthBearerValidatorCallbackHandler handler = createHandler(configs, builder)) { + OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(jwt); handler.handle(new Callback[]{callback}); assertNotNull(callback.token()); OAuthBearerToken token = callback.token(); - assertEquals(accessToken, token.value()); + assertEquals(jwt, token.value()); assertEquals(builder.subject(), token.principalName()); assertEquals(builder.expirationSeconds() * 1000, token.lifetimeMs()); assertEquals(builder.issuedAtSeconds() * 1000, token.startTimeMs()); - } finally { - handler.close(); } } @Test - public void testInvalidAccessToken() throws Exception { + public void testInvalidJwt() throws Exception { // There aren't different error messages for the validation step, so these are all the // same :( String substring = "invalid_token"; - assertInvalidAccessTokenFails("this isn't valid", substring); - assertInvalidAccessTokenFails("this.isn't.valid", substring); - assertInvalidAccessTokenFails(createAccessKey("this", "isn't", "valid"), substring); - assertInvalidAccessTokenFails(createAccessKey("{}", "{}", "{}"), substring); + assertInvalidJwtFails("this isn't valid", substring); + assertInvalidJwtFails("this.isn't.valid", substring); + assertInvalidJwtFails(createAccessKey("this", "isn't", "valid"), substring); + assertInvalidJwtFails(createAccessKey("{}", "{}", "{}"), substring); } - private void assertInvalidAccessTokenFails(String accessToken, String expectedMessageSubstring) throws Exception { + private void assertInvalidJwtFails(String jwt, String expectedMessageSubstring) throws Exception { Map configs = getSaslConfigs(); - OAuthBearerValidatorCallbackHandler handler = createHandler(configs, new AccessTokenBuilder()); - try { - OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); - handler.handle(new Callback[] {callback}); + try (OAuthBearerValidatorCallbackHandler handler = createHandler(configs, new JwtBuilder())) { + OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(jwt); + 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, JwtBuilder builder) throws Exception { OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); - CloseableVerificationKeyResolver verificationKeyResolver = (jws, nestingContext) -> - builder.jwk().getPublicKey(); - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(options, verificationKeyResolver); - handler.init(verificationKeyResolver, accessTokenValidator); + DefaultJwtValidator validator = new DefaultJwtValidator(); + handler.configure(validator, 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/AccessTokenRetrieverFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java deleted file mode 100644 index 3e85f7b0ce4fa..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java +++ /dev/null @@ -1,111 +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.ConfigException; - -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.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.junit.jupiter.api.Assertions.assertEquals; - -public class AccessTokenRetrieverFactoryTest extends OAuthBearerTest { - - @AfterEach - public void tearDown() throws Exception { - System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); - } - - @Test - public void testConfigureRefreshingFileAccessTokenRetriever() throws Exception { - String expected = "{}"; - - File tmpDir = createTempDir("access-token"); - File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", expected); - - 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(); - assertEquals(expected, accessTokenRetriever.retrieve()); - } - } - - @Test - public void testConfigureRefreshingFileAccessTokenRetrieverWithInvalidDirectory() { - // 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_TOKEN_ENDPOINT_URL, file); - Map jaasConfig = Collections.emptyMap(); - assertThrowsWithMessage(ConfigException.class, () -> AccessTokenRetrieverFactory.create(configs, jaasConfig), "that doesn't exist"); - } - - @Test - public void testConfigureRefreshingFileAccessTokenRetrieverWithInvalidFile() throws Exception { - // Should fail because while the parent path exists, the file itself doesn't. - File tmpDir = createTempDir("this-directory-does-exist"); - 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"); - } - - @Test - public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception { - // Should fail if the URL is not allowed - 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) - ); - } - -} 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/ClaimValidationUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java deleted file mode 100644 index 89387797cdc30..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtilsTest.java +++ /dev/null @@ -1,165 +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.junit.jupiter.api.Test; - -import java.util.Arrays; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; - -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; - -public class ClaimValidationUtilsTest extends OAuthBearerTest { - - @Test - public void testValidateScopes() { - Set scopes = ClaimValidationUtils.validateScopes("scope", Arrays.asList(" a ", " b ")); - - assertEquals(2, scopes.size()); - assertTrue(scopes.contains("a")); - assertTrue(scopes.contains("b")); - } - - @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 "))); - } - - @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", " "))); - } - - @Test - public void testValidateScopesResultIsImmutable() { - SortedSet callerSet = new TreeSet<>(Arrays.asList("a", "b", "c")); - Set scopes = ClaimValidationUtils.validateScopes("scope", callerSet); - - assertEquals(3, scopes.size()); - - callerSet.add("d"); - assertEquals(4, callerSet.size()); - assertTrue(callerSet.contains("d")); - assertEquals(3, scopes.size()); - assertFalse(scopes.contains("d")); - - callerSet.remove("c"); - assertEquals(3, callerSet.size()); - assertFalse(callerSet.contains("c")); - assertEquals(3, scopes.size()); - assertTrue(scopes.contains("c")); - - callerSet.clear(); - assertEquals(0, callerSet.size()); - assertEquals(3, scopes.size()); - } - - @Test - public void testValidateScopesResultThrowsExceptionOnMutation() { - SortedSet callerSet = new TreeSet<>(Arrays.asList("a", "b", "c")); - Set scopes = ClaimValidationUtils.validateScopes("scope", callerSet); - assertThrows(UnsupportedOperationException.class, scopes::clear); - } - - @Test - public void testValidateExpiration() { - Long expected = 1L; - Long actual = ClaimValidationUtils.validateExpiration("exp", expected); - assertEquals(expected, actual); - } - - @Test - public void testValidateExpirationAllowsZero() { - Long expected = 0L; - Long actual = ClaimValidationUtils.validateExpiration("exp", expected); - assertEquals(expected, actual); - } - - @Test - public void testValidateExpirationDisallowsNull() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateExpiration("exp", null)); - } - - @Test - public void testValidateExpirationDisallowsNegatives() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateExpiration("exp", -1L)); - } - - @Test - public void testValidateSubject() { - String expected = "jdoe"; - String actual = ClaimValidationUtils.validateSubject("sub", expected); - assertEquals(expected, actual); - } - - @Test - public void testValidateSubjectDisallowsEmptyNullAndWhitespace() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", "")); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", null)); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", " ")); - } - - @Test - public void testValidateClaimNameOverride() { - String expected = "email"; - String actual = ClaimValidationUtils.validateClaimNameOverride("sub", String.format(" %s ", expected)); - assertEquals(expected, actual); - } - - @Test - public void testValidateClaimNameOverrideDisallowsEmptyNullAndWhitespace() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", "")); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", null)); - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateSubject("sub", " ")); - } - - @Test - public void testValidateIssuedAt() { - Long expected = 1L; - Long actual = ClaimValidationUtils.validateIssuedAt("iat", expected); - assertEquals(expected, actual); - } - - @Test - public void testValidateIssuedAtAllowsZero() { - Long expected = 0L; - Long actual = ClaimValidationUtils.validateIssuedAt("iat", expected); - assertEquals(expected, actual); - } - - @Test - public void testValidateIssuedAtAllowsNull() { - Long expected = null; - Long actual = ClaimValidationUtils.validateIssuedAt("iat", expected); - assertEquals(expected, actual); - } - - @Test - public void testValidateIssuedAtDisallowsNegatives() { - assertThrows(ValidateException.class, () -> ClaimValidationUtils.validateIssuedAt("iat", -1L)); - } - -} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGeneratorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGeneratorTest.java new file mode 100644 index 0000000000000..2c602d63d0276 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGeneratorTest.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.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.net.URL; +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.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.urlencodeHeader; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +public class ClientCredentialsRequestGeneratorTest extends HttpRequestGeneratorTest { + + @Test + public void testFormatAuthorizationHeader() throws Exception { + ClientCredentialsRequestGenerator requestGenerator = new Builder() + .setClientId("id") + .setClientSecret("secret") + .build(); + assertAuthorizationHeaderEquals(requestGenerator, "Basic aWQ6c2VjcmV0"); + } + + @Test + public void testFormatAuthorizationHeaderEncoding() throws Exception { + ClientCredentialsRequestGenerator requestGenerator = 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(requestGenerator, "Basic U09NRV9SQU5ET01fTE9OR19VU0VSXzAxMjM0OjlRfDBgOGl+dXRlLW45a3NqTFdiXDUwIkFYQFVVRUQ1RQ=="); + + requestGenerator = 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(requestGenerator, "Basic dXNlciUyMSU0MCU3RSUyNzpzZWNyZXQtJTI4KiUyOSUyMQ=="); + } + + @Test + public void testFormatRequestBody() throws Exception { + ClientCredentialsRequestGenerator requestGenerator = new Builder() + .setScope("test") + .build(); + assertBodyEquals(requestGenerator, "grant_type=client_credentials&scope=test"); + } + + @Test + public void testFormatRequestBodyWithEscaped() throws Exception { + 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() throws Exception { + 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) { + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, OAUTHBEARER_MECHANISM); + boolean actualValue = urlencodeHeader(oauthConfig); + 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(ClientCredentialsRequestGenerator requestGenerator, String expected) { + String actual = requestGenerator.generateHeaders().get("Authorization"); + assertNotNull(actual); + assertEquals(expected, actual); + } + + 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 ClientCredentialsRequestGenerator build() throws Exception { + return new ClientCredentialsRequestGenerator( + new URL("http://www.example.com"), + 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 deleted file mode 100644 index 9a62f480215f7..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtilsTest.java +++ /dev/null @@ -1,178 +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.ConfigException; -import org.apache.kafka.test.TestUtils; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; - -public class ConfigurationUtilsTest extends OAuthBearerTest { - - private static final String URL_CONFIG_NAME = "url"; - private static final String FILE_CONFIG_NAME = "file"; - - @AfterEach - public void tearDown() throws Exception { - System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); - } - - @Test - public void testUrl() { - testUrl("http://www.example.com"); - } - - @Test - public void testUrlWithSuperfluousWhitespace() { - testUrl(String.format(" %s ", "http://www.example.com")); - } - - @Test - public void testUrlCaseInsensitivity() { - testUrl("HTTPS://WWW.EXAMPLE.COM"); - } - - @Test - public void testUrlFile() { - testUrl("file:///tmp/foo.txt"); - } - - @Test - public void testUrlFullPath() { - testUrl("https://myidp.example.com/oauth2/default/v1/token"); - } - - @Test - public void testUrlMissingProtocol() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl("www.example.com"), "no protocol"); - } - - @Test - public void testUrlInvalidProtocol() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl("ftp://ftp.example.com"), "invalid protocol"); - } - - @Test - public void testUrlNull() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl(null), "must be non-null"); - } - - @Test - public void testUrlEmptyString() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl(""), "must not contain only whitespace"); - } - - @Test - public void testUrlWhitespace() { - assertThrowsWithMessage(ConfigException.class, () -> testUrl(" "), "must not contain only whitespace"); - } - - 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); - cu.validateUrl(URL_CONFIG_NAME); - } - - @Test - public void testFile() throws IOException { - File file = TestUtils.tempFile("some contents!"); - testFile(file.toURI().toURL().toString()); - } - - @Test - public void testFileWithSuperfluousWhitespace() throws IOException { - File file = TestUtils.tempFile(); - testFile(String.format(" %s ", file.toURI().toURL())); - } - - @Test - public void testFileDoesNotExist() { - assertThrowsWithMessage(ConfigException.class, () -> testFile(new File("/tmp/not/a/real/file.txt").toURI().toURL().toString()), "that doesn't exist"); - } - - @Test - public void testFileUnreadable() throws IOException { - File file = TestUtils.tempFile(); - - if (!file.setReadable(false)) - throw new IllegalStateException(String.format("Can't test file permissions as test couldn't programmatically make temp file %s un-readable", file.getAbsolutePath())); - - assertThrowsWithMessage(ConfigException.class, () -> testFile(file.toURI().toURL().toString()), "that doesn't have read permission"); - } - - @Test - public void testFileNull() { - assertThrowsWithMessage(ConfigException.class, () -> testFile(null), "must be non-null"); - } - - @Test - public void testFileEmptyString() { - assertThrowsWithMessage(ConfigException.class, () -> testFile(""), "must not contain only whitespace"); - } - - @Test - public void testFileWhitespace() { - assertThrowsWithMessage(ConfigException.class, () -> testFile(" "), "must not contain only whitespace"); - } - - @Test - public void testThrowIfURLIsNotAllowed() { - String url = "http://www.example.com"; - String fileUrl = "file:///etc/passwd"; - Map configs = new HashMap<>(); - configs.put(URL_CONFIG_NAME, url); - configs.put(FILE_CONFIG_NAME, fileUrl); - ConfigurationUtils cu = new ConfigurationUtils(configs); - - // By default, no URL is allowed - assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(url), - ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); - assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(fileUrl), - ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); - - // add one url into allowed list - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, url); - assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(url)); - assertThrowsWithMessage(ConfigException.class, () -> cu.throwIfURLIsNotAllowed(fileUrl), - ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); - - // add all urls into allowed list - System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, url + "," + fileUrl); - assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(url)); - assertDoesNotThrow(() -> cu.throwIfURLIsNotAllowed(fileUrl)); - } - - 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); - cu.validateFile(URL_CONFIG_NAME); - } - -} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreatorTest.java new file mode 100644 index 0000000000000..6febe7b78505b --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreatorTest.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.security.oauthbearer.AssertionCreator; +import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; + +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.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.security.GeneralSecurityException; +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.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultAssertionCreator.TOKEN_SIGNING_ALGORITHM_RS256; +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.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class DefaultAssertionCreatorTest { + + @Test + public void testPrivateKeyId() throws Exception { + TestAssertionJwtTemplate jwtTemplate = new TestAssertionJwtTemplate(); + jwtTemplate.setHeaders(Map.of("kid", "test-id")); + + KeyPair keyPair = generateKeyPair(); + Builder builder = new Builder() + .setPrivateKeyFile(generatePrivateKeySecret(keyPair.getPrivate())); + AssertionCreator assertionCreator = builder.build(); + String assertion = assertionCreator.create(jwtTemplate); + 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 { + TestAssertionJwtTemplate jwtTemplate = new TestAssertionJwtTemplate(); + KeyPair keyPair = generateKeyPair(); + Builder builder = new Builder() + .setPrivateKeyFile(generatePrivateKeySecret(keyPair.getPrivate())); + AssertionCreator assertionCreator = builder.build(); + String assertion = assertionCreator.create(jwtTemplate); + assertClaims(builder, keyPair.getPublic(), assertion); + } + + @Test + public void testInvalidPrivateKeySecret() throws Exception { + File privateKeyFile = generatePrivateKeySecret(); + long originalFileLength = privateKeyFile.length(); + int bytesToTruncate = 10; // A single byte isn't enough + + // Intentionally "mangle" the private key secret by truncating the file. + try (FileChannel channel = FileChannel.open(privateKeyFile.toPath(), StandardOpenOption.WRITE)) { + long size = channel.size(); + assertEquals(originalFileLength, size); + assertTrue(size > bytesToTruncate); + channel.truncate(size - bytesToTruncate); + } + + assertEquals(originalFileLength - bytesToTruncate, privateKeyFile.length()); + + TestAssertionJwtTemplate jwtTemplate = new TestAssertionJwtTemplate(); + AssertionCreator assertionCreator = new Builder() + .setPrivateKeyFile(privateKeyFile) + .build(); + KafkaException e = assertThrows(KafkaException.class, () -> assertionCreator.create(jwtTemplate)); + assertNotNull(e.getCause()); + assertInstanceOf(GeneralSecurityException.class, e.getCause()); + } + + @ParameterizedTest + @CsvSource("RS256,ES256") + public void testAlgorithm(String algorithm) throws Exception { + KeyPair keyPair = generateKeyPair(); + Builder builder = new Builder() + .setPrivateKeyFile(generatePrivateKeySecret(keyPair.getPrivate())) + .setAlgorithm(algorithm); + AssertionCreator assertionCreator = builder.build(); + + TestAssertionJwtTemplate jwtTemplate = new TestAssertionJwtTemplate(); + String assertion = assertionCreator.create(jwtTemplate); + + 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(algorithm, jsonWebStructure.getAlgorithmHeaderValue()); + } + + @Test + public void testInvalidAlgorithm() throws IOException { + PrivateKey privateKey = generateKeyPair().getPrivate(); + Builder builder = new Builder() + .setPrivateKeyFile(generatePrivateKeySecret(privateKey)) + .setAlgorithm("thisisnotvalid"); + DefaultAssertionCreator assertionCreator = builder.build(); + assertThrows(NoSuchAlgorithmException.class, assertionCreator::getSignature); + assertThrows( + NoSuchAlgorithmException.class, + () -> assertionCreator.sign(privateKey, "dummy content")); + } + + 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 + .build(); + } + + private File generatePrivateKeySecret(PrivateKey privateKey) throws IOException { + File file = File.createTempFile("private-", ".key"); + byte[] bytes = Base64.getEncoder().encode(privateKey.getEncoded()); + + try (FileChannel channel = FileChannel.open(file.toPath(), EnumSet.of(StandardOpenOption.WRITE))) { + Utils.writeFully(channel, ByteBuffer.wrap(bytes)); + } + + return file; + } + + private File generatePrivateKeySecret() throws IOException { + 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 algorithm = TOKEN_SIGNING_ALGORITHM_RS256; + private File privateKeyFile; + + public Builder setAlgorithm(String algorithm) { + this.algorithm = algorithm; + return this; + } + + public Builder setPrivateKeyFile(File privateKeyFile) { + this.privateKeyFile = privateKeyFile; + return this; + } + + private DefaultAssertionCreator build() { + return new DefaultAssertionCreator(time, algorithm, privateKeyFile); + } + } + + private static class TestAssertionJwtTemplate implements AssertionJwtTemplate { + + private Map headers = new HashMap<>(); + private Map payload = new HashMap<>(); + + public TestAssertionJwtTemplate setHeaders(Map headers) { + this.headers = headers; + return this; + } + + public TestAssertionJwtTemplate setPayload(Map payload) { + this.payload = payload; + return this; + } + + @Override + public Map header() { + return headers; + } + + @Override + public Map payload() { + return payload; + } + } +} 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 deleted file mode 100644 index 8b1c5a370652e..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetrieverTest.java +++ /dev/null @@ -1,238 +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 com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ObjectNode; - -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 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\"}")); - } - - @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\"}")); - - // 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\"}")); - - // error response is valid json but unknown keys - when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream( - "{\"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)); - } - - @Test - public void testParseAccessToken() throws IOException { - String expected = "abc"; - ObjectMapper mapper = new ObjectMapper(); - ObjectNode node = mapper.createObjectNode(); - node.put("access_token", expected); - - String actual = HttpAccessTokenRetriever.parseAccessToken(mapper.writeValueAsString(node)); - assertEquals(expected, actual); - } - - @Test - public void testParseAccessTokenEmptyAccessToken() { - ObjectMapper mapper = new ObjectMapper(); - ObjectNode node = mapper.createObjectNode(); - node.put("access_token", ""); - - assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.parseAccessToken(mapper.writeValueAsString(node))); - } - - @Test - public void testParseAccessTokenMissingAccessToken() { - ObjectMapper mapper = new ObjectMapper(); - ObjectNode node = mapper.createObjectNode(); - node.put("sub", "jdoe"); - - assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.parseAccessToken(mapper.writeValueAsString(node))); - } - - @Test - 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)); - } - - @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/HttpRequestGeneratorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestGeneratorTest.java new file mode 100644 index 0000000000000..c24743646e3a3 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestGeneratorTest.java @@ -0,0 +1,34 @@ +/* + * 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.OAuthBearerTest; + +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public abstract class HttpRequestGeneratorTest extends OAuthBearerTest { + + protected void assertBodyEquals(HttpRequestGenerator requestGenerator, String expected) { + assertEquals(expected, requestGenerator.generateBody()); + } + + protected void assertHeadersEqual(HttpRequestGenerator requestGenerator, Map expected) { + assertEquals(expected, requestGenerator.generateHeaders()); + } +} 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 deleted file mode 100644 index 722c734dcc7d6..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtilsTest.java +++ /dev/null @@ -1,62 +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.SslConfigs; - -import org.junit.jupiter.api.Test; - -import java.net.URL; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -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; - -public class JaasOptionsUtilsTest extends OAuthBearerTest { - - @Test - public void testSSLClientConfig() { - Map options = new HashMap<>(); - String sslKeystore = "test.keystore.jks"; - String sslTruststore = "test.truststore.jks"; - - options.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslKeystore); - options.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "$3cr3+"); - options.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslTruststore); - - JaasOptionsUtils jou = new JaasOptionsUtils(options); - Map sslClientConfig = jou.getSslClientConfig(); - assertNotNull(sslClientConfig); - assertEquals(sslKeystore, sslClientConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)); - assertEquals(sslTruststore, sslClientConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); - assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, sslClientConfig.get(SslConfigs.SSL_PROTOCOL_CONFIG)); - } - - @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"))); - } - -} 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/JwksFileVerificationKeyResolverTest.java similarity index 56% 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/JwksFileVerificationKeyResolverTest.java index c2324b9d2dac2..b2397aacbf128 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/JwksFileVerificationKeyResolverTest.java @@ -14,22 +14,31 @@ * 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.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.test.TestUtils; +import org.jose4j.lang.JoseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import java.io.File; +import java.nio.file.Files; import java.util.Collections; 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.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; -public class VerificationKeyResolverFactoryTest extends OAuthBearerTest { +public class JwksFileVerificationKeyResolverTest { @AfterEach public void tearDown() throws Exception { @@ -38,15 +47,13 @@ public void tearDown() throws Exception { @Test public void testConfigureRefreshingFileVerificationKeyResolver() throws Exception { - File tmpDir = createTempDir("access-token"); - File verificationKeyFile = createTempFile(tmpDir, "access-token-", ".json", "{}"); - - 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)) { } + File file = TestUtils.tempFile("access-token-", ".json"); + Files.writeString(file.toPath(), "{}"); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file.toURI().toString()); + Map configs = Collections.singletonMap(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, file.toURI().toString()); + KafkaException error = assertThrows(KafkaException.class, () -> new JwksFileVerificationKeyResolver(configs, OAUTHBEARER_MECHANISM)); + assertNotNull(error.getCause()); + assertInstanceOf(JoseException.class, error.getCause()); } @Test @@ -55,28 +62,30 @@ public void testConfigureRefreshingFileVerificationKeyResolverWithInvalidDirecto 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"); + assertThrows(ConfigException.class, () -> new JwksFileVerificationKeyResolver(configs, OAUTHBEARER_MECHANISM)); } @Test - public void testConfigureRefreshingFileVerificationKeyResolverWithInvalidFile() throws Exception { + public void testConfigureRefreshingFileVerificationKeyResolverWithInvalidFile() { // Should fail because while the parent path exists, the file itself doesn't. - File tmpDir = createTempDir("this-directory-does-exist"); - 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"); + File file = new File(TestUtils.tempDirectory(), "this-file-does-not-exist.json"); + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file.toURI().toString()); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, file.toURI().toString()); + assertThrows(ConfigException.class, () -> new JwksFileVerificationKeyResolver(configs, OAUTHBEARER_MECHANISM)); } @Test - public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception { + public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() { // Should fail if the URL is not allowed - 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); + File file = new File("not_allowed.json"); + Map configs = getSaslConfigs(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, file.toURI().toString()); + assertThrows(ConfigException.class, () -> new JwksFileVerificationKeyResolver(configs, OAUTHBEARER_MECHANISM)); + } + + private Map getSaslConfigs(String name, Object value) { + ConfigDef configDef = new ConfigDef(); + configDef.withClientSaslSupport(); + AbstractConfig sslClientConfig = new AbstractConfig(configDef, Collections.singletonMap(name, value)); + return sslClientConfig.values(); } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGeneratorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGeneratorTest.java new file mode 100644 index 0000000000000..49c0cb2e8e041 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGeneratorTest.java @@ -0,0 +1,71 @@ +/* + * 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.AssertionCreator; +import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate; + +import org.junit.jupiter.api.Test; + +import java.net.URL; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +import static org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBearerRequestGenerator.GRANT_TYPE; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class JwtBearerRequestGeneratorTest extends HttpRequestGeneratorTest { + + private static final String FAKE_ASSERTION = "this-is-fake"; + + @Test + public void testRequestBodyParameters() throws Exception { + Builder builder = new Builder(); + JwtBearerRequestGenerator requestGenerator = builder.build(); + String requestBody = requestGenerator.generateBody(); + String expected = "grant_type=" + URLEncoder.encode(GRANT_TYPE, StandardCharsets.UTF_8) + "&assertion=" + FAKE_ASSERTION; + assertEquals( + expected, + requestBody + ); + } + + private static class Builder { + + private JwtBearerRequestGenerator build() throws Exception { + AssertionCreator assertionCreator = t -> FAKE_ASSERTION; + AssertionJwtTemplate assertionJwtTemplate = new AssertionJwtTemplate() { + @Override + public Map header() { + return Map.of(); + } + + @Override + public Map payload() { + return Map.of(); + } + }; + + return new JwtBearerRequestGenerator( + new URL("http://www.example.com"), + assertionCreator, + assertionJwtTemplate + ); + } + } +} 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/JwtBuilder.java similarity index 86% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenBuilder.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBuilder.java index cc910e0d16c4f..844b1319c69ea 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/JwtBuilder.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; @@ -34,7 +33,9 @@ import java.util.HashMap; import java.util.Map; -public class AccessTokenBuilder { +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimNameOverride; + +public class JwtBuilder { private final ObjectMapper objectMapper = new ObjectMapper(); @@ -58,11 +59,11 @@ public class AccessTokenBuilder { private final Map customClaims = new HashMap<>(); - public AccessTokenBuilder() { + public JwtBuilder() { this(new MockTime()); } - public AccessTokenBuilder(Time time) { + public JwtBuilder(Time time) { this.issuedAtSeconds = time.milliseconds() / 1000; this.expirationSeconds = this.issuedAtSeconds + 60; } @@ -71,12 +72,12 @@ public String alg() { return alg; } - public AccessTokenBuilder alg(String alg) { + public JwtBuilder alg(String alg) { this.alg = alg; return this; } - public AccessTokenBuilder audience(String audience) { + public JwtBuilder audience(String audience) { this.audience = audience; return this; } @@ -85,7 +86,7 @@ public String subject() { return subject; } - public AccessTokenBuilder subject(String subject) { + public JwtBuilder subject(String subject) { this.subject = subject; return this; } @@ -94,7 +95,7 @@ public String subjectClaimName() { return subjectClaimName; } - public AccessTokenBuilder subjectClaimName(String subjectClaimName) { + public JwtBuilder subjectClaimName(String subjectClaimName) { this.subjectClaimName = subjectClaimName; return this; } @@ -103,7 +104,7 @@ public Object scope() { return scope; } - public AccessTokenBuilder scope(Object scope) { + public JwtBuilder scope(Object scope) { this.scope = scope; if (scope instanceof String) { @@ -131,7 +132,7 @@ public Long expirationSeconds() { return expirationSeconds; } - public AccessTokenBuilder expirationSeconds(Long expirationSeconds) { + public JwtBuilder expirationSeconds(Long expirationSeconds) { this.expirationSeconds = expirationSeconds; return this; } @@ -140,14 +141,14 @@ public PublicJsonWebKey jwk() { return jwk; } - public AccessTokenBuilder jwk(PublicJsonWebKey jwk) { + public JwtBuilder jwk(PublicJsonWebKey jwk) { this.jwk = jwk; return this; } - public AccessTokenBuilder addCustomClaim(String name, String value) { - String validatedName = ClaimValidationUtils.validateClaimNameOverride("claim name", name); - String validatedValue = ClaimValidationUtils.validateClaimNameOverride(validatedName, value); + public JwtBuilder addCustomClaim(String name, String value) { + String validatedName = validateClaimNameOverride("claim name", name); + String validatedValue = validateClaimNameOverride(validatedName, value); customClaims.put(validatedName, validatedValue); return this; @@ -201,5 +202,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/JwtHttpResponseBodyHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpResponseBodyHandlerTest.java new file mode 100644 index 0000000000000..b477bc1ad4030 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpResponseBodyHandlerTest.java @@ -0,0 +1,71 @@ +/* + * 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.JwtRetrieverException; +import org.apache.kafka.common.security.oauthbearer.OAuthBearerTest; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class JwtHttpResponseBodyHandlerTest extends OAuthBearerTest { + + @Test + public void testParseJwt() throws IOException { + String expected = "abc"; + ObjectMapper mapper = new ObjectMapper(); + ObjectNode node = mapper.createObjectNode(); + node.put("id_token", expected); + + JwtHttpResponseBodyHandler handler = new JwtHttpResponseBodyHandler(); + String actual = handler.extractJwt(mapper.writeValueAsString(node)); + assertEquals(expected, actual); + } + + @Test + public void testParseJwtEmptyJwt() { + ObjectMapper mapper = new ObjectMapper(); + ObjectNode node = mapper.createObjectNode(); + node.put("id_token", ""); + + JwtHttpResponseBodyHandler handler = new JwtHttpResponseBodyHandler(); + assertThrows(JwtRetrieverException.class, () -> handler.extractJwt(mapper.writeValueAsString(node))); + } + + @Test + public void testParseJwtMissingJwt() { + ObjectMapper mapper = new ObjectMapper(); + ObjectNode node = mapper.createObjectNode(); + node.put("sub", "jdoe"); + + JwtHttpResponseBodyHandler handler = new JwtHttpResponseBodyHandler(); + assertThrows(JwtRetrieverException.class, () -> handler.extractJwt(mapper.writeValueAsString(node))); + } + + @Test + public void testParseJwtInvalidJson() { + JwtHttpResponseBodyHandler handler = new JwtHttpResponseBodyHandler(); + assertThrows(JwtRetrieverException.class, () -> handler.extractJwt("not valid JSON")); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtilsTest.java new file mode 100644 index 0000000000000..e97f85c20e150 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtilsTest.java @@ -0,0 +1,351 @@ +/* + * 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.ConfigException; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.security.oauthbearer.JwtValidatorException; +import org.apache.kafka.common.security.oauthbearer.OAuthBearerTest; +import org.apache.kafka.test.TestUtils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +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.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.getSslClientConfig; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.maybeCreateSslResource; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.throwIfURLIsNotAllowed; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimExpiration; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimIssuedAt; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimNameOverride; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimScopes; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateClaimSubject; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateFileUrl; +import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateUrl; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +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.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class OAuthBearerUtilsTest extends OAuthBearerTest { + + private static final String URL_CONFIG_NAME = "url"; + + @AfterEach + public void tearDown() throws Exception { + System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + } + + @Test + public void testUrl() { + testUrl("http://www.example.com"); + } + + @Test + public void testUrlWithSuperfluousWhitespace() { + testUrl(String.format(" %s ", "http://www.example.com")); + } + + @Test + public void testUrlCaseInsensitivity() { + testUrl("HTTPS://WWW.EXAMPLE.COM"); + } + + @Test + public void testUrlFile() { + testUrl("file:///tmp/foo.txt"); + } + + @Test + public void testUrlFullPath() { + testUrl("https://myidp.example.com/oauth2/default/v1/token"); + } + + @Test + public void testUrlMissingProtocol() { + assertThrowsWithMessage(ConfigException.class, () -> testUrl("www.example.com"), "no protocol"); + } + + @Test + public void testUrlInvalidProtocol() { + assertThrowsWithMessage(ConfigException.class, () -> testUrl("ftp://ftp.example.com"), "invalid protocol"); + } + + @Test + public void testUrlNull() { + assertThrows(ConfigException.class, () -> testUrl(null)); + } + + @Test + public void testUrlEmptyString() { + assertThrows(ConfigException.class, () -> testUrl("")); + } + + @Test + public void testUrlWhitespace() { + assertThrows(ConfigException.class, () -> testUrl(" ")); + } + + @Test + public void testFile() throws IOException { + File file = TestUtils.tempFile("some contents!"); + testFile(file.toURI().toURL().toString()); + } + + @Test + public void testFileWithSuperfluousWhitespace() throws IOException { + File file = TestUtils.tempFile(); + testFile(String.format(" %s ", file.toURI().toURL())); + } + + @Test + public void testFileDoesNotExist() { + assertThrowsWithMessage(ConfigException.class, () -> testFile(new File("/tmp/not/a/real/file.txt").toURI().toURL().toString()), "that doesn't exist"); + } + + @Test + public void testFileUnreadable() throws IOException { + File file = TestUtils.tempFile(); + + if (!file.setReadable(false)) + throw new IllegalStateException(String.format("Can't test file permissions as test couldn't programmatically make temp file %s un-readable", file.getAbsolutePath())); + + assertThrowsWithMessage(ConfigException.class, () -> testFile(file.toURI().toURL().toString()), "that doesn't have read permission"); + } + + @Test + public void testFileNull() { + assertThrows(ConfigException.class, () -> testFile(null)); + } + + @Test + public void testFileEmptyString() { + assertThrows(ConfigException.class, () -> testFile("")); + } + + @Test + public void testFileWhitespace() { + assertThrows(ConfigException.class, () -> testFile(" ")); + } + + @Test + public void testThrowIfURLIsNotAllowed() { + String url = "http://www.example.com"; + String fileUrl = "file:///etc/passwd"; + + // By default, no URL is allowed + assertThrowsWithMessage(ConfigException.class, () -> throwIfURLIsNotAllowed(url), + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + assertThrowsWithMessage(ConfigException.class, () -> throwIfURLIsNotAllowed(fileUrl), + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + + // add one url into allowed list + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, url); + assertDoesNotThrow(() -> throwIfURLIsNotAllowed(url)); + assertThrowsWithMessage(ConfigException.class, () -> throwIfURLIsNotAllowed(fileUrl), + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + + // add all urls into allowed list + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, url + "," + fileUrl); + assertDoesNotThrow(() -> throwIfURLIsNotAllowed(url)); + assertDoesNotThrow(() -> throwIfURLIsNotAllowed(fileUrl)); + } + + @Test + public void testValidateScopes() { + Set scopes = validateClaimScopes("scope", Arrays.asList(" a ", " b ")); + + assertEquals(2, scopes.size()); + assertTrue(scopes.contains("a")); + assertTrue(scopes.contains("b")); + } + + @Test + public void testValidateScopesDisallowsDuplicates() { + assertThrows(JwtValidatorException.class, () -> validateClaimScopes("scope", Arrays.asList("a", "b", "a"))); + assertThrows(JwtValidatorException.class, () -> validateClaimScopes("scope", Arrays.asList("a", "b", " a "))); + } + + @Test + public void testValidateScopesDisallowsEmptyNullAndWhitespace() { + assertThrows(JwtValidatorException.class, () -> validateClaimScopes("scope", Arrays.asList("a", ""))); + assertThrows(JwtValidatorException.class, () -> validateClaimScopes("scope", Arrays.asList("a", null))); + assertThrows(JwtValidatorException.class, () -> validateClaimScopes("scope", Arrays.asList("a", " "))); + } + + @Test + public void testValidateScopesResultIsImmutable() { + SortedSet callerSet = new TreeSet<>(Arrays.asList("a", "b", "c")); + Set scopes = validateClaimScopes("scope", callerSet); + + assertEquals(3, scopes.size()); + + callerSet.add("d"); + assertEquals(4, callerSet.size()); + assertTrue(callerSet.contains("d")); + assertEquals(3, scopes.size()); + assertFalse(scopes.contains("d")); + + callerSet.remove("c"); + assertEquals(3, callerSet.size()); + assertFalse(callerSet.contains("c")); + assertEquals(3, scopes.size()); + assertTrue(scopes.contains("c")); + + callerSet.clear(); + assertEquals(3, scopes.size()); + } + + @Test + public void testValidateScopesResultThrowsExceptionOnMutation() { + SortedSet callerSet = new TreeSet<>(Arrays.asList("a", "b", "c")); + Set scopes = validateClaimScopes("scope", callerSet); + assertThrows(UnsupportedOperationException.class, scopes::clear); + } + + @Test + public void testValidateExpiration() { + Long expected = 1L; + Long actual = validateClaimExpiration("exp", expected); + assertEquals(expected, actual); + } + + @Test + public void testValidateExpirationAllowsZero() { + Long expected = 0L; + Long actual = validateClaimExpiration("exp", expected); + assertEquals(expected, actual); + } + + @Test + public void testValidateExpirationDisallowsNull() { + assertThrows(JwtValidatorException.class, () -> validateClaimExpiration("exp", null)); + } + + @Test + public void testValidateExpirationDisallowsNegatives() { + assertThrows(JwtValidatorException.class, () -> validateClaimExpiration("exp", -1L)); + } + + @Test + public void testValidateSubject() { + String expected = "jdoe"; + String actual = validateClaimSubject("sub", expected); + assertEquals(expected, actual); + } + + @Test + public void testValidateSubjectDisallowsEmptyNullAndWhitespace() { + assertThrows(JwtValidatorException.class, () -> validateClaimSubject("sub", "")); + assertThrows(JwtValidatorException.class, () -> validateClaimSubject("sub", null)); + assertThrows(JwtValidatorException.class, () -> validateClaimSubject("sub", " ")); + } + + @Test + public void testValidateClaimNameOverride() { + String expected = "email"; + String actual = validateClaimNameOverride("sub", String.format(" %s ", expected)); + assertEquals(expected, actual); + } + + @Test + public void testValidateClaimNameOverrideDisallowsEmptyNullAndWhitespace() { + assertThrows(JwtValidatorException.class, () -> validateClaimSubject("sub", "")); + assertThrows(JwtValidatorException.class, () -> validateClaimSubject("sub", null)); + assertThrows(JwtValidatorException.class, () -> validateClaimSubject("sub", " ")); + } + + @Test + public void testValidateIssuedAt() { + Long expected = 1L; + Long actual = validateClaimIssuedAt("iat", expected); + assertEquals(expected, actual); + } + + @Test + public void testValidateIssuedAtAllowsZero() { + Long expected = 0L; + Long actual = validateClaimIssuedAt("iat", expected); + assertEquals(expected, actual); + } + + @Test + public void testValidateIssuedAtAllowsNull() { + Long expected = null; + Long actual = validateClaimIssuedAt("iat", expected); + assertEquals(expected, actual); + } + + @Test + public void testValidateIssuedAtDisallowsNegatives() { + assertThrows(JwtValidatorException.class, () -> validateClaimIssuedAt("iat", -1L)); + } + + @Test + public void testSSLClientConfig() { + String sslKeystore = "test.keystore.jks"; + String sslTruststore = "test.truststore.jks"; + + Map options = new HashMap<>(); + options.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslKeystore); + options.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "$3cr3+"); + options.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslTruststore); + + OAuthBearerJaasConfig jaasConfig = new OAuthBearerJaasConfig(options); + Map sslClientConfig = getSslClientConfig(jaasConfig); + assertNotNull(sslClientConfig); + assertEquals(sslKeystore, sslClientConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)); + assertEquals(sslTruststore, sslClientConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); + assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, sslClientConfig.get(SslConfigs.SSL_PROTOCOL_CONFIG)); + } + + @Test + public void testShouldUseSslClientConfig() throws Exception { + OAuthBearerJaasConfig jaasConfig = new OAuthBearerJaasConfig(Collections.emptyMap()); + assertFalse(maybeCreateSslResource(new URL("http://www.example.com"), jaasConfig).isPresent()); + assertTrue(maybeCreateSslResource(new URL("https://www.example.com"), jaasConfig).isPresent()); + assertFalse(maybeCreateSslResource(new URL("file:///tmp/test.txt"), jaasConfig).isPresent()); + } + + private void testUrl(String value) { + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, value == null ? "" : value); + Map configs = Collections.singletonMap(URL_CONFIG_NAME, value); + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, OAUTHBEARER_MECHANISM); + validateUrl(oauthConfig, URL_CONFIG_NAME); + } + + private void testFile(String value) { + System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, value == null ? "" : value); + Map configs = Collections.singletonMap(URL_CONFIG_NAME, value); + OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, OAUTHBEARER_MECHANISM); + validateFileUrl(oauthConfig, URL_CONFIG_NAME); + } +} 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..cbe9f1928a2df 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; @@ -32,6 +31,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.TreeMap; import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; @@ -42,17 +42,18 @@ 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 { +public class RefreshingHttpsJwksTest { - 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 +65,10 @@ public void testBasicScheduleRefresh() throws Exception { MockTime time = new MockTime(); HttpsJwks httpsJwks = spyHttpsJwks(); - // 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 + 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(); verify(httpsJwks, times(1)).refresh(); assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId)); verify(httpsJwks, times(2)).refresh(); @@ -80,13 +81,12 @@ 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(); assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId)); assertFalse(refreshingHttpsJwks.maybeExpediteRefresh(keyId)); } @@ -98,7 +98,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 +118,6 @@ public void testLongKey() throws Exception { HttpsJwks httpsJwks = spyHttpsJwks(); try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks)) { - refreshingHttpsJwks.init(); verify(httpsJwks, times(1)).refresh(); assertFalse(refreshingHttpsJwks.maybeExpediteRefresh(keyId)); verify(httpsJwks, times(1)).refresh(); @@ -137,7 +136,6 @@ public void testSecondaryRefreshAfterElapsedDelay() throws Exception { HttpsJwks httpsJwks = spyHttpsJwks(); try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks)) { - refreshingHttpsJwks.init(); // We refresh once at the initialization time from getJsonWebKeys. verify(httpsJwks, times(1)).refresh(); assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId)); @@ -173,13 +171,12 @@ 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(); assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId)); time.sleep(sleepDelay); assertEquals(shouldBeScheduled, refreshingHttpsJwks.maybeExpediteRefresh(keyId)); @@ -187,7 +184,15 @@ private void assertMaybeExpediteRefreshWithDelay(long sleepDelay, boolean should } 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)); + return new RefreshingHttpsJwks( + time, + httpsJwks, + Optional.empty(), + mockExecutorService(time), + 1000, + 10, + 10000 + ); } /** @@ -301,7 +306,7 @@ public ScheduledFuture schedule(final Callable callable, long delayMs, try { callable.call(); } catch (Throwable e) { - e.printStackTrace(); + e.printStackTrace(System.err); } return null; }); @@ -309,5 +314,4 @@ public ScheduledFuture schedule(final Callable callable, long delayMs, return null; } } - } \ No newline at end of file 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 deleted file mode 100644 index b239f325cffa3..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RetryTest.java +++ /dev/null @@ -1,138 +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.MockTime; -import org.apache.kafka.common.utils.Time; - -import org.junit.jupiter.api.Test; - -import java.io.IOException; -import java.util.concurrent.ExecutionException; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class RetryTest extends OAuthBearerTest { - - @Test - public void test() throws ExecutionException { - Exception[] attempts = new Exception[] { - new IOException("pretend connect error"), - new IOException("pretend timeout error"), - new IOException("pretend read error"), - null // success! - }; - long retryWaitMs = 1000; - long maxWaitMs = 10000; - Retryable call = createRetryable(attempts); - - Time time = new MockTime(0, 0, 0); - assertEquals(0L, time.milliseconds()); - Retry r = new Retry<>(time, retryWaitMs, maxWaitMs); - r.execute(call); - - long secondWait = retryWaitMs * 2; - long thirdWait = retryWaitMs * 4; - long totalWait = retryWaitMs + secondWait + thirdWait; - assertEquals(totalWait, time.milliseconds()); - } - - @Test - public void testIOExceptionFailure() { - Exception[] attempts = new Exception[] { - new IOException("pretend connect error"), - new IOException("pretend timeout error"), - new IOException("pretend read error"), - new IOException("pretend another read error"), - }; - long retryWaitMs = 1000; - long maxWaitMs = 1000 + 2000 + 3999; - Retryable call = createRetryable(attempts); - - Time time = new MockTime(0, 0, 0); - assertEquals(0L, time.milliseconds()); - Retry r = new Retry<>(time, retryWaitMs, maxWaitMs); - - assertThrows(ExecutionException.class, () -> r.execute(call)); - - assertEquals(maxWaitMs, time.milliseconds()); - } - - @Test - public void testRuntimeExceptionFailureOnLastAttempt() { - Exception[] attempts = new Exception[] { - new IOException("pretend connect error"), - new IOException("pretend timeout error"), - new NullPointerException("pretend JSON node /userId in response is null") - }; - long retryWaitMs = 1000; - long maxWaitMs = 10000; - Retryable call = createRetryable(attempts); - - Time time = new MockTime(0, 0, 0); - assertEquals(0L, time.milliseconds()); - Retry r = new Retry<>(time, retryWaitMs, maxWaitMs); - - assertThrows(RuntimeException.class, () -> r.execute(call)); - - long secondWait = retryWaitMs * 2; - long totalWait = retryWaitMs + secondWait; - assertEquals(totalWait, time.milliseconds()); - } - - @Test - public void testRuntimeExceptionFailureOnFirstAttempt() { - Exception[] attempts = new Exception[] { - new NullPointerException("pretend JSON node /userId in response is null"), - null - }; - long retryWaitMs = 1000; - long maxWaitMs = 10000; - Retryable call = createRetryable(attempts); - - Time time = new MockTime(0, 0, 0); - assertEquals(0L, time.milliseconds()); - Retry r = new Retry<>(time, retryWaitMs, maxWaitMs); - - assertThrows(RuntimeException.class, () -> r.execute(call)); - - assertEquals(0, time.milliseconds()); - } - - @Test - public void testUseMaxTimeout() { - Exception[] attempts = new Exception[] { - new IOException("pretend connect error"), - new IOException("pretend timeout error"), - new IOException("pretend read error") - }; - long retryWaitMs = 5000; - long maxWaitMs = 5000; - Retryable call = createRetryable(attempts); - - Time time = new MockTime(0, 0, 0); - assertEquals(0L, time.milliseconds()); - Retry r = new Retry<>(time, retryWaitMs, maxWaitMs); - - assertThrows(ExecutionException.class, () -> r.execute(call)); - - assertEquals(maxWaitMs, time.milliseconds()); - } - -} 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..c2d429e6d568c 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.DefaultJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.DefaultJwtValidator; +import org.apache.kafka.common.security.oauthbearer.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.JwtValidator; 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"); - } + String jwt; + + // Client retrieval + try (JwtRetriever retriever = new DefaultJwtRetriever(); + JwtValidator validator = new DefaultJwtValidator()) { + // 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"); + + jwt = retriever.retrieve(); + System.out.println("PASSED 2/5: client JWT retrieval"); + + validator.validate(jwt); + 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 (JwtValidator validator = new DefaultJwtValidator()) { + String fileName = namespace.getString("brokerConfigurationFileName"); + Map configs = Utils.propsToMap(Utils.loadProps(fileName)); + JaasContext context = JaasContext.loadClientContext(configs); + List jaasConfigEntries = context.configurationEntries(); + + validator.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries); + System.out.println("PASSED 4/5: broker configuration"); - atv.validate(accessToken); - System.out.println("PASSED 5/5: broker JWT validation"); - } + validator.validate(jwt); + 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); - } - - } - }