From 667bb22ffa4d09a9b139bdea8d5bec8bee59ff90 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 2 May 2025 08:30:57 -0700 Subject: [PATCH 01/27] KAFKA-18847: Refactor OAuth layer to improve reusability 1/N Rename AccessTokenRetriever and AccessTokenValidator to JwtRetriever and JwtValidator, respectively. Also converting the factory pattern classes AccessTokenRetrieverFactory and AccessTokenValidatorFactory into delegate/wrapper classes DefaultJwtRetriever and DefaultJwtValidator, respectively. These are all internal changes, no configuration, user APIs, RPCs, etc. were changed. --- .../OAuthBearerLoginCallbackHandler.java | 52 ++++----- .../OAuthBearerValidatorCallbackHandler.java | 32 +++--- .../secured/AccessTokenValidatorFactory.java | 73 ------------- ...Validator.java => BrokerJwtValidator.java} | 31 ++++-- ...Validator.java => ClientJwtValidator.java} | 21 +++- ...rFactory.java => DefaultJwtRetriever.java} | 53 +++++---- .../secured/DefaultJwtValidator.java | 103 ++++++++++++++++++ ...enRetriever.java => FileJwtRetriever.java} | 8 +- ...enRetriever.java => HttpJwtRetriever.java} | 28 ++--- ...sTokenRetriever.java => JwtRetriever.java} | 10 +- ...sTokenValidator.java => JwtValidator.java} | 13 ++- .../secured/RefreshingHttpsJwks.java | 4 +- .../internals/secured/ValidateException.java | 2 +- .../VerificationKeyResolverFactory.java | 2 +- .../OAuthBearerLoginCallbackHandlerTest.java | 39 ++++--- ...uthBearerValidatorCallbackHandlerTest.java | 8 +- ...rTest.java => BrokerJwtValidatorTest.java} | 10 +- ...rTest.java => ClientJwtValidatorTest.java} | 6 +- ...Test.java => DefaultJwtRetrieverTest.java} | 34 ++++-- ...verTest.java => HttpJwtRetrieverTest.java} | 62 +++++------ ...Test.java => JwtValidatorFactoryTest.java} | 28 +++-- ...lidatorTest.java => JwtValidatorTest.java} | 22 ++-- .../kafka/tools/OAuthCompatibilityTool.java | 37 ++++--- 23 files changed, 386 insertions(+), 292 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactory.java rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{ValidatorAccessTokenValidator.java => BrokerJwtValidator.java} (91%) rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{LoginAccessTokenValidator.java => ClientJwtValidator.java} (89%) rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{AccessTokenRetrieverFactory.java => DefaultJwtRetriever.java} (77%) create mode 100644 clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{FileTokenRetriever.java => FileJwtRetriever.java} (87%) rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{HttpAccessTokenRetriever.java => HttpJwtRetriever.java} (95%) rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{AccessTokenRetriever.java => JwtRetriever.java} (88%) rename clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{AccessTokenValidator.java => JwtValidator.java} (84%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{ValidatorAccessTokenValidatorTest.java => BrokerJwtValidatorTest.java} (89%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{LoginAccessTokenValidatorTest.java => ClientJwtValidatorTest.java} (76%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{AccessTokenRetrieverFactoryTest.java => DefaultJwtRetrieverTest.java} (74%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{HttpAccessTokenRetrieverTest.java => HttpJwtRetrieverTest.java} (73%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{AccessTokenValidatorFactoryTest.java => JwtValidatorFactoryTest.java} (65%) rename clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/{AccessTokenValidatorTest.java => JwtValidatorTest.java} (80%) 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..0a7a148521177 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 @@ -24,13 +24,14 @@ 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.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; 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; @@ -179,28 +180,34 @@ public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHand private Map moduleOptions; - private AccessTokenRetriever accessTokenRetriever; + private JwtRetriever jwtRetriever; - private AccessTokenValidator accessTokenValidator; + private JwtValidator jwtValidator; private boolean isInitialized = false; @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); + JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, saslMechanism, moduleOptions); + JwtValidator jwtValidator = new DefaultJwtValidator(configs, saslMechanism); + init(jwtRetriever, jwtValidator); } - public void init(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator accessTokenValidator) { - this.accessTokenRetriever = accessTokenRetriever; - this.accessTokenValidator = accessTokenValidator; + public void init(JwtRetriever jwtRetriever, JwtValidator jwtValidator) { + this.jwtRetriever = jwtRetriever; + this.jwtValidator = jwtValidator; try { - this.accessTokenRetriever.init(); + this.jwtRetriever.init(); } catch (IOException e) { - throw new KafkaException("The OAuth login configuration encountered an error when initializing the AccessTokenRetriever", e); + throw new KafkaException("The OAuth login configuration encountered an error when initializing the JwtRetriever", e); + } + + try { + this.jwtValidator.init(); + } catch (IOException e) { + throw new KafkaException("The OAuth login configuration encountered an error when initializing the JwtValidator", e); } isInitialized = true; @@ -210,19 +217,14 @@ public void init(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator * Package-visible for testing. */ - AccessTokenRetriever getAccessTokenRetriever() { - return accessTokenRetriever; + JwtRetriever jwtRetriever() { + return jwtRetriever; } @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, "The OAuth login callback encountered an error when closing the JwtRetriever"); + Utils.closeQuietly(jwtValidator, "The OAuth login callback encountered an error when closing the JwtValidator"); } @Override @@ -242,10 +244,10 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback private void handleTokenCallback(OAuthBearerTokenCallback callback) throws IOException { checkInitialized(); - String accessToken = accessTokenRetriever.retrieve(); + String accessToken = jwtRetriever.retrieve(); try { - OAuthBearerToken token = accessTokenValidator.validate(accessToken); + OAuthBearerToken token = jwtValidator.validate(accessToken); callback.token(token); } catch (ValidateException e) { log.warn(e.getMessage(), e); 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..ec56fb93d8c25 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 @@ -19,14 +19,15 @@ import org.apache.kafka.common.KafkaException; 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.JwtValidator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; 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; @@ -119,7 +120,7 @@ public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallback private CloseableVerificationKeyResolver verificationKeyResolver; - private AccessTokenValidator accessTokenValidator; + private JwtValidator jwtValidator; private boolean isInitialized = false; @@ -135,13 +136,19 @@ public void configure(Map configs, String saslMechanism, List 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/ValidatorAccessTokenValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java similarity index 91% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidator.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java index c7ae8edae9d93..ac481a75129ac 100644 --- 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/BrokerJwtValidator.java @@ -31,6 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Set; @@ -38,7 +39,7 @@ import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE; /** - * ValidatorAccessTokenValidator is an implementation of {@link AccessTokenValidator} that is used + * {@code BrokerJwtValidator} is an 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. @@ -62,9 +63,9 @@ * */ -public class ValidatorAccessTokenValidator implements AccessTokenValidator { +public class BrokerJwtValidator implements JwtValidator { - private static final Logger log = LoggerFactory.getLogger(ValidatorAccessTokenValidator.class); + private static final Logger log = LoggerFactory.getLogger(BrokerJwtValidator.class); private final JwtConsumer jwtConsumer; @@ -73,7 +74,7 @@ public class ValidatorAccessTokenValidator implements AccessTokenValidator { private final String subClaimName; /** - * Creates a new ValidatorAccessTokenValidator that will be used by the broker for more + * Creates a new {@code BrokerJwtValidator} 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 @@ -112,12 +113,12 @@ public class ValidatorAccessTokenValidator implements AccessTokenValidator { * @see VerificationKeyResolver */ - public ValidatorAccessTokenValidator(Integer clockSkew, - Set expectedAudiences, - String expectedIssuer, - VerificationKeyResolver verificationKeyResolver, - String scopeClaimName, - String subClaimName) { + public BrokerJwtValidator(Integer clockSkew, + Set expectedAudiences, + String expectedIssuer, + VerificationKeyResolver verificationKeyResolver, + String scopeClaimName, + String subClaimName) { final JwtConsumerBuilder jwtConsumerBuilder = new JwtConsumerBuilder(); if (clockSkew != null) @@ -190,6 +191,16 @@ else if (scopeRaw instanceof Collection) issuedAt); } + @Override + public void init() throws IOException { + // Do nothing... + } + + @Override + public void close() throws IOException { + // Do nothing... + } + private T getClaim(ClaimSupplier supplier, String claimName) throws ValidateException { try { T value = supplier.get(); 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/internals/secured/ClientJwtValidator.java similarity index 89% 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/internals/secured/ClientJwtValidator.java index 773311ff0ab18..60f3743ae8089 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/internals/secured/ClientJwtValidator.java @@ -24,6 +24,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Map; @@ -33,7 +34,7 @@ import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME; /** - * 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. @@ -50,9 +51,9 @@ * */ -public class LoginAccessTokenValidator implements AccessTokenValidator { +public class ClientJwtValidator implements JwtValidator { - 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"; @@ -63,14 +64,14 @@ public class LoginAccessTokenValidator implements AccessTokenValidator { private final String subClaimName; /** - * Creates a new LoginAccessTokenValidator that will be used by the client for lightweight + * Creates a new {@code ClientJwtValidator} 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) { + public ClientJwtValidator(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); } @@ -123,6 +124,16 @@ else if (scopeRaw instanceof Collection) issuedAt); } + @Override + public void init() throws IOException { + // Do nothing... + } + + @Override + public void close() throws IOException { + // Do nothing... + } + private Object getClaim(Map payload, String claimName) { Object value = payload.get(claimName); log.debug("getClaim - {}: {}", claimName, value); 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/DefaultJwtRetriever.java similarity index 77% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java index 0ed4a1a230349..51f1042be83ed 100644 --- 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/DefaultJwtRetriever.java @@ -18,10 +18,13 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.utils.Utils; +import java.io.IOException; import java.net.URL; import java.util.Locale; import java.util.Map; +import java.util.Objects; import javax.net.ssl.SSLSocketFactory; @@ -36,32 +39,27 @@ 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 { +public class DefaultJwtRetriever implements JwtRetriever { - /** - * 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} - */ + private final Map configs; + private final String saslMechanism; + private final Map jaasConfig; + + private JwtRetriever delegate; - public static AccessTokenRetriever create(Map configs, Map jaasConfig) { - return create(configs, null, jaasConfig); + public DefaultJwtRetriever(Map configs, String saslMechanism, Map jaasConfig) { + this.configs = configs; + this.saslMechanism = saslMechanism; + this.jaasConfig = jaasConfig; } - public static AccessTokenRetriever create(Map configs, - String saslMechanism, - Map jaasConfig) { + @Override + public void init() throws IOException { 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)); + delegate = new FileJwtRetriever(cu.validateFile(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL)); } else { JaasOptionsUtils jou = new JaasOptionsUtils(jaasConfig); String clientId = jou.validateString(CLIENT_ID_CONFIG); @@ -75,7 +73,7 @@ public static AccessTokenRetriever create(Map configs, boolean urlencodeHeader = validateUrlencodeHeader(cu); - return new HttpAccessTokenRetriever(clientId, + delegate = new HttpJwtRetriever(clientId, clientSecret, scope, sslSocketFactory, @@ -86,6 +84,18 @@ public static AccessTokenRetriever create(Map configs, cu.validateInteger(SASL_LOGIN_READ_TIMEOUT_MS, false), urlencodeHeader); } + + delegate.init(); + } + + @Override + public String retrieve() throws IOException { + return Objects.requireNonNull(delegate).retrieve(); + } + + @Override + public void close() throws IOException { + Utils.closeQuietly(delegate, "delegate"); } /** @@ -96,7 +106,7 @@ public static AccessTokenRetriever create(Map configs, *

* * This utility method ensures that we have a non-{@code null} value to use in the - * {@link HttpAccessTokenRetriever} constructor. + * {@link HttpJwtRetriever} constructor. */ static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) { Boolean urlencodeHeader = configurationUtils.validateBoolean(SASL_OAUTHBEARER_HEADER_URLENCODE, false); @@ -107,4 +117,7 @@ static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) { return DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; } + public JwtRetriever delegate() { + return delegate; + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java new file mode 100644 index 0000000000000..56c9238a5ccb3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java @@ -0,0 +1,103 @@ +/* + * 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.apache.kafka.common.utils.Utils; +import org.jose4j.keys.resolvers.VerificationKeyResolver; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +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 DefaultJwtValidator implements JwtValidator { + + private final Map configs; + private final String saslMechanism; + private final Optional verificationKeyResolver; + + private JwtValidator delegate; + + public DefaultJwtValidator(Map configs, String saslMechanism) { + this.configs = configs; + this.saslMechanism = saslMechanism; + this.verificationKeyResolver = Optional.empty(); + } + + public DefaultJwtValidator(Map configs, + String saslMechanism, + VerificationKeyResolver verificationKeyResolver) { + this.configs = configs; + this.saslMechanism = saslMechanism; + this.verificationKeyResolver = Optional.of(verificationKeyResolver); + } + + @Override + public void init() throws IOException { + ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); + + if (verificationKeyResolver.isPresent()) { + 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); + + delegate = new BrokerJwtValidator(clockSkew, + expectedAudiences, + expectedIssuer, + verificationKeyResolver.get(), + scopeClaimName, + subClaimName); + } else { + String scopeClaimName = cu.get(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME); + String subClaimName = cu.get(SASL_OAUTHBEARER_SUB_CLAIM_NAME); + delegate = new ClientJwtValidator(scopeClaimName, subClaimName); + } + + delegate.init(); + } + + @Override + public OAuthBearerToken validate(String accessToken) throws ValidateException { + return Objects.requireNonNull(delegate).validate(accessToken); + } + + @Override + public void close() throws IOException { + Utils.closeQuietly(delegate, "delegate"); + } + + public JwtValidator delegate() { + return delegate; + } +} 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/FileJwtRetriever.java similarity index 87% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java index c145cf7596959..35cd7cb9fb16f 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java @@ -23,19 +23,19 @@ import java.nio.file.Path; /** - * FileTokenRetriever is an {@link AccessTokenRetriever} that will load the contents, + * FileJwtRetriever is an {@link JwtRetriever} that will load the contents, * interpreting them as a JWT access key in the serialized form. * - * @see AccessTokenRetriever + * @see JwtRetriever */ -public class FileTokenRetriever implements AccessTokenRetriever { +public class FileJwtRetriever implements JwtRetriever { private final Path accessTokenFile; private String accessToken; - public FileTokenRetriever(Path accessTokenFile) { + public FileJwtRetriever(Path accessTokenFile) { this.accessTokenFile = accessTokenFile; } 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/HttpJwtRetriever.java similarity index 95% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetriever.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java index fdc5707278a60..ce440834ef2ba 100644 --- 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/HttpJwtRetriever.java @@ -49,22 +49,22 @@ import javax.net.ssl.SSLSocketFactory; /** - * HttpAccessTokenRetriever is an {@link AccessTokenRetriever} that will + * HttpJwtRetriever is an {@link JwtRetriever} 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 JwtRetriever * @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 { +public class HttpJwtRetriever implements JwtRetriever { - private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class); + private static final Logger log = LoggerFactory.getLogger(HttpJwtRetriever.class); private static final Set UNRETRYABLE_HTTP_CODES; @@ -117,16 +117,16 @@ public class HttpAccessTokenRetriever implements AccessTokenRetriever { 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) { + public HttpJwtRetriever(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; 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/JwtRetriever.java similarity index 88% rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetriever.java rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtRetriever.java index 080ea4515b4dd..b8991250df0e3 100644 --- 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/JwtRetriever.java @@ -21,20 +21,20 @@ import java.io.IOException; /** - * An AccessTokenRetriever is the internal API by which the login module will + * A JwtRetriever 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 + * the JwtRetriever implementation to validate the integrity of the JWT * access token. * - * @see HttpAccessTokenRetriever - * @see FileTokenRetriever + * @see HttpJwtRetriever + * @see FileJwtRetriever */ -public interface AccessTokenRetriever extends Initable, Closeable { +public interface JwtRetriever extends Initable, Closeable { /** * Retrieves a JWT access token in its serialized three-part form. The implementation 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/internals/secured/JwtValidator.java similarity index 84% 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/internals/secured/JwtValidator.java index 0b107a09bc065..cb6698e09aefd 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/internals/secured/JwtValidator.java @@ -19,8 +19,10 @@ import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken; +import java.io.Closeable; + /** - * 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,13 +42,12 @@ *

  • 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 ClientJwtValidator A basic JwtValidator used by client-side login authentication + * @see BrokerJwtValidator A more robust JwtValidator that is used on the broker to validate the token's + * contents and verify the signature */ -public interface AccessTokenValidator { +public interface JwtValidator extends Initable, Closeable { /** * Accepts an OAuth JWT access token in base-64 encoded format, validates, and returns an 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..4d75ff847eae8 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 @@ -49,12 +49,12 @@ * 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 BrokerJwtValidator} to use in validating the signature of * a JWT. * * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver * @see org.jose4j.keys.resolvers.VerificationKeyResolver - * @see ValidatorAccessTokenValidator + * @see BrokerJwtValidator */ public final class RefreshingHttpsJwks implements Initable, Closeable { 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/internals/secured/ValidateException.java index 430b9007830cb..8c107abc831a4 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/internals/secured/ValidateException.java @@ -27,7 +27,7 @@ * processing of a {@link javax.security.auth.callback.CallbackHandler#handle(Callback[])}. * This error, however, is not thrown from that method directly. * - * @see AccessTokenValidator#validate(String) + * @see JwtValidator#validate(String) */ public class ValidateException extends KafkaException { 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 index 0422045fc029d..c4cce0f7e8b38 100644 --- 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 @@ -37,7 +37,7 @@ public class VerificationKeyResolverFactory { /** - * Create an {@link AccessTokenRetriever} from the given + * Create an {@link JwtRetriever} from the given * {@link org.apache.kafka.common.config.SaslConfigs}. * * Note: the returned CloseableVerificationKeyResolver is not 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..f2fa6e1aaa25c 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 @@ -21,11 +21,12 @@ 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.DefaultJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.FileJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpJwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.apache.kafka.common.utils.Utils; @@ -70,9 +71,9 @@ public void testHandleTokenCallback() throws Exception { .jwk(createRsaJwk()) .alg(AlgorithmIdentifiers.RSA_USING_SHA256); String accessToken = builder.build(); - AccessTokenRetriever accessTokenRetriever = () -> accessToken; + JwtRetriever jwtRetriever = () -> accessToken; - OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs); + OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever, configs); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -144,9 +145,9 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() { public void testInvalidCallbackGeneratesUnsupportedCallbackException() { Map configs = getSaslConfigs(); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - AccessTokenRetriever accessTokenRetriever = () -> "foo"; - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs); - handler.init(accessTokenRetriever, accessTokenValidator); + JwtRetriever jwtRetriever = () -> "foo"; + JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM); + handler.init(jwtRetriever, jwtValidator); try { Callback unsupportedCallback = new Callback() { }; @@ -166,11 +167,11 @@ public void testInvalidAccessToken() throws Exception { @Test public void testMissingAccessToken() { - AccessTokenRetriever accessTokenRetriever = () -> { + JwtRetriever jwtRetriever = () -> { throw new IOException("The token endpoint response access_token value must be non-null"); }; Map configs = getSaslConfigs(); - OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs); + OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever, configs); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -196,7 +197,7 @@ public void testFileTokenRetrieverHandlesNewline() throws IOException { File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", withNewline); Map configs = getSaslConfigs(); - OAuthBearerLoginCallbackHandler handler = createHandler(new FileTokenRetriever(accessTokenFile.toPath()), configs); + OAuthBearerLoginCallbackHandler handler = createHandler(new FileJwtRetriever(accessTokenFile.toPath()), configs); OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); try { handler.handle(new Callback[]{callback}); @@ -226,7 +227,8 @@ public void testConfigureWithAccessTokenFile() throws Exception { Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); Map jaasConfigs = Collections.emptyMap(); configureHandler(handler, configs, jaasConfigs); - assertInstanceOf(FileTokenRetriever.class, handler.getAccessTokenRetriever()); + DefaultJwtRetriever defaultJwtRetriever = assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever()); + assertInstanceOf(FileJwtRetriever.class, defaultJwtRetriever.delegate()); } @Test @@ -238,7 +240,8 @@ public void testConfigureWithAccessClientCredentials() { jaasConfigs.put(CLIENT_ID_CONFIG, "an ID"); jaasConfigs.put(CLIENT_SECRET_CONFIG, "a secret"); configureHandler(handler, configs, jaasConfigs); - assertInstanceOf(HttpAccessTokenRetriever.class, handler.getAccessTokenRetriever()); + DefaultJwtRetriever defaultJwtRetriever = assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever()); + assertInstanceOf(HttpJwtRetriever.class, defaultJwtRetriever.delegate()); } private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception { @@ -268,10 +271,10 @@ private String createAccessKey(String header, String payload, String signature) return String.format("%s.%s.%s", header, payload, signature); } - private OAuthBearerLoginCallbackHandler createHandler(AccessTokenRetriever accessTokenRetriever, Map configs) { + private OAuthBearerLoginCallbackHandler createHandler(JwtRetriever jwtRetriever, Map configs) { OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs); - handler.init(accessTokenRetriever, accessTokenValidator); + JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM); + handler.init(jwtRetriever, jwtValidator); return handler; } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java index d682a05ec11cc..e4702cd80ce2f 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java @@ -18,8 +18,8 @@ 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.JwtValidator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; 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; @@ -103,8 +103,8 @@ private OAuthBearerValidatorCallbackHandler createHandler(Map options OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); CloseableVerificationKeyResolver verificationKeyResolver = (jws, nestingContext) -> builder.jwk().getPublicKey(); - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(options, verificationKeyResolver); - handler.init(verificationKeyResolver, accessTokenValidator); + JwtValidator jwtValidator = new DefaultJwtValidator(options, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, verificationKeyResolver); + handler.init(verificationKeyResolver, jwtValidator); return handler; } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidatorTest.java similarity index 89% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidatorTest.java index 4db20e9ee10d6..3b06bf07dece7 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidatorTest.java @@ -28,11 +28,11 @@ import static org.junit.jupiter.api.Assertions.assertEquals; -public class ValidatorAccessTokenValidatorTest extends AccessTokenValidatorTest { +public class BrokerJwtValidatorTest extends JwtValidatorTest { @Override - protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) { - return new ValidatorAccessTokenValidator(30, + protected JwtValidator createJwtValidator(AccessTokenBuilder builder) { + return new BrokerJwtValidator(30, Collections.emptySet(), null, (jws, nestingContext) -> builder.jwk().getKey(), @@ -72,7 +72,7 @@ public void testMissingSubShouldBeValid() throws Exception { .addCustomClaim(subClaimName, subject) .subjectClaimName(subClaimName) .subject(null); - AccessTokenValidator validator = createAccessTokenValidator(tokenBuilder); + JwtValidator validator = createJwtValidator(tokenBuilder); // Validation should succeed (e.g. signature verification) even if sub claim is missing OAuthBearerToken token = validator.validate(tokenBuilder.build()); @@ -82,7 +82,7 @@ public void testMissingSubShouldBeValid() throws Exception { private void testEncryptionAlgorithm(PublicJsonWebKey jwk, String alg) throws Exception { AccessTokenBuilder builder = new AccessTokenBuilder().jwk(jwk).alg(alg); - AccessTokenValidator validator = createAccessTokenValidator(builder); + JwtValidator validator = createJwtValidator(builder); String accessToken = builder.build(); OAuthBearerToken token = validator.validate(accessToken); 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/internals/secured/ClientJwtValidatorTest.java similarity index 76% 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/internals/secured/ClientJwtValidatorTest.java index fc2e3d2a2e83a..280aecd82c3a4 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/internals/secured/ClientJwtValidatorTest.java @@ -17,11 +17,11 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; -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 createJwtValidator(AccessTokenBuilder builder) { + return new ClientJwtValidator(builder.scopeClaimName(), builder.subjectClaimName()); } } 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/DefaultJwtRetrieverTest.java similarity index 74% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java index 3e85f7b0ce4fa..59db66c67cc86 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -26,6 +27,7 @@ import org.junit.jupiter.params.provider.MethodSource; import java.io.File; +import java.io.IOException; import java.util.Collections; import java.util.Map; import java.util.stream.Stream; @@ -36,7 +38,7 @@ 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 { +public class DefaultJwtRetrieverTest extends OAuthBearerTest { @AfterEach public void tearDown() throws Exception { @@ -44,7 +46,7 @@ public void tearDown() throws Exception { } @Test - public void testConfigureRefreshingFileAccessTokenRetriever() throws Exception { + public void testConfigureRefreshingFileJwtRetriever() throws Exception { String expected = "{}"; File tmpDir = createTempDir("access-token"); @@ -54,31 +56,37 @@ public void testConfigureRefreshingFileAccessTokenRetriever() throws Exception { 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()); + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfig)) { + jwtRetriever.init(); + assertEquals(expected, jwtRetriever.retrieve()); } } @Test - public void testConfigureRefreshingFileAccessTokenRetrieverWithInvalidDirectory() { + public void testConfigureRefreshingFileJwtRetrieverWithInvalidDirectory() throws IOException { // 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"); + + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfig)) { + assertThrowsWithMessage(ConfigException.class, jwtRetriever::init, "that doesn't exist"); + } } @Test - public void testConfigureRefreshingFileAccessTokenRetrieverWithInvalidFile() throws Exception { + public void testConfigureRefreshingFileJwtRetrieverWithInvalidFile() 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"); + + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfig)) { + assertThrowsWithMessage(ConfigException.class, jwtRetriever::init, "that doesn't exist"); + } } @Test @@ -87,15 +95,17 @@ public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception { File tmpDir = createTempDir("not_allowed"); File accessTokenFile = new File(tmpDir, "not_allowed.json"); Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); - assertThrowsWithMessage(ConfigException.class, () -> AccessTokenRetrieverFactory.create(configs, Collections.emptyMap()), - ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); + + try (JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, Collections.emptyMap())) { + assertThrowsWithMessage(ConfigException.class, jwtRetriever::init, 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); + boolean actualValue = DefaultJwtRetriever.validateUrlencodeHeader(cu); assertEquals(expectedValue, actualValue); } 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/HttpJwtRetrieverTest.java similarity index 73% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetrieverTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetrieverTest.java index 8b1c5a370652e..0bd903300ff1f 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetrieverTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetrieverTest.java @@ -39,20 +39,20 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class HttpAccessTokenRetrieverTest extends OAuthBearerTest { +public class HttpJwtRetrieverTest 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); + String response = HttpJwtRetriever.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)); + assertThrows(IOException.class, () -> HttpJwtRetriever.post(mockedCon, null, null, null, null)); } @Test @@ -60,7 +60,7 @@ 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)); + assertThrows(IOException.class, () -> HttpJwtRetriever.post(mockedCon, null, null, null, null)); } @Test @@ -72,7 +72,7 @@ public void testErrorResponseUnretryableCode() throws IOException { .getBytes(StandardCharsets.UTF_8))); when(mockedCon.getResponseCode()).thenReturn(HttpURLConnection.HTTP_BAD_REQUEST); UnretryableException ioe = assertThrows(UnretryableException.class, - () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + () -> HttpJwtRetriever.post(mockedCon, null, null, null, null)); assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}")); } @@ -85,7 +85,7 @@ public void testErrorResponseRetryableCode() throws IOException { .getBytes(StandardCharsets.UTF_8))); when(mockedCon.getResponseCode()).thenReturn(HttpURLConnection.HTTP_INTERNAL_ERROR); IOException ioe = assertThrows(IOException.class, - () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + () -> HttpJwtRetriever.post(mockedCon, null, null, null, null)); assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}")); // error response body has different keys @@ -93,7 +93,7 @@ public void testErrorResponseRetryableCode() throws IOException { "{\"errorCode\":\"some_arg\", \"errorSummary\":\"some problem with arg\"}" .getBytes(StandardCharsets.UTF_8))); ioe = assertThrows(IOException.class, - () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + () -> HttpJwtRetriever.post(mockedCon, null, null, null, null)); assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}")); // error response is valid json but unknown keys @@ -101,7 +101,7 @@ public void testErrorResponseRetryableCode() throws IOException { "{\"err\":\"some_arg\", \"err_des\":\"some problem with arg\"}" .getBytes(StandardCharsets.UTF_8))); ioe = assertThrows(IOException.class, - () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + () -> HttpJwtRetriever.post(mockedCon, null, null, null, null)); assertTrue(ioe.getMessage().contains("{\"err\":\"some_arg\", \"err_des\":\"some problem with arg\"}")); } @@ -113,7 +113,7 @@ public void testErrorResponseIsInvalidJson() throws IOException { "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)); + () -> HttpJwtRetriever.post(mockedCon, null, null, null, null)); assertTrue(ioe.getMessage().contains("{non json error output}")); } @@ -124,7 +124,7 @@ public void testCopy() throws IOException { r.nextBytes(expected); InputStream in = new ByteArrayInputStream(expected); ByteArrayOutputStream out = new ByteArrayOutputStream(); - HttpAccessTokenRetriever.copy(in, out); + HttpJwtRetriever.copy(in, out); assertArrayEquals(expected, out.toByteArray()); } @@ -133,7 +133,7 @@ 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)); + assertThrows(IOException.class, () -> HttpJwtRetriever.copy(mockedIn, out)); } @Test @@ -143,7 +143,7 @@ public void testParseAccessToken() throws IOException { ObjectNode node = mapper.createObjectNode(); node.put("access_token", expected); - String actual = HttpAccessTokenRetriever.parseAccessToken(mapper.writeValueAsString(node)); + String actual = HttpJwtRetriever.parseAccessToken(mapper.writeValueAsString(node)); assertEquals(expected, actual); } @@ -153,7 +153,7 @@ public void testParseAccessTokenEmptyAccessToken() { ObjectNode node = mapper.createObjectNode(); node.put("access_token", ""); - assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.parseAccessToken(mapper.writeValueAsString(node))); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.parseAccessToken(mapper.writeValueAsString(node))); } @Test @@ -162,12 +162,12 @@ public void testParseAccessTokenMissingAccessToken() { ObjectNode node = mapper.createObjectNode(); node.put("sub", "jdoe"); - assertThrows(IllegalArgumentException.class, () -> HttpAccessTokenRetriever.parseAccessToken(mapper.writeValueAsString(node))); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.parseAccessToken(mapper.writeValueAsString(node))); } @Test public void testParseAccessTokenInvalidJson() { - assertThrows(IOException.class, () -> HttpAccessTokenRetriever.parseAccessToken("not valid JSON")); + assertThrows(IOException.class, () -> HttpJwtRetriever.parseAccessToken("not valid JSON")); } @Test @@ -184,27 +184,27 @@ public void testFormatAuthorizationHeaderEncoding() { } private void assertAuthorizationHeader(String clientId, String clientSecret, boolean urlencode, String expected) { - String actual = HttpAccessTokenRetriever.formatAuthorizationHeader(clientId, clientSecret, urlencode); + String actual = HttpJwtRetriever.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)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader(null, "secret", false)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("id", null, false)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader(null, null, false)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("", "secret", false)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("id", "", false)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("", "", false)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader(" ", "secret", false)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader("id", " ", false)); + assertThrows(IllegalArgumentException.class, () -> HttpJwtRetriever.formatAuthorizationHeader(" ", " ", false)); } @Test public void testFormatRequestBody() { String expected = "grant_type=client_credentials&scope=scope"; - String actual = HttpAccessTokenRetriever.formatRequestBody("scope"); + String actual = HttpJwtRetriever.formatRequestBody("scope"); assertEquals(expected, actual); } @@ -214,24 +214,24 @@ public void testFormatRequestBodyWithEscaped() { String exclamationMark = "%21"; String expected = String.format("grant_type=client_credentials&scope=earth+is+great%s", exclamationMark); - String actual = HttpAccessTokenRetriever.formatRequestBody("earth is great!"); + String actual = HttpJwtRetriever.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?!?!?"); + actual = HttpJwtRetriever.formatRequestBody("what on earth?!?!?"); assertEquals(expected, actual); } @Test public void testFormatRequestBodyMissingValues() { String expected = "grant_type=client_credentials"; - String actual = HttpAccessTokenRetriever.formatRequestBody(null); + String actual = HttpJwtRetriever.formatRequestBody(null); assertEquals(expected, actual); - actual = HttpAccessTokenRetriever.formatRequestBody(""); + actual = HttpJwtRetriever.formatRequestBody(""); assertEquals(expected, actual); - actual = HttpAccessTokenRetriever.formatRequestBody(" "); + actual = HttpJwtRetriever.formatRequestBody(" "); assertEquals(expected, actual); } 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/JwtValidatorFactoryTest.java similarity index 65% rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactoryTest.java rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java index 2fd02e3f9a826..e1ae8c0d86d17 100644 --- 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/JwtValidatorFactoryTest.java @@ -20,21 +20,23 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler; +import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Map; -public class AccessTokenValidatorFactoryTest extends OAuthBearerTest { +public class JwtValidatorFactoryTest extends OAuthBearerTest { @Test - public void testConfigureThrowsExceptionOnAccessTokenValidatorInit() { + public void testConfigureThrowsExceptionOnJwtValidatorInit() throws IOException { OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() { + JwtRetriever jwtRetriever = new JwtRetriever() { @Override public void init() throws IOException { throw new IOException("My init had an error!"); } + @Override public String retrieve() { return "dummy"; @@ -42,16 +44,17 @@ public String retrieve() { }; Map configs = getSaslConfigs(); - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs); - assertThrowsWithMessage( - KafkaException.class, () -> handler.init(accessTokenRetriever, accessTokenValidator), "encountered an error when initializing"); + try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { + assertThrowsWithMessage( + KafkaException.class, () -> handler.init(jwtRetriever, jwtValidator), "encountered an error when initializing"); + } } @Test - public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() { + public void testConfigureThrowsExceptionOnJwtValidatorClose() throws IOException { OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() { + JwtRetriever jwtRetriever = new JwtRetriever() { @Override public void close() throws IOException { throw new IOException("My close had an error!"); @@ -63,11 +66,12 @@ public String retrieve() { }; Map configs = getSaslConfigs(); - AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs); - handler.init(accessTokenRetriever, accessTokenValidator); + try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { + handler.init(jwtRetriever, jwtValidator); - // Basically asserting this doesn't throw an exception :( - handler.close(); + // 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/AccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorTest.java similarity index 80% 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/internals/secured/JwtValidatorTest.java index 0adaf34bbbeea..bfbf29d0266fc 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/internals/secured/JwtValidatorTest.java @@ -26,42 +26,42 @@ 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 createJwtValidator(AccessTokenBuilder accessTokenBuilder) throws Exception; - protected AccessTokenValidator createAccessTokenValidator() throws Exception { + protected JwtValidator createJwtValidator() throws Exception { AccessTokenBuilder builder = new AccessTokenBuilder(); - return createAccessTokenValidator(builder); + return createJwtValidator(builder); } @Test public void testNull() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createJwtValidator(); assertThrowsWithMessage(ValidateException.class, () -> validator.validate(null), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test public void testEmptyString() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createJwtValidator(); assertThrowsWithMessage(ValidateException.class, () -> validator.validate(""), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test public void testWhitespace() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createJwtValidator(); assertThrowsWithMessage(ValidateException.class, () -> validator.validate(" "), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test public void testEmptySections() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createJwtValidator(); assertThrowsWithMessage(ValidateException.class, () -> validator.validate(".."), "Malformed JWT provided; expected three sections (header, payload, and signature)"); } @Test public void testMissingHeader() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createJwtValidator(); String header = ""; String payload = createBase64JsonJwtSection(node -> { }); String signature = ""; @@ -71,7 +71,7 @@ public void testMissingHeader() throws Exception { @Test public void testMissingPayload() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createJwtValidator(); String header = createBase64JsonJwtSection(node -> node.put(HeaderParameterNames.ALGORITHM, AlgorithmIdentifiers.NONE)); String payload = ""; String signature = ""; @@ -81,7 +81,7 @@ public void testMissingPayload() throws Exception { @Test public void testMissingSignature() throws Exception { - AccessTokenValidator validator = createAccessTokenValidator(); + JwtValidator validator = createJwtValidator(); String header = createBase64JsonJwtSection(node -> node.put(HeaderParameterNames.ALGORITHM, AlgorithmIdentifiers.NONE)); String payload = createBase64JsonJwtSection(node -> { }); String signature = ""; 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..9807a05738a82 100644 --- a/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java @@ -24,10 +24,11 @@ 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.OAuthBearerLoginModule; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; 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.utils.Exit; @@ -139,16 +140,19 @@ public static void main(String[] args) { { // Client side... - try (AccessTokenRetriever atr = AccessTokenRetrieverFactory.create(configs, jaasConfigs)) { + try (JwtRetriever atr = new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, 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"); + try (JwtValidator atv = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { + atv.init(); + System.out.println("PASSED 1/5: client configuration"); - atv.validate(accessToken); - System.out.println("PASSED 3/5: client JWT validation"); + accessToken = atr.retrieve(); + System.out.println("PASSED 2/5: client JWT retrieval"); + + atv.validate(accessToken); + System.out.println("PASSED 3/5: client JWT validation"); + } } } @@ -156,11 +160,14 @@ public static void main(String[] args) { // 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"); - atv.validate(accessToken); - System.out.println("PASSED 5/5: broker JWT validation"); + try (JwtValidator atv = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, vkr)) { + atv.init(); + System.out.println("PASSED 4/5: broker configuration"); + + atv.validate(accessToken); + System.out.println("PASSED 5/5: broker JWT validation"); + } } } From 088d49b7b3d00f183461b2d10ec857e07e78a914 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 2 May 2025 11:10:41 -0700 Subject: [PATCH 02/27] Updates from linter --- .../oauthbearer/OAuthBearerLoginCallbackHandler.java | 6 +++--- .../oauthbearer/OAuthBearerValidatorCallbackHandler.java | 6 +++--- .../oauthbearer/internals/secured/DefaultJwtValidator.java | 1 + .../oauthbearer/OAuthBearerLoginCallbackHandlerTest.java | 4 ++-- .../OAuthBearerValidatorCallbackHandlerTest.java | 4 ++-- .../internals/secured/DefaultJwtRetrieverTest.java | 2 +- .../internals/secured/JwtValidatorFactoryTest.java | 2 +- 7 files changed, 13 insertions(+), 12 deletions(-) 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 0a7a148521177..70f88dde62dfc 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 @@ -24,14 +24,14 @@ 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.JwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; 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; 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 ec56fb93d8c25..bc3a3e89125cc 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 @@ -19,15 +19,15 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; 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; diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java index 56c9238a5ccb3..f6d183e67ad3b 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken; import org.apache.kafka.common.utils.Utils; + import org.jose4j.keys.resolvers.VerificationKeyResolver; import java.io.IOException; 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 f2fa6e1aaa25c..81ff4452d91d5 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 @@ -22,11 +22,11 @@ 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.DefaultJwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.FileJwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpJwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.apache.kafka.common.utils.Utils; 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 e4702cd80ce2f..3ed689d26a6ea 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java @@ -18,9 +18,9 @@ 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.JwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.apache.kafka.common.utils.Utils; diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java index 59db66c67cc86..11b45b6fc2264 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java @@ -18,8 +18,8 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; import org.apache.kafka.common.config.ConfigException; - import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java index e1ae8c0d86d17..62bf1160e3f56 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler; - import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; + import org.junit.jupiter.api.Test; import java.io.IOException; From aa47fe3ac2f4b19e1bf28d65cf8ed06a350ef4e0 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 2 May 2025 13:25:45 -0700 Subject: [PATCH 03/27] Update OAuthCompatibilityTool to fix import ordering problems caught by linter --- .../java/org/apache/kafka/tools/OAuthCompatibilityTool.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 9807a05738a82..7852c3a07e084 100644 --- a/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTool.java @@ -25,11 +25,11 @@ import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.VerificationKeyResolverFactory; import org.apache.kafka.common.utils.Exit; From c45219b6dd1d964b1867807bc8608b48546196e3 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 15:40:42 -0700 Subject: [PATCH 04/27] Changed wording in Initable's init() Javadoc --- .../security/oauthbearer/internals/secured/Initable.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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/Initable.java index 0a38f2b5094d5..eff1b5438860c 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/Initable.java @@ -22,8 +22,8 @@ public interface Initable { /** - * 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. + * Lifecycle method to perform any one-time initialization of a given resource. This must + * be invoked by the caller to ensure the correct state before methods are invoked. * * @throws IOException Thrown on errors related to IO during initialization */ @@ -31,5 +31,4 @@ public interface Initable { default void init() throws IOException { // This method left intentionally blank. } - } From 3d1a2badb4da262ced78ef910938d6e009517804 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 15:41:06 -0700 Subject: [PATCH 05/27] Removed ambiguity in OAuthBearerValidatorCallbackHandler's log messages --- .../oauthbearer/OAuthBearerValidatorCallbackHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 bc3a3e89125cc..46738fc0fa338 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 @@ -147,13 +147,13 @@ public void init(CloseableVerificationKeyResolver verificationKeyResolver, JwtVa try { this.jwtValidator.init(); } catch (IOException e) { - throw new KafkaException("The OAuth validator configuration encountered an error when initializing the JwtValidator", e); + throw new KafkaException("The OAuth validator callback encountered an error when initializing the JwtValidator", e); } try { verificationKeyResolver.init(); } catch (Exception e) { - throw new KafkaException("The OAuth validator configuration encountered an error when initializing the VerificationKeyResolver", e); + throw new KafkaException("The OAuth validator callback encountered an error when initializing the VerificationKeyResolver", e); } isInitialized = true; From 5cb1de3dbaf72624d31d41a381d240420e72cf7d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 15:41:22 -0700 Subject: [PATCH 06/27] Removed superfluous no-op init() overrides --- .../oauthbearer/internals/secured/BrokerJwtValidator.java | 5 ----- .../oauthbearer/internals/secured/ClientJwtValidator.java | 5 ----- 2 files changed, 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java index ac481a75129ac..66c49fa24aa3f 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java @@ -191,11 +191,6 @@ else if (scopeRaw instanceof Collection) issuedAt); } - @Override - public void init() throws IOException { - // Do nothing... - } - @Override public void close() throws IOException { // Do nothing... diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java index 60f3743ae8089..5bc89f26e5765 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java @@ -124,11 +124,6 @@ else if (scopeRaw instanceof Collection) issuedAt); } - @Override - public void init() throws IOException { - // Do nothing... - } - @Override public void close() throws IOException { // Do nothing... From 84e21293f21dce77a36bd9da988742535673ea8d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 15:42:47 -0700 Subject: [PATCH 07/27] Fixed inconsistent Javadoc formatting in ClientJwtValidator --- .../oauthbearer/internals/secured/ClientJwtValidator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java index 5bc89f26e5765..a2cbd650d9c08 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java @@ -47,7 +47,7 @@ * 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, subject, and iat claims
  • * */ From 35d345424aaa3bbe48d81bf6a17c463ceb185cdc Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 15:44:59 -0700 Subject: [PATCH 08/27] Created a default, no-op close() implementation and removed the no-op close() methods from BrokerJwtValidator and ClientJwtValidator --- .../oauthbearer/internals/secured/BrokerJwtValidator.java | 5 ----- .../oauthbearer/internals/secured/ClientJwtValidator.java | 5 ----- .../oauthbearer/internals/secured/JwtValidator.java | 7 +++++++ 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java index 66c49fa24aa3f..63582f4a0485b 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java @@ -191,11 +191,6 @@ else if (scopeRaw instanceof Collection) issuedAt); } - @Override - public void close() throws IOException { - // Do nothing... - } - private T getClaim(ClaimSupplier supplier, String claimName) throws ValidateException { try { T value = supplier.get(); diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java index a2cbd650d9c08..70e748a925f4f 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java @@ -124,11 +124,6 @@ else if (scopeRaw instanceof Collection) issuedAt); } - @Override - public void close() throws IOException { - // Do nothing... - } - private Object getClaim(Map payload, String claimName) { Object value = payload.get(claimName); log.debug("getClaim - {}: {}", claimName, value); diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidator.java index cb6698e09aefd..82ba10652a140 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidator.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken; import java.io.Closeable; +import java.io.IOException; /** * An instance of JwtValidator acts as a function object that, given an access @@ -62,4 +63,10 @@ public interface JwtValidator extends Initable, Closeable { OAuthBearerToken validate(String accessToken) throws ValidateException; + /** + * Closes any resources that were initialized by {@link #init()}. + */ + default void close() throws IOException { + // Do nothing... + } } From f298a16b49a2e426bebe234685a1ccff2cb66d45 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 15:47:09 -0700 Subject: [PATCH 09/27] =?UTF-8?q?Updated=20validateUrlencodeHeader=20to=20?= =?UTF-8?q?use=20ConfigurationUtils=E2=80=99=20get()=20method=20instead=20?= =?UTF-8?q?of=20the=20clumsier=20validateBoolean()=20method?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../oauthbearer/internals/secured/DefaultJwtRetriever.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java index 51f1042be83ed..ef40af79cfbe6 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java @@ -109,7 +109,7 @@ public void close() throws IOException { * {@link HttpJwtRetriever} constructor. */ static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) { - Boolean urlencodeHeader = configurationUtils.validateBoolean(SASL_OAUTHBEARER_HEADER_URLENCODE, false); + Boolean urlencodeHeader = configurationUtils.get(SASL_OAUTHBEARER_HEADER_URLENCODE); if (urlencodeHeader != null) return urlencodeHeader; From e5e97c82a4b8e63b50c109fa2185439af3190912 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 16:02:14 -0700 Subject: [PATCH 10/27] Added clarifying Javadoc for DefaultJwtValidator --- .../oauthbearer/internals/secured/DefaultJwtValidator.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java index f6d183e67ad3b..0fa9fafea48df 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java @@ -35,6 +35,12 @@ 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; +/** + * This {@link JwtValidator} uses the delegation approach, instantiating and delegating calls to a + * more concrete implementation. The underlying implementation is determined by the presence/absence + * of the {@link VerificationKeyResolver}: if it's present, a {@link BrokerJwtValidator} is + * created, otherwise a {@link ClientJwtValidator} is created. + */ public class DefaultJwtValidator implements JwtValidator { private final Map configs; From 08ec47d03c48446438f7d885ae6138fb661cd873 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 16:02:55 -0700 Subject: [PATCH 11/27] Fixed a missing phrase in the FileJwtRetriever documentation. --- .../oauthbearer/internals/secured/FileJwtRetriever.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java index 35cd7cb9fb16f..f04b560016898 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileJwtRetriever.java @@ -23,8 +23,8 @@ import java.nio.file.Path; /** - * FileJwtRetriever is an {@link JwtRetriever} that will load the contents, - * interpreting them as a JWT access key in the serialized form. + * FileJwtRetriever is an {@link JwtRetriever} that will load the contents + * of a file, interpreting them as a JWT access key in the serialized form. * * @see JwtRetriever */ From bddc0e23622d01a8a4930af52e0715e3b3118ac5 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 16:04:01 -0700 Subject: [PATCH 12/27] HttpJwtRetriever Javadoc cleanup --- .../internals/secured/HttpJwtRetriever.java | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java index ce440834ef2ba..35d25564bc0ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpJwtRetriever.java @@ -49,19 +49,11 @@ import javax.net.ssl.SSLSocketFactory; /** - * HttpJwtRetriever is an {@link JwtRetriever} that will - * communicate with an OAuth/OIDC provider directly via HTTP to post client credentials + * HttpJwtRetriever is a {@link JwtRetriever} 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 JwtRetriever - * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG - * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG - * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG - * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL + * to a publicized token endpoint URL ({@link SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL}). */ - public class HttpJwtRetriever implements JwtRetriever { private static final Logger log = LoggerFactory.getLogger(HttpJwtRetriever.class); From 4c48438e8b25db1b61e067e32f4f683d666ad36c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 16:10:32 -0700 Subject: [PATCH 13/27] Moved tests from JwtValidatorFactoryTest to OAuthBearerValidatorCallbackHandlerTest --- ...uthBearerValidatorCallbackHandlerTest.java | 49 ++++++++++++ .../secured/JwtValidatorFactoryTest.java | 77 ------------------- 2 files changed, 49 insertions(+), 77 deletions(-) delete mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java 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 3ed689d26a6ea..f17a76cf56a59 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,9 +17,11 @@ package org.apache.kafka.common.security.oauthbearer; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder; import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; +import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.apache.kafka.common.utils.Utils; @@ -27,6 +29,7 @@ import org.jose4j.jws.AlgorithmIdentifiers; import org.junit.jupiter.api.Test; +import java.io.IOException; import java.util.Arrays; import java.util.Base64; import java.util.List; @@ -81,6 +84,52 @@ public void testInvalidAccessToken() throws Exception { assertInvalidAccessTokenFails(createAccessKey("{}", "{}", "{}"), substring); } + @Test + public void testConfigureThrowsExceptionOnJwtValidatorInit() throws IOException { + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + JwtRetriever jwtRetriever = new JwtRetriever() { + @Override + public void init() throws IOException { + throw new IOException("My init had an error!"); + } + + @Override + public String retrieve() { + return "dummy"; + } + }; + + Map configs = getSaslConfigs(); + + try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { + assertThrowsWithMessage( + KafkaException.class, () -> handler.init(jwtRetriever, jwtValidator), "encountered an error when initializing"); + } + } + + @Test + public void testConfigureThrowsExceptionOnJwtValidatorClose() throws IOException { + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + JwtRetriever jwtRetriever = new JwtRetriever() { + @Override + public void close() throws IOException { + throw new IOException("My close had an error!"); + } + @Override + public String retrieve() { + return "dummy"; + } + }; + + Map configs = getSaslConfigs(); + try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { + handler.init(jwtRetriever, jwtValidator); + + // Basically asserting this doesn't throw an exception :( + handler.close(); + } + } + private void assertInvalidAccessTokenFails(String accessToken, String expectedMessageSubstring) throws Exception { Map configs = getSaslConfigs(); OAuthBearerValidatorCallbackHandler handler = createHandler(configs, new AccessTokenBuilder()); diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java deleted file mode 100644 index 62bf1160e3f56..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtValidatorFactoryTest.java +++ /dev/null @@ -1,77 +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.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; - -import org.junit.jupiter.api.Test; - -import java.io.IOException; -import java.util.Map; - -public class JwtValidatorFactoryTest extends OAuthBearerTest { - - @Test - public void testConfigureThrowsExceptionOnJwtValidatorInit() throws IOException { - OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - JwtRetriever jwtRetriever = new JwtRetriever() { - @Override - public void init() throws IOException { - throw new IOException("My init had an error!"); - } - - @Override - public String retrieve() { - return "dummy"; - } - }; - - Map configs = getSaslConfigs(); - - try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { - assertThrowsWithMessage( - KafkaException.class, () -> handler.init(jwtRetriever, jwtValidator), "encountered an error when initializing"); - } - } - - @Test - public void testConfigureThrowsExceptionOnJwtValidatorClose() throws IOException { - OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - JwtRetriever jwtRetriever = new JwtRetriever() { - @Override - public void close() throws IOException { - throw new IOException("My close had an error!"); - } - @Override - public String retrieve() { - return "dummy"; - } - }; - - Map configs = getSaslConfigs(); - try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { - handler.init(jwtRetriever, jwtValidator); - - // Basically asserting this doesn't throw an exception :( - handler.close(); - } - } - -} From 9882fbec15b338934b787de9b38936c695ad6f76 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 6 May 2025 16:10:48 -0700 Subject: [PATCH 14/27] Fixed import linting issues --- .../oauthbearer/internals/secured/BrokerJwtValidator.java | 1 - .../oauthbearer/internals/secured/ClientJwtValidator.java | 1 - 2 files changed, 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java index 63582f4a0485b..74ad476522297 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BrokerJwtValidator.java @@ -31,7 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Set; diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java index 70e748a925f4f..1dee4671d3937 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientJwtValidator.java @@ -24,7 +24,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Map; From 395d779c4e4541083ab3fad96f7c78d1bc503b84 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 7 May 2025 11:05:06 -0700 Subject: [PATCH 15/27] Update clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java Co-authored-by: Ken Huang --- .../internals/secured/VerificationKeyResolverFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index c4cce0f7e8b38..c9ad41d5a97e6 100644 --- 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 @@ -37,7 +37,7 @@ public class VerificationKeyResolverFactory { /** - * Create an {@link JwtRetriever} from the given + * Create a {@link JwtRetriever} from the given * {@link org.apache.kafka.common.config.SaslConfigs}. * * Note: the returned CloseableVerificationKeyResolver is not From dbb4ac9ff7849b1d4be8dad9c5abde400b803341 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 7 May 2025 11:05:16 -0700 Subject: [PATCH 16/27] Update clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java Co-authored-by: Ken Huang --- .../oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 f17a76cf56a59..3e246186bc819 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 @@ -125,8 +125,7 @@ public String retrieve() { try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { handler.init(jwtRetriever, jwtValidator); - // Basically asserting this doesn't throw an exception :( - handler.close(); + assertDoesNotThrow(handler::close); } } From 54e6f69577f802fcc07d05603046c56047318c7d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 7 May 2025 11:48:34 -0700 Subject: [PATCH 17/27] Update OAuthBearerValidatorCallbackHandlerTest.java Fixed missing import from hastily taking GitHub PR suggestion --- .../oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 3e246186bc819..20094537eb1d5 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 @@ -38,6 +38,7 @@ import javax.security.auth.callback.Callback; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -124,7 +125,6 @@ public String retrieve() { Map configs = getSaslConfigs(); try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { handler.init(jwtRetriever, jwtValidator); - assertDoesNotThrow(handler::close); } } From e95660e726394640d2e1effebc5632cde5fb8e49 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 7 May 2025 11:55:28 -0700 Subject: [PATCH 18/27] Clean up awkward code for expected audience configuration in DefaultJwtValidator.init() --- .../internals/secured/DefaultJwtValidator.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java index 0fa9fafea48df..17873f4e69574 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java @@ -68,12 +68,8 @@ public void init() throws IOException { ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); if (verificationKeyResolver.isPresent()) { - Set expectedAudiences = null; - List l = cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE); - - if (l != null) - expectedAudiences = Set.copyOf(l); - + List expectedAudiencesList = cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE); + Set expectedAudiences = expectedAudiencesList != null ? Set.copyOf(expectedAudiencesList) : null; 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); From df5239335019dfea827a0697dc725b45d0015d63 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 7 May 2025 18:09:00 -0700 Subject: [PATCH 19/27] Updates to remove the init() methods from OAuthBearerLoginCallbackHandler and OAuthBearerValidatorCallbackHandler --- .../OAuthBearerLoginCallbackHandler.java | 45 +++----- .../OAuthBearerValidatorCallbackHandler.java | 41 +++---- .../OAuthBearerLoginCallbackHandlerTest.java | 101 +++++++++++++----- ...uthBearerValidatorCallbackHandlerTest.java | 93 +++++++++++----- .../internals/secured/OAuthBearerTest.java | 8 +- 5 files changed, 176 insertions(+), 112 deletions(-) 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 70f88dde62dfc..32ca739cdac60 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 @@ -178,47 +178,30 @@ public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHand private static final String EXTENSION_PREFIX = "extension_"; - private Map moduleOptions; + protected Map moduleOptions; - private JwtRetriever jwtRetriever; + protected JwtRetriever jwtRetriever; - private JwtValidator jwtValidator; - - private boolean isInitialized = false; + protected JwtValidator jwtValidator; @Override public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); - JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, saslMechanism, moduleOptions); - JwtValidator jwtValidator = new DefaultJwtValidator(configs, saslMechanism); - init(jwtRetriever, jwtValidator); - } - - public void init(JwtRetriever jwtRetriever, JwtValidator jwtValidator) { - this.jwtRetriever = jwtRetriever; - this.jwtValidator = jwtValidator; + jwtRetriever = new DefaultJwtRetriever(configs, saslMechanism, moduleOptions); try { this.jwtRetriever.init(); } catch (IOException e) { - throw new KafkaException("The OAuth login configuration encountered an error when initializing the JwtRetriever", e); + throw new KafkaException("The OAuth login callback encountered an error when initializing the JwtRetriever", e); } + jwtValidator = new DefaultJwtValidator(configs, saslMechanism); + try { this.jwtValidator.init(); } catch (IOException e) { - throw new KafkaException("The OAuth login configuration encountered an error when initializing the JwtValidator", e); + throw new KafkaException("The OAuth login callback encountered an error when initializing the JwtValidator", e); } - - isInitialized = true; - } - - /* - * Package-visible for testing. - */ - - JwtRetriever jwtRetriever() { - return jwtRetriever; } @Override @@ -229,7 +212,7 @@ public void close() { @Override public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { - checkInitialized(); + checkConfigured(); for (Callback callback : callbacks) { if (callback instanceof OAuthBearerTokenCallback) { @@ -243,7 +226,7 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback } private void handleTokenCallback(OAuthBearerTokenCallback callback) throws IOException { - checkInitialized(); + checkConfigured(); String accessToken = jwtRetriever.retrieve(); try { @@ -256,7 +239,7 @@ private void handleTokenCallback(OAuthBearerTokenCallback callback) throws IOExc } private void handleExtensionsCallback(SaslExtensionsCallback callback) { - checkInitialized(); + checkConfigured(); Map extensions = new HashMap<>(); @@ -288,9 +271,9 @@ private void handleExtensionsCallback(SaslExtensionsCallback callback) { callback.extensions(saslExtensions); } - private void checkInitialized() { - if (!isInitialized) - throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName())); + private void checkConfigured() { + if (moduleOptions == null || jwtRetriever == null || jwtValidator == null) + 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 46738fc0fa338..5c653bcb0c876 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 @@ -118,16 +118,13 @@ public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallback private static final Map VERIFICATION_KEY_RESOLVER_CACHE = new HashMap<>(); - private CloseableVerificationKeyResolver verificationKeyResolver; + protected CloseableVerificationKeyResolver verificationKeyResolver; - private JwtValidator jwtValidator; - - private boolean isInitialized = false; + protected JwtValidator jwtValidator; @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) { @@ -136,27 +133,19 @@ public void configure(Map configs, String saslMechanism, List extensionsValidatorCallback.valid(extensionName)); } - private void checkInitialized() { - if (!isInitialized) - throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName())); + private void checkConfigured() { + if (verificationKeyResolver == null || jwtValidator == null) + throw new IllegalStateException(String.format("To use %s, first call the configure method", getClass().getSimpleName())); } /** 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 81ff4452d91d5..f73612d5e2fd1 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,6 +17,7 @@ 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; @@ -38,13 +39,14 @@ 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 javax.security.auth.login.AppConfigurationEntry; 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; @@ -59,6 +61,7 @@ import static org.junit.jupiter.api.Assertions.fail; public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { + @AfterEach public void tearDown() throws Exception { System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG); @@ -72,8 +75,9 @@ public void testHandleTokenCallback() throws Exception { .alg(AlgorithmIdentifiers.RSA_USING_SHA256); String accessToken = builder.build(); JwtRetriever jwtRetriever = () -> accessToken; - - OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever, configs); + JwtValidator jwtValidator = createJwtValidator(configs); + OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); + configureHandler(handler, configs); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -92,7 +96,6 @@ public void testHandleTokenCallback() throws Exception { @Test public void testHandleSaslExtensionsCallback() throws Exception { - 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 jaasConfig = new HashMap<>(); @@ -101,6 +104,8 @@ public void testHandleSaslExtensionsCallback() throws Exception { jaasConfig.put("extension_foo", "1"); jaasConfig.put("extension_bar", 2); jaasConfig.put("EXTENSION_baz", "3"); + + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); configureHandler(handler, configs, jaasConfig); try { @@ -122,13 +127,14 @@ public void testHandleSaslExtensionsCallback() throws Exception { public void testHandleSaslExtensionsCallbackWithInvalidExtension() { String illegalKey = "extension_" + OAuthBearerClientInitialResponse.AUTH_KEY; - 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 jaasConfig = new HashMap<>(); jaasConfig.put(CLIENT_ID_CONFIG, "an ID"); jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret"); jaasConfig.put(illegalKey, "this key isn't allowed per OAuthBearerClientInitialResponse.validateExtensions"); + + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); configureHandler(handler, configs, jaasConfig); try { @@ -144,10 +150,10 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() { @Test public void testInvalidCallbackGeneratesUnsupportedCallbackException() { Map configs = getSaslConfigs(); - OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - JwtRetriever jwtRetriever = () -> "foo"; - JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM); - handler.init(jwtRetriever, jwtValidator); + JwtRetriever jwtRetriever = () -> "test"; + JwtValidator jwtValidator = createJwtValidator(configs); + OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); + configureHandler(handler, configs); try { Callback unsupportedCallback = new Callback() { }; @@ -167,11 +173,13 @@ public void testInvalidAccessToken() throws Exception { @Test public void testMissingAccessToken() { + Map configs = getSaslConfigs(); JwtRetriever jwtRetriever = () -> { throw new IOException("The token endpoint response access_token value must be non-null"); }; - Map configs = getSaslConfigs(); - OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever, configs); + JwtValidator jwtValidator = createJwtValidator(configs); + OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); + configureHandler(handler, configs); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -197,7 +205,11 @@ public void testFileTokenRetrieverHandlesNewline() throws IOException { File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", withNewline); Map configs = getSaslConfigs(); - OAuthBearerLoginCallbackHandler handler = createHandler(new FileJwtRetriever(accessTokenFile.toPath()), configs); + JwtRetriever jwtRetriever = new FileJwtRetriever(accessTokenFile.toPath()); + JwtValidator jwtValidator = createJwtValidator(configs); + OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); + configureHandler(handler, configs); + OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); try { handler.handle(new Callback[]{callback}); @@ -212,7 +224,7 @@ public void testFileTokenRetrieverHandlesNewline() throws IOException { @Test public void testNotConfigured() { OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure or init method"); + assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure method"); } @Test @@ -223,30 +235,37 @@ public void testConfigureWithAccessTokenFile() throws Exception { File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", expected); System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.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); - DefaultJwtRetriever defaultJwtRetriever = assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever()); + TestOAuthBearerLogicCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(configs); + configureHandler(handler, configs); + + DefaultJwtRetriever defaultJwtRetriever = assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever); assertInstanceOf(FileJwtRetriever.class, defaultJwtRetriever.delegate()); } @Test 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"); + + DefaultJwtRetriever jwtRetriever = createJwtRetriever(configs, jaasConfigs); + JwtValidator jwtValidator = createJwtValidator(configs); + TestOAuthBearerLogicCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); configureHandler(handler, configs, jaasConfigs); - DefaultJwtRetriever defaultJwtRetriever = assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever()); + + DefaultJwtRetriever defaultJwtRetriever = assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever); assertInstanceOf(HttpJwtRetriever.class, defaultJwtRetriever.delegate()); } private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception { Map configs = getSaslConfigs(); - OAuthBearerLoginCallbackHandler handler = createHandler(() -> accessToken, configs); + JwtRetriever jwtRetriever = () -> accessToken; + JwtValidator jwtValidator = createJwtValidator(configs); + OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); + configureHandler(handler, configs); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -271,11 +290,43 @@ private String createAccessKey(String header, String payload, String signature) return String.format("%s.%s.%s", header, payload, signature); } - private OAuthBearerLoginCallbackHandler createHandler(JwtRetriever jwtRetriever, Map configs) { - OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM); - handler.init(jwtRetriever, jwtValidator); - return handler; + private static DefaultJwtRetriever createJwtRetriever(Map configs) { + return createJwtRetriever(configs, Map.of()); + } + + private static DefaultJwtRetriever createJwtRetriever(Map configs, Map jaasConfigs) { + return new DefaultJwtRetriever(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, jaasConfigs); + } + + private static DefaultJwtValidator createJwtValidator(Map configs) { + return new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM); } + private static class TestOAuthBearerLogicCallbackHandler extends OAuthBearerLoginCallbackHandler { + + public TestOAuthBearerLogicCallbackHandler(Map configs) { + this(createJwtRetriever(configs), createJwtValidator(configs)); + } + + public TestOAuthBearerLogicCallbackHandler(JwtRetriever jwtRetriever, JwtValidator jwtValidator) { + this.moduleOptions = Map.of(); + this.jwtRetriever = jwtRetriever; + this.jwtValidator = jwtValidator; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + try { + jwtRetriever.init(); + } catch (Exception e) { + throw new KafkaException("The OAuth login callback encountered an error when initializing the VerificationKeyResolver", e); + } + + try { + jwtValidator.init(); + } catch (IOException e) { + throw new KafkaException("The OAuth login callback encountered an error when initializing the JwtValidator", e); + } + } + } } 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 20094537eb1d5..111dcbfa7d815 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 @@ -21,9 +21,9 @@ import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder; import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver; import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; -import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; +import org.apache.kafka.common.security.oauthbearer.internals.secured.ValidateException; import org.apache.kafka.common.utils.Utils; import org.jose4j.jws.AlgorithmIdentifiers; @@ -36,6 +36,7 @@ import java.util.Map; import javax.security.auth.callback.Callback; +import javax.security.auth.login.AppConfigurationEntry; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -57,7 +58,10 @@ public void testBasic() throws Exception { String accessToken = builder.build(); Map configs = getSaslConfigs(SASL_OAUTHBEARER_EXPECTED_AUDIENCE, allAudiences); - OAuthBearerValidatorCallbackHandler handler = createHandler(configs, builder); + CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); + JwtValidator jwtValidator = createJwtValidator(configs, verificationKeyResolver); + OAuthBearerValidatorCallbackHandler handler = new TestOAuthBearerValidatorCallbackHandler(verificationKeyResolver, jwtValidator); + configureHandler(handler, configs); try { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); @@ -87,51 +91,63 @@ public void testInvalidAccessToken() throws Exception { @Test public void testConfigureThrowsExceptionOnJwtValidatorInit() throws IOException { - OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - JwtRetriever jwtRetriever = new JwtRetriever() { + JwtValidator jwtValidator = new JwtValidator() { @Override public void init() throws IOException { throw new IOException("My init had an error!"); } @Override - public String retrieve() { - return "dummy"; + public OAuthBearerToken validate(String accessToken) throws ValidateException { + return null; } }; + AccessTokenBuilder builder = new AccessTokenBuilder() + .alg(AlgorithmIdentifiers.RSA_USING_SHA256); Map configs = getSaslConfigs(); - - try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { - assertThrowsWithMessage( - KafkaException.class, () -> handler.init(jwtRetriever, jwtValidator), "encountered an error when initializing"); - } + CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); + OAuthBearerValidatorCallbackHandler handler = new TestOAuthBearerValidatorCallbackHandler(verificationKeyResolver, jwtValidator); + + assertThrowsWithMessage( + KafkaException.class, + () -> configureHandler(handler, configs), + "encountered an error when initializing" + ); } @Test public void testConfigureThrowsExceptionOnJwtValidatorClose() throws IOException { - OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - JwtRetriever jwtRetriever = new JwtRetriever() { + JwtValidator jwtValidator = new JwtValidator() { @Override public void close() throws IOException { throw new IOException("My close had an error!"); } + @Override - public String retrieve() { - return "dummy"; + public OAuthBearerToken validate(String accessToken) throws ValidateException { + return null; } }; + AccessTokenBuilder builder = new AccessTokenBuilder() + .alg(AlgorithmIdentifiers.RSA_USING_SHA256); Map configs = getSaslConfigs(); - try (JwtValidator jwtValidator = new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) { - handler.init(jwtRetriever, jwtValidator); - assertDoesNotThrow(handler::close); - } + CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); + OAuthBearerValidatorCallbackHandler handler = new TestOAuthBearerValidatorCallbackHandler(verificationKeyResolver, jwtValidator); + configureHandler(handler, configs); + + assertDoesNotThrow(handler::close); } private void assertInvalidAccessTokenFails(String accessToken, String expectedMessageSubstring) throws Exception { + AccessTokenBuilder builder = new AccessTokenBuilder() + .alg(AlgorithmIdentifiers.RSA_USING_SHA256); Map configs = getSaslConfigs(); - OAuthBearerValidatorCallbackHandler handler = createHandler(configs, new AccessTokenBuilder()); + CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); + JwtValidator jwtValidator = createJwtValidator(configs, verificationKeyResolver); + OAuthBearerValidatorCallbackHandler handler = new TestOAuthBearerValidatorCallbackHandler(verificationKeyResolver, jwtValidator); + configureHandler(handler, configs); try { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); @@ -146,14 +162,12 @@ private void assertInvalidAccessTokenFails(String accessToken, String expectedMe } } - private OAuthBearerValidatorCallbackHandler createHandler(Map options, - AccessTokenBuilder builder) { - OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); - CloseableVerificationKeyResolver verificationKeyResolver = (jws, nestingContext) -> - builder.jwk().getPublicKey(); - JwtValidator jwtValidator = new DefaultJwtValidator(options, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, verificationKeyResolver); - handler.init(verificationKeyResolver, jwtValidator); - return handler; + private JwtValidator createJwtValidator(Map configs, CloseableVerificationKeyResolver verificationKeyResolver) { + return new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, verificationKeyResolver); + } + + private CloseableVerificationKeyResolver createVerificationKeyResolver(AccessTokenBuilder builder) { + return (jws, nestingContext) -> builder.jwk().getPublicKey(); } private String createAccessKey(String header, String payload, String signature) { @@ -164,4 +178,27 @@ private String createAccessKey(String header, String payload, String signature) return String.format("%s.%s.%s", header, payload, signature); } + private static class TestOAuthBearerValidatorCallbackHandler extends OAuthBearerValidatorCallbackHandler { + + public TestOAuthBearerValidatorCallbackHandler(CloseableVerificationKeyResolver verificationKeyResolver, + JwtValidator jwtValidator) { + this.verificationKeyResolver = verificationKeyResolver; + this.jwtValidator = jwtValidator; + } + + @Override + public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { + try { + verificationKeyResolver.init(); + } catch (Exception e) { + throw new KafkaException("The OAuth validator callback encountered an error when initializing the VerificationKeyResolver", e); + } + + try { + jwtValidator.init(); + } catch (IOException e) { + throw new KafkaException("The OAuth validator callback encountered an error when initializing the JwtValidator", e); + } + } + } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java index 7f20b9464faea..25c6a4cfeb54e 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java @@ -80,9 +80,13 @@ protected void assertErrorMessageContains(String actual, String expectedSubstrin expectedSubstring)); } + protected void configureHandler(AuthenticateCallbackHandler handler, Map configs) { + configureHandler(handler, configs, Map.of()); + } + protected void configureHandler(AuthenticateCallbackHandler handler, - Map configs, - Map jaasConfig) { + Map configs, + Map jaasConfig) { TestJaasConfig config = new TestJaasConfig(); config.createOrUpdateEntry("KafkaClient", OAuthBearerLoginModule.class.getName(), jaasConfig); AppConfigurationEntry kafkaClient = config.getAppConfigurationEntry("KafkaClient")[0]; From 74fd8cd3034ef29b98a1dcb0bce682bd3b2e7530 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 7 May 2025 18:10:41 -0700 Subject: [PATCH 20/27] Moved createAccessKey to OAuthBearerTest for reuse --- .../OAuthBearerLoginCallbackHandlerTest.java | 10 ---------- .../OAuthBearerValidatorCallbackHandlerTest.java | 10 ---------- .../oauthbearer/internals/secured/OAuthBearerTest.java | 7 +++++++ 3 files changed, 7 insertions(+), 20 deletions(-) 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 f73612d5e2fd1..97a15380ff5d5 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 @@ -29,7 +29,6 @@ import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; -import org.apache.kafka.common.utils.Utils; import org.jose4j.jws.AlgorithmIdentifiers; import org.junit.jupiter.api.AfterEach; @@ -37,7 +36,6 @@ import java.io.File; import java.io.IOException; -import java.util.Base64; import java.util.Calendar; import java.util.HashMap; import java.util.List; @@ -282,14 +280,6 @@ private void testInvalidAccessToken(String accessToken, String expectedMessageSu } } - 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 static DefaultJwtRetriever createJwtRetriever(Map configs) { return createJwtRetriever(configs, Map.of()); } 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 111dcbfa7d815..b6850d33c8f53 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 @@ -24,14 +24,12 @@ import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; import org.apache.kafka.common.security.oauthbearer.internals.secured.ValidateException; -import org.apache.kafka.common.utils.Utils; import org.jose4j.jws.AlgorithmIdentifiers; import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Arrays; -import java.util.Base64; import java.util.List; import java.util.Map; @@ -170,14 +168,6 @@ private CloseableVerificationKeyResolver createVerificationKeyResolver(AccessTok return (jws, nestingContext) -> builder.jwk().getPublicKey(); } - 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 static class TestOAuthBearerValidatorCallbackHandler extends OAuthBearerValidatorCallbackHandler { public TestOAuthBearerValidatorCallbackHandler(CloseableVerificationKeyResolver verificationKeyResolver, diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java index 25c6a4cfeb54e..884ca9434b7d3 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java @@ -216,4 +216,11 @@ protected PublicJsonWebKey createEcJwk() throws JoseException { return jwk; } + 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); + } } \ No newline at end of file From 4fa46c52dea515a979d63c1f788d9d2e4c688f09 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 8 May 2025 11:29:38 -0700 Subject: [PATCH 21/27] Refactoring for unit tests to make less public API surface area --- .../OAuthBearerLoginCallbackHandler.java | 20 ++-- .../OAuthBearerValidatorCallbackHandler.java | 15 ++- .../secured/DefaultJwtRetriever.java | 8 +- .../secured/DefaultJwtValidator.java | 2 +- .../OAuthBearerLoginCallbackHandlerTest.java | 94 ++++--------------- ...uthBearerValidatorCallbackHandlerTest.java | 55 +++-------- .../secured/DefaultJwtRetrieverTest.java | 41 ++++++++ .../secured/DefaultJwtValidatorTest.java | 61 ++++++++++++ .../internals/secured/OAuthBearerTest.java | 21 ----- 9 files changed, 164 insertions(+), 153 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidatorTest.java 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 32ca739cdac60..4c6d52654e712 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 @@ -178,16 +178,24 @@ public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHand private static final String EXTENSION_PREFIX = "extension_"; - protected Map moduleOptions; + private Map moduleOptions; - protected JwtRetriever jwtRetriever; + private JwtRetriever jwtRetriever; - protected JwtValidator jwtValidator; + private JwtValidator jwtValidator; @Override public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { - moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); - jwtRetriever = new DefaultJwtRetriever(configs, saslMechanism, moduleOptions); + Map moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); + JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, saslMechanism, moduleOptions); + JwtValidator jwtValidator = new DefaultJwtValidator(configs, saslMechanism); + configure(moduleOptions, jwtRetriever, jwtValidator); + } + + void configure(Map moduleOptions, JwtRetriever jwtRetriever, JwtValidator jwtValidator) { + this.moduleOptions = moduleOptions; + this.jwtRetriever = jwtRetriever; + this.jwtValidator = jwtValidator; try { this.jwtRetriever.init(); @@ -195,8 +203,6 @@ public void configure(Map configs, String saslMechanism, List VERIFICATION_KEY_RESOLVER_CACHE = new HashMap<>(); - protected CloseableVerificationKeyResolver verificationKeyResolver; + private CloseableVerificationKeyResolver verificationKeyResolver; - protected JwtValidator jwtValidator; + private JwtValidator jwtValidator; @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) { @@ -133,14 +134,20 @@ public void configure(Map configs, String saslMechanism, List configs; @@ -117,7 +123,7 @@ static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) { return DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; } - public JwtRetriever delegate() { + JwtRetriever delegate() { return delegate; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java index 17873f4e69574..9febbacfd3d71 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java @@ -100,7 +100,7 @@ public void close() throws IOException { Utils.closeQuietly(delegate, "delegate"); } - public JwtValidator delegate() { + JwtValidator delegate() { return delegate; } } 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 97a15380ff5d5..1f1f379cff2db 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,7 +17,6 @@ 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; @@ -25,7 +24,6 @@ import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.DefaultJwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.FileJwtRetriever; -import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpJwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtRetriever; import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtValidator; import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest; @@ -38,20 +36,17 @@ import java.io.IOException; import java.util.Calendar; 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 javax.security.auth.login.AppConfigurationEntry; 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.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; @@ -74,8 +69,8 @@ public void testHandleTokenCallback() throws Exception { String accessToken = builder.build(); JwtRetriever jwtRetriever = () -> accessToken; JwtValidator jwtValidator = createJwtValidator(configs); - OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); - configureHandler(handler, configs); + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + handler.configure(Map.of(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -103,8 +98,10 @@ public void testHandleSaslExtensionsCallback() throws Exception { jaasConfig.put("extension_bar", 2); jaasConfig.put("EXTENSION_baz", "3"); + JwtRetriever jwtRetriever = createJwtRetriever(configs, jaasConfig); + JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - configureHandler(handler, configs, jaasConfig); + handler.configure(jaasConfig, jwtRetriever, jwtValidator); try { SaslExtensionsCallback callback = new SaslExtensionsCallback(); @@ -132,8 +129,10 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() { jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret"); jaasConfig.put(illegalKey, "this key isn't allowed per OAuthBearerClientInitialResponse.validateExtensions"); + JwtRetriever jwtRetriever = createJwtRetriever(configs, jaasConfig); + JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - configureHandler(handler, configs, jaasConfig); + handler.configure(jaasConfig, jwtRetriever, jwtValidator); try { SaslExtensionsCallback callback = new SaslExtensionsCallback(); @@ -150,8 +149,8 @@ public void testInvalidCallbackGeneratesUnsupportedCallbackException() { Map configs = getSaslConfigs(); JwtRetriever jwtRetriever = () -> "test"; JwtValidator jwtValidator = createJwtValidator(configs); - OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); - configureHandler(handler, configs); + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + handler.configure(Map.of(), jwtRetriever, jwtValidator); try { Callback unsupportedCallback = new Callback() { }; @@ -176,8 +175,8 @@ public void testMissingAccessToken() { throw new IOException("The token endpoint response access_token value must be non-null"); }; JwtValidator jwtValidator = createJwtValidator(configs); - OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); - configureHandler(handler, configs); + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + handler.configure(Map.of(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -205,8 +204,8 @@ public void testFileTokenRetrieverHandlesNewline() throws IOException { Map configs = getSaslConfigs(); JwtRetriever jwtRetriever = new FileJwtRetriever(accessTokenFile.toPath()); JwtValidator jwtValidator = createJwtValidator(configs); - OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); - configureHandler(handler, configs); + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + handler.configure(Map.of(), jwtRetriever, jwtValidator); OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); try { @@ -225,45 +224,12 @@ public void testNotConfigured() { assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure method"); } - @Test - public void testConfigureWithAccessTokenFile() 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 = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); - TestOAuthBearerLogicCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(configs); - configureHandler(handler, configs); - - DefaultJwtRetriever defaultJwtRetriever = assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever); - assertInstanceOf(FileJwtRetriever.class, defaultJwtRetriever.delegate()); - } - - @Test - public void testConfigureWithAccessClientCredentials() { - 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"); - - DefaultJwtRetriever jwtRetriever = createJwtRetriever(configs, jaasConfigs); - JwtValidator jwtValidator = createJwtValidator(configs); - TestOAuthBearerLogicCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); - configureHandler(handler, configs, jaasConfigs); - - DefaultJwtRetriever defaultJwtRetriever = assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever); - assertInstanceOf(HttpJwtRetriever.class, defaultJwtRetriever.delegate()); - } - private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception { Map configs = getSaslConfigs(); JwtRetriever jwtRetriever = () -> accessToken; JwtValidator jwtValidator = createJwtValidator(configs); - OAuthBearerLoginCallbackHandler handler = new TestOAuthBearerLogicCallbackHandler(jwtRetriever, jwtValidator); - configureHandler(handler, configs); + OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); + handler.configure(Map.of(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -291,32 +257,4 @@ private static DefaultJwtRetriever createJwtRetriever(Map configs, Ma private static DefaultJwtValidator createJwtValidator(Map configs) { return new DefaultJwtValidator(configs, OAuthBearerLoginModule.OAUTHBEARER_MECHANISM); } - - private static class TestOAuthBearerLogicCallbackHandler extends OAuthBearerLoginCallbackHandler { - - public TestOAuthBearerLogicCallbackHandler(Map configs) { - this(createJwtRetriever(configs), createJwtValidator(configs)); - } - - public TestOAuthBearerLogicCallbackHandler(JwtRetriever jwtRetriever, JwtValidator jwtValidator) { - this.moduleOptions = Map.of(); - this.jwtRetriever = jwtRetriever; - this.jwtValidator = jwtValidator; - } - - @Override - public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { - try { - jwtRetriever.init(); - } catch (Exception e) { - throw new KafkaException("The OAuth login callback encountered an error when initializing the VerificationKeyResolver", e); - } - - try { - jwtValidator.init(); - } catch (IOException e) { - throw new KafkaException("The OAuth login callback encountered an error when initializing the JwtValidator", e); - } - } - } } 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 b6850d33c8f53..a4ee40d15b348 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 @@ -34,7 +34,6 @@ import java.util.Map; import javax.security.auth.callback.Callback; -import javax.security.auth.login.AppConfigurationEntry; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -58,8 +57,8 @@ public void testBasic() throws Exception { Map configs = getSaslConfigs(SASL_OAUTHBEARER_EXPECTED_AUDIENCE, allAudiences); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); JwtValidator jwtValidator = createJwtValidator(configs, verificationKeyResolver); - OAuthBearerValidatorCallbackHandler handler = new TestOAuthBearerValidatorCallbackHandler(verificationKeyResolver, jwtValidator); - configureHandler(handler, configs); + OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); + handler.configure(verificationKeyResolver, jwtValidator); try { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); @@ -89,6 +88,9 @@ public void testInvalidAccessToken() throws Exception { @Test public void testConfigureThrowsExceptionOnJwtValidatorInit() throws IOException { + AccessTokenBuilder builder = new AccessTokenBuilder() + .alg(AlgorithmIdentifiers.RSA_USING_SHA256); + CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); JwtValidator jwtValidator = new JwtValidator() { @Override public void init() throws IOException { @@ -101,21 +103,20 @@ public OAuthBearerToken validate(String accessToken) throws ValidateException { } }; - AccessTokenBuilder builder = new AccessTokenBuilder() - .alg(AlgorithmIdentifiers.RSA_USING_SHA256); - Map configs = getSaslConfigs(); - CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); - OAuthBearerValidatorCallbackHandler handler = new TestOAuthBearerValidatorCallbackHandler(verificationKeyResolver, jwtValidator); + OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); assertThrowsWithMessage( KafkaException.class, - () -> configureHandler(handler, configs), + () -> handler.configure(verificationKeyResolver, jwtValidator), "encountered an error when initializing" ); } @Test public void testConfigureThrowsExceptionOnJwtValidatorClose() throws IOException { + AccessTokenBuilder builder = new AccessTokenBuilder() + .alg(AlgorithmIdentifiers.RSA_USING_SHA256); + CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); JwtValidator jwtValidator = new JwtValidator() { @Override public void close() throws IOException { @@ -128,12 +129,8 @@ public OAuthBearerToken validate(String accessToken) throws ValidateException { } }; - AccessTokenBuilder builder = new AccessTokenBuilder() - .alg(AlgorithmIdentifiers.RSA_USING_SHA256); - Map configs = getSaslConfigs(); - CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); - OAuthBearerValidatorCallbackHandler handler = new TestOAuthBearerValidatorCallbackHandler(verificationKeyResolver, jwtValidator); - configureHandler(handler, configs); + OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); + handler.configure(verificationKeyResolver, jwtValidator); assertDoesNotThrow(handler::close); } @@ -144,8 +141,8 @@ private void assertInvalidAccessTokenFails(String accessToken, String expectedMe Map configs = getSaslConfigs(); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); JwtValidator jwtValidator = createJwtValidator(configs, verificationKeyResolver); - OAuthBearerValidatorCallbackHandler handler = new TestOAuthBearerValidatorCallbackHandler(verificationKeyResolver, jwtValidator); - configureHandler(handler, configs); + OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); + handler.configure(verificationKeyResolver, jwtValidator); try { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); @@ -167,28 +164,4 @@ private JwtValidator createJwtValidator(Map configs, CloseableVerific private CloseableVerificationKeyResolver createVerificationKeyResolver(AccessTokenBuilder builder) { return (jws, nestingContext) -> builder.jwk().getPublicKey(); } - - private static class TestOAuthBearerValidatorCallbackHandler extends OAuthBearerValidatorCallbackHandler { - - public TestOAuthBearerValidatorCallbackHandler(CloseableVerificationKeyResolver verificationKeyResolver, - JwtValidator jwtValidator) { - this.verificationKeyResolver = verificationKeyResolver; - this.jwtValidator = jwtValidator; - } - - @Override - public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { - try { - verificationKeyResolver.init(); - } catch (Exception e) { - throw new KafkaException("The OAuth validator callback encountered an error when initializing the VerificationKeyResolver", e); - } - - try { - jwtValidator.init(); - } catch (IOException e) { - throw new KafkaException("The OAuth validator callback encountered an error when initializing the JwtValidator", e); - } - } - } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java index 11b45b6fc2264..83fd57713b00c 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetrieverTest.java @@ -29,6 +29,7 @@ import java.io.File; import java.io.IOException; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.stream.Stream; @@ -36,7 +37,11 @@ import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG; +import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; public class DefaultJwtRetrieverTest extends OAuthBearerTest { @@ -101,6 +106,42 @@ public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception { } } + @Test + public void testConfigureWithAccessTokenFile() 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 = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); + + DefaultJwtRetriever jwtRetriever = new DefaultJwtRetriever( + configs, + OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, + Map.of() + ); + assertDoesNotThrow(jwtRetriever::init); + assertInstanceOf(FileJwtRetriever.class, jwtRetriever.delegate()); + } + + @Test + public void testConfigureWithAccessClientCredentials() { + 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"); + + DefaultJwtRetriever jwtRetriever = new DefaultJwtRetriever( + configs, + OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, + jaasConfigs + ); + assertDoesNotThrow(jwtRetriever::init); + assertInstanceOf(HttpJwtRetriever.class, jwtRetriever.delegate()); + } + @ParameterizedTest @MethodSource("urlencodeHeaderSupplier") public void testUrlencodeHeader(Map configs, boolean expectedValue) { diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidatorTest.java new file mode 100644 index 0000000000000..9d136b72b1430 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidatorTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common.security.oauthbearer.internals.secured; + +import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; + +import org.jose4j.jws.AlgorithmIdentifiers; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; + +public class DefaultJwtValidatorTest extends OAuthBearerTest { + + @Test + public void testConfigureWithVerificationKeyResolver() { + AccessTokenBuilder builder = new AccessTokenBuilder() + .alg(AlgorithmIdentifiers.RSA_USING_SHA256); + CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); + Map configs = getSaslConfigs(); + DefaultJwtValidator jwtValidator = new DefaultJwtValidator( + configs, + OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, + verificationKeyResolver + ); + assertDoesNotThrow(jwtValidator::init); + assertInstanceOf(BrokerJwtValidator.class, jwtValidator.delegate()); + } + + @Test + public void testConfigureWithoutVerificationKeyResolver() { + Map configs = getSaslConfigs(); + DefaultJwtValidator jwtValidator = new DefaultJwtValidator( + configs, + OAuthBearerLoginModule.OAUTHBEARER_MECHANISM + ); + assertDoesNotThrow(jwtValidator::init); + assertInstanceOf(ClientJwtValidator.class, jwtValidator.delegate()); + } + + private CloseableVerificationKeyResolver createVerificationKeyResolver(AccessTokenBuilder builder) { + return (jws, nestingContext) -> builder.jwk().getPublicKey(); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java index 884ca9434b7d3..8e82092f28d9e 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java @@ -19,9 +19,6 @@ 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.Utils; import com.fasterxml.jackson.databind.ObjectMapper; @@ -52,8 +49,6 @@ import java.util.concurrent.ExecutionException; import java.util.function.Consumer; -import javax.security.auth.login.AppConfigurationEntry; - import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -80,22 +75,6 @@ protected void assertErrorMessageContains(String actual, String expectedSubstrin expectedSubstring)); } - protected void configureHandler(AuthenticateCallbackHandler handler, Map configs) { - configureHandler(handler, configs, Map.of()); - } - - protected void configureHandler(AuthenticateCallbackHandler handler, - Map configs, - Map jaasConfig) { - TestJaasConfig config = new TestJaasConfig(); - config.createOrUpdateEntry("KafkaClient", OAuthBearerLoginModule.class.getName(), jaasConfig); - AppConfigurationEntry kafkaClient = config.getAppConfigurationEntry("KafkaClient")[0]; - - handler.configure(configs, - OAuthBearerLoginModule.OAUTHBEARER_MECHANISM, - Collections.singletonList(kafkaClient)); - } - protected String createBase64JsonJwtSection(Consumer c) { String json = createJsonJwtSection(c); From 565e7cae513749a8879208ae49fefeddf70bde74 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 12 May 2025 11:17:32 -0700 Subject: [PATCH 22/27] Removed incorrect Utils.closeQuietly message and added "Package-visible for testing" comments for internal configure() methods --- .../oauthbearer/OAuthBearerLoginCallbackHandler.java | 7 +++++-- .../oauthbearer/OAuthBearerValidatorCallbackHandler.java | 7 +++++-- 2 files changed, 10 insertions(+), 4 deletions(-) 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 4c6d52654e712..35dae9da83225 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 @@ -192,6 +192,9 @@ public void configure(Map configs, String saslMechanism, List moduleOptions, JwtRetriever jwtRetriever, JwtValidator jwtValidator) { this.moduleOptions = moduleOptions; this.jwtRetriever = jwtRetriever; @@ -212,8 +215,8 @@ void configure(Map moduleOptions, JwtRetriever jwtRetriever, Jwt @Override public void close() { - Utils.closeQuietly(jwtRetriever, "The OAuth login callback encountered an error when closing the JwtRetriever"); - Utils.closeQuietly(jwtValidator, "The OAuth login callback encountered an error when closing the JwtValidator"); + Utils.closeQuietly(jwtRetriever, "jwtRetriever"); + Utils.closeQuietly(jwtValidator, "jwtValidator"); } @Override 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 346a5b2b87371..528dcdc6c00fc 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 @@ -138,6 +138,9 @@ public void configure(Map configs, String saslMechanism, List Date: Mon, 12 May 2025 11:24:53 -0700 Subject: [PATCH 23/27] Reverted handler's internal configure() back to init() --- .../OAuthBearerLoginCallbackHandler.java | 4 ++-- .../OAuthBearerValidatorCallbackHandler.java | 4 ++-- .../OAuthBearerLoginCallbackHandlerTest.java | 14 +++++++------- .../OAuthBearerValidatorCallbackHandlerTest.java | 14 ++++++++------ 4 files changed, 19 insertions(+), 17 deletions(-) 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 35dae9da83225..98dc8c4c4cad5 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 @@ -189,13 +189,13 @@ public void configure(Map configs, String saslMechanism, List moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries); JwtRetriever jwtRetriever = new DefaultJwtRetriever(configs, saslMechanism, moduleOptions); JwtValidator jwtValidator = new DefaultJwtValidator(configs, saslMechanism); - configure(moduleOptions, jwtRetriever, jwtValidator); + init(moduleOptions, jwtRetriever, jwtValidator); } /* * Package-visible for testing. */ - void configure(Map moduleOptions, JwtRetriever jwtRetriever, JwtValidator jwtValidator) { + void init(Map moduleOptions, JwtRetriever jwtRetriever, JwtValidator jwtValidator) { this.moduleOptions = moduleOptions; this.jwtRetriever = jwtRetriever; this.jwtValidator = jwtValidator; 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 528dcdc6c00fc..cf2687c1a0d1c 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 @@ -135,13 +135,13 @@ public void configure(Map configs, String saslMechanism, List accessToken; JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.configure(Map.of(), jwtRetriever, jwtValidator); + handler.init(Map.of(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -101,7 +101,7 @@ public void testHandleSaslExtensionsCallback() throws Exception { JwtRetriever jwtRetriever = createJwtRetriever(configs, jaasConfig); JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.configure(jaasConfig, jwtRetriever, jwtValidator); + handler.init(jaasConfig, jwtRetriever, jwtValidator); try { SaslExtensionsCallback callback = new SaslExtensionsCallback(); @@ -132,7 +132,7 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() { JwtRetriever jwtRetriever = createJwtRetriever(configs, jaasConfig); JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.configure(jaasConfig, jwtRetriever, jwtValidator); + handler.init(jaasConfig, jwtRetriever, jwtValidator); try { SaslExtensionsCallback callback = new SaslExtensionsCallback(); @@ -150,7 +150,7 @@ public void testInvalidCallbackGeneratesUnsupportedCallbackException() { JwtRetriever jwtRetriever = () -> "test"; JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.configure(Map.of(), jwtRetriever, jwtValidator); + handler.init(Map.of(), jwtRetriever, jwtValidator); try { Callback unsupportedCallback = new Callback() { }; @@ -176,7 +176,7 @@ public void testMissingAccessToken() { }; JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.configure(Map.of(), jwtRetriever, jwtValidator); + handler.init(Map.of(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); @@ -205,7 +205,7 @@ public void testFileTokenRetrieverHandlesNewline() throws IOException { JwtRetriever jwtRetriever = new FileJwtRetriever(accessTokenFile.toPath()); JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.configure(Map.of(), jwtRetriever, jwtValidator); + handler.init(Map.of(), jwtRetriever, jwtValidator); OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); try { @@ -229,7 +229,7 @@ private void testInvalidAccessToken(String accessToken, String expectedMessageSu JwtRetriever jwtRetriever = () -> accessToken; JwtValidator jwtValidator = createJwtValidator(configs); OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler(); - handler.configure(Map.of(), jwtRetriever, jwtValidator); + handler.init(Map.of(), jwtRetriever, jwtValidator); try { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); 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 a4ee40d15b348..cf7bb0fc481b6 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 @@ -58,7 +58,7 @@ public void testBasic() throws Exception { CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); JwtValidator jwtValidator = createJwtValidator(configs, verificationKeyResolver); OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); - handler.configure(verificationKeyResolver, jwtValidator); + handler.init(verificationKeyResolver, jwtValidator); try { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); @@ -87,7 +87,7 @@ public void testInvalidAccessToken() throws Exception { } @Test - public void testConfigureThrowsExceptionOnJwtValidatorInit() throws IOException { + public void testHandlerInitThrowsException() throws IOException { AccessTokenBuilder builder = new AccessTokenBuilder() .alg(AlgorithmIdentifiers.RSA_USING_SHA256); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); @@ -105,15 +105,16 @@ public OAuthBearerToken validate(String accessToken) throws ValidateException { OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); + // An error initializing the JwtValidator should cause OAuthBearerValidatorCallbackHandler.init() to fail. assertThrowsWithMessage( KafkaException.class, - () -> handler.configure(verificationKeyResolver, jwtValidator), + () -> handler.init(verificationKeyResolver, jwtValidator), "encountered an error when initializing" ); } @Test - public void testConfigureThrowsExceptionOnJwtValidatorClose() throws IOException { + public void testHandlerCloseDoesNotThrowException() throws IOException { AccessTokenBuilder builder = new AccessTokenBuilder() .alg(AlgorithmIdentifiers.RSA_USING_SHA256); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); @@ -130,8 +131,9 @@ public OAuthBearerToken validate(String accessToken) throws ValidateException { }; OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); - handler.configure(verificationKeyResolver, jwtValidator); + handler.init(verificationKeyResolver, jwtValidator); + // An error closings the JwtValidator should *not* cause OAuthBearerValidatorCallbackHandler.close() to fail. assertDoesNotThrow(handler::close); } @@ -142,7 +144,7 @@ private void assertInvalidAccessTokenFails(String accessToken, String expectedMe CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); JwtValidator jwtValidator = createJwtValidator(configs, verificationKeyResolver); OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); - handler.configure(verificationKeyResolver, jwtValidator); + handler.init(verificationKeyResolver, jwtValidator); try { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken); From 07f2874f2cde658ebd66d5819bfc58d02696196e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 12 May 2025 11:36:45 -0700 Subject: [PATCH 24/27] Updated string passed to Utils.closeQuietly --- .../security/oauthbearer/OAuthBearerLoginCallbackHandler.java | 4 ++-- .../oauthbearer/OAuthBearerValidatorCallbackHandler.java | 4 ++-- .../oauthbearer/internals/secured/DefaultJwtRetriever.java | 2 +- .../oauthbearer/internals/secured/DefaultJwtValidator.java | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) 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 98dc8c4c4cad5..0d8701ba11db3 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 @@ -215,8 +215,8 @@ void init(Map moduleOptions, JwtRetriever jwtRetriever, JwtValid @Override public void close() { - Utils.closeQuietly(jwtRetriever, "jwtRetriever"); - Utils.closeQuietly(jwtValidator, "jwtValidator"); + Utils.closeQuietly(jwtRetriever, "JWT retriever"); + Utils.closeQuietly(jwtValidator, "JWT validator"); } @Override 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 cf2687c1a0d1c..c10b7db4e241d 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 @@ -160,8 +160,8 @@ void init(CloseableVerificationKeyResolver verificationKeyResolver, JwtValidator @Override public void close() { - Utils.closeQuietly(jwtValidator, "jwtValidator"); - Utils.closeQuietly(verificationKeyResolver, "verificationKeyResolver"); + Utils.closeQuietly(jwtValidator, "JWT validator"); + Utils.closeQuietly(verificationKeyResolver, "JWT verification key resolver"); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java index 615429ed29dbb..e16ee9a1e3da4 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java @@ -101,7 +101,7 @@ public String retrieve() throws IOException { @Override public void close() throws IOException { - Utils.closeQuietly(delegate, "delegate"); + Utils.closeQuietly(delegate, "JWT retriever delegate"); } /** diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java index 9febbacfd3d71..5a8b05191d9b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java @@ -97,7 +97,7 @@ public OAuthBearerToken validate(String accessToken) throws ValidateException { @Override public void close() throws IOException { - Utils.closeQuietly(delegate, "delegate"); + Utils.closeQuietly(delegate, "JWT validator delegate"); } JwtValidator delegate() { From 4c7d412ce4a00aca8799f01e2b601109f3c87210 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 12 May 2025 18:56:11 -0700 Subject: [PATCH 25/27] Updated error type and message when DefaultJwtRetriever or DefaultJwtValidator have a null delegate --- .../oauthbearer/internals/secured/DefaultJwtRetriever.java | 6 ++++-- .../oauthbearer/internals/secured/DefaultJwtValidator.java | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java index e16ee9a1e3da4..2d607ddcda8ad 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtRetriever.java @@ -24,7 +24,6 @@ import java.net.URL; import java.util.Locale; import java.util.Map; -import java.util.Objects; import javax.net.ssl.SSLSocketFactory; @@ -96,7 +95,10 @@ public void init() throws IOException { @Override public String retrieve() throws IOException { - return Objects.requireNonNull(delegate).retrieve(); + if (delegate == null) + throw new IllegalStateException("JWT retriever delegate is null; please call init() first"); + + return delegate.retrieve(); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java index 5a8b05191d9b2..5cd1e61db8886 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultJwtValidator.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -92,7 +91,10 @@ public void init() throws IOException { @Override public OAuthBearerToken validate(String accessToken) throws ValidateException { - return Objects.requireNonNull(delegate).validate(accessToken); + if (delegate == null) + throw new IllegalStateException("JWT validator delegate is null; please call init() first"); + + return delegate.validate(accessToken); } @Override From 187cfd6a3f4a9cbdd1f12393f6c85592abca8f6c Mon Sep 17 00:00:00 2001 From: Kirk True Date: Mon, 12 May 2025 18:57:21 -0700 Subject: [PATCH 26/27] Updated testHandlerInitThrowsException to ensure the error was that thrown from JwtValidator.init() --- .../OAuthBearerValidatorCallbackHandlerTest.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) 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 cf7bb0fc481b6..5eda63ce5fd71 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 @@ -40,6 +40,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; 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; public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest { @@ -88,13 +89,15 @@ public void testInvalidAccessToken() throws Exception { @Test public void testHandlerInitThrowsException() throws IOException { + IOException initError = new IOException("init() error"); + AccessTokenBuilder builder = new AccessTokenBuilder() .alg(AlgorithmIdentifiers.RSA_USING_SHA256); CloseableVerificationKeyResolver verificationKeyResolver = createVerificationKeyResolver(builder); JwtValidator jwtValidator = new JwtValidator() { @Override public void init() throws IOException { - throw new IOException("My init had an error!"); + throw initError; } @Override @@ -106,11 +109,13 @@ public OAuthBearerToken validate(String accessToken) throws ValidateException { OAuthBearerValidatorCallbackHandler handler = new OAuthBearerValidatorCallbackHandler(); // An error initializing the JwtValidator should cause OAuthBearerValidatorCallbackHandler.init() to fail. - assertThrowsWithMessage( + KafkaException root = assertThrows( KafkaException.class, - () -> handler.init(verificationKeyResolver, jwtValidator), - "encountered an error when initializing" + () -> handler.init(verificationKeyResolver, jwtValidator) ); + assertNotNull(root); + assertNotNull(root.getCause()); + assertEquals(initError, root.getCause()); } @Test @@ -121,7 +126,7 @@ public void testHandlerCloseDoesNotThrowException() throws IOException { JwtValidator jwtValidator = new JwtValidator() { @Override public void close() throws IOException { - throw new IOException("My close had an error!"); + throw new IOException("close() error"); } @Override From 64277d6df1f488b41e20e754229134e51546908e Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 13 May 2025 06:47:25 -0700 Subject: [PATCH 27/27] Removed unnecessary not-null check from assertThrows in OAuthBearerValidatorCallbackHandlerTest --- .../oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java | 1 - 1 file changed, 1 deletion(-) 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 5eda63ce5fd71..0f1315b428121 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 @@ -113,7 +113,6 @@ public OAuthBearerToken validate(String accessToken) throws ValidateException { KafkaException.class, () -> handler.init(verificationKeyResolver, jwtValidator) ); - assertNotNull(root); assertNotNull(root.getCause()); assertEquals(initError, root.getCause()); }