extensions = new HashMap<>();
@@ -286,9 +287,8 @@ private void handleExtensionsCallback(SaslExtensionsCallback callback) {
callback.extensions(saslExtensions);
}
- private void checkInitialized() {
+ protected void checkInitialized() {
if (!isInitialized)
- throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName()));
+ throw new IllegalStateException(String.format("To use %s, first call the configure method", getClass().getSimpleName()));
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
index f9422370db18b..fb94057f57652 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java
@@ -14,37 +14,28 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer;
-import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidator;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidatorFactory;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.JaasOptionsUtils;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.RefreshingHttpsJwksVerificationKeyResolver;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.ValidateException;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.VerificationKeyResolverFactory;
+import org.apache.kafka.common.utils.Utils;
-import org.jose4j.jws.JsonWebSignature;
-import org.jose4j.jwx.JsonWebStructure;
-import org.jose4j.lang.UnresolvableKeyException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.Closeable;
import java.io.IOException;
-import java.security.Key;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.atomic.AtomicInteger;
import javax.security.auth.callback.Callback;
import javax.security.auth.callback.UnsupportedCallbackException;
import javax.security.auth.login.AppConfigurationEntry;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS;
+import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.getConfiguredInstanceOrDefault;
+
/**
*
* OAuthBearerValidatorCallbackHandler
is an {@link AuthenticateCallbackHandler} that
@@ -58,8 +49,7 @@
*
*
* This {@link AuthenticateCallbackHandler} is enabled in the broker configuration by setting the
- * {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}
- * like so:
+ * {@link BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG} like so:
*
*
* listener.name..oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallbackHandler
@@ -73,7 +63,7 @@
*
*
* The configuration option
- * {@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}
+ * {@link SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}
* is also required in order to contact the OAuth/OIDC provider to retrieve the JWKS for use in
* JWT signature validation. For example:
*
@@ -103,64 +93,39 @@
*
*
*/
-
-public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler, Closeable {
private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
- /**
- * Because a {@link CloseableVerificationKeyResolver} instance can spawn threads and issue
- * HTTP(S) calls ({@link RefreshingHttpsJwksVerificationKeyResolver}), we only want to create
- * a new instance for each particular set of configuration. Because each set of configuration
- * may have multiple instances, we want to reuse the single instance.
- */
-
- private static final Map VERIFICATION_KEY_RESOLVER_CACHE = new HashMap<>();
-
- private CloseableVerificationKeyResolver verificationKeyResolver;
-
- private AccessTokenValidator accessTokenValidator;
+ private JwtValidator jwtValidator;
private boolean isInitialized = false;
@Override
public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
- Map moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries);
- CloseableVerificationKeyResolver verificationKeyResolver;
-
- // Here's the logic which keeps our VerificationKeyResolvers down to a single instance.
- synchronized (VERIFICATION_KEY_RESOLVER_CACHE) {
- VerificationKeyResolverKey key = new VerificationKeyResolverKey(configs, moduleOptions);
- verificationKeyResolver = VERIFICATION_KEY_RESOLVER_CACHE.computeIfAbsent(key, k ->
- new RefCountingVerificationKeyResolver(VerificationKeyResolverFactory.create(configs, saslMechanism, moduleOptions)));
- }
-
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
- init(verificationKeyResolver, accessTokenValidator);
+ this.jwtValidator = getConfiguredInstanceOrDefault(
+ configs,
+ saslMechanism,
+ jaasConfigEntries,
+ SASL_OAUTHBEARER_JWT_VALIDATOR_CLASS,
+ JwtValidator.class
+ );
+
+ this.isInitialized = true;
}
- public void init(CloseableVerificationKeyResolver verificationKeyResolver, AccessTokenValidator accessTokenValidator) {
- this.verificationKeyResolver = verificationKeyResolver;
- this.accessTokenValidator = accessTokenValidator;
-
- try {
- verificationKeyResolver.init();
- } catch (Exception e) {
- throw new KafkaException("The OAuth validator configuration encountered an error when initializing the VerificationKeyResolver", e);
- }
-
- isInitialized = true;
+ void configure(JwtValidator jwtValidator,
+ Map configs,
+ String saslMechanism,
+ List jaasConfigEntries) {
+ this.jwtValidator = jwtValidator;
+ this.jwtValidator.configure(configs, saslMechanism, jaasConfigEntries);
+ this.isInitialized = true;
}
@Override
public void close() {
- if (verificationKeyResolver != null) {
- try {
- verificationKeyResolver.close();
- } catch (Exception e) {
- log.error(e.getMessage(), e);
- }
- }
+ Utils.closeQuietly(jwtValidator, "jwtValidator");
}
@Override
@@ -181,12 +146,10 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback
private void handleValidatorCallback(OAuthBearerValidatorCallback callback) {
checkInitialized();
- OAuthBearerToken token;
-
try {
- token = accessTokenValidator.validate(callback.tokenValue());
+ OAuthBearerToken token = jwtValidator.validate(callback.tokenValue());
callback.token(token);
- } catch (ValidateException e) {
+ } catch (JwtValidatorException e) {
log.warn(e.getMessage(), e);
callback.error("invalid_token", null, null);
}
@@ -200,81 +163,6 @@ private void handleExtensionsValidatorCallback(OAuthBearerExtensionsValidatorCal
private void checkInitialized() {
if (!isInitialized)
- throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName()));
+ throw new IllegalStateException(String.format("To use %s, first call the configure method", getClass().getSimpleName()));
}
-
- /**
- * VkrKey
is a simple structure which encapsulates the criteria for different
- * sets of configuration. This will allow us to use this object as a key in a {@link Map}
- * to keep a single instance per key.
- */
-
- private static class VerificationKeyResolverKey {
-
- private final Map configs;
-
- private final Map moduleOptions;
-
- public VerificationKeyResolverKey(Map configs, Map moduleOptions) {
- this.configs = configs;
- this.moduleOptions = moduleOptions;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- VerificationKeyResolverKey that = (VerificationKeyResolverKey) o;
- return configs.equals(that.configs) && moduleOptions.equals(that.moduleOptions);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(configs, moduleOptions);
- }
-
- }
-
- /**
- * RefCountingVerificationKeyResolver
allows us to share a single
- * {@link CloseableVerificationKeyResolver} instance between multiple
- * {@link AuthenticateCallbackHandler} instances and perform the lifecycle methods the
- * appropriate number of times.
- */
-
- private static class RefCountingVerificationKeyResolver implements CloseableVerificationKeyResolver {
-
- private final CloseableVerificationKeyResolver delegate;
-
- private final AtomicInteger count = new AtomicInteger(0);
-
- public RefCountingVerificationKeyResolver(CloseableVerificationKeyResolver delegate) {
- this.delegate = delegate;
- }
-
- @Override
- public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException {
- return delegate.resolveKey(jws, nestingContext);
- }
-
- @Override
- public void init() throws IOException {
- if (count.incrementAndGet() == 1)
- delegate.init();
- }
-
- @Override
- public void close() throws IOException {
- if (count.decrementAndGet() == 0)
- delegate.close();
- }
-
- }
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetriever.java
deleted file mode 100644
index 080ea4515b4dd..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetriever.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * An AccessTokenRetriever
is the internal API by which the login module will
- * retrieve an access token for use in authorization by the broker. The implementation may
- * involve authentication to a remote system, or it can be as simple as loading the contents
- * of a file or configuration setting.
- *
- * Retrieval is a separate concern from validation, so it isn't necessary for
- * the AccessTokenRetriever
implementation to validate the integrity of the JWT
- * access token.
- *
- * @see HttpAccessTokenRetriever
- * @see FileTokenRetriever
- */
-
-public interface AccessTokenRetriever extends Initable, Closeable {
-
- /**
- * Retrieves a JWT access token in its serialized three-part form. The implementation
- * is free to determine how it should be retrieved but should not perform validation
- * on the result.
- *
- * Note: This is a blocking function and callers should be aware that the
- * implementation may be communicating over a network, with the file system, coordinating
- * threads, etc. The facility in the {@link javax.security.auth.spi.LoginModule} from
- * which this is ultimately called does not provide an asynchronous approach.
- *
- * @return Non-null
JWT access token string
- *
- * @throws IOException Thrown on errors related to IO during retrieval
- */
-
- String retrieve() throws IOException;
-
- /**
- * Lifecycle method to perform a clean shutdown of the retriever. This must
- * be performed by the caller to ensure the correct state, freeing up and releasing any
- * resources performed in {@link #init()}.
- *
- * @throws IOException Thrown on errors related to IO during closure
- */
-
- default void close() throws IOException {
- // This method left intentionally blank.
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java
deleted file mode 100644
index 0ed4a1a230349..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.config.SaslConfigs;
-
-import java.net.URL;
-import java.util.Locale;
-import java.util.Map;
-
-import javax.net.ssl.SSLSocketFactory;
-
-import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG;
-
-public class AccessTokenRetrieverFactory {
-
- /**
- * Create an {@link AccessTokenRetriever} from the given SASL and JAAS configuration.
- *
- * Note: the returned AccessTokenRetriever
is not initialized
- * here and must be done by the caller prior to use.
- *
- * @param configs SASL configuration
- * @param jaasConfig JAAS configuration
- *
- * @return Non-null
{@link AccessTokenRetriever}
- */
-
- public static AccessTokenRetriever create(Map configs, Map jaasConfig) {
- return create(configs, null, jaasConfig);
- }
-
- public static AccessTokenRetriever create(Map configs,
- String saslMechanism,
- Map jaasConfig) {
- ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
- URL tokenEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL);
-
- if (tokenEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file")) {
- return new FileTokenRetriever(cu.validateFile(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL));
- } else {
- JaasOptionsUtils jou = new JaasOptionsUtils(jaasConfig);
- String clientId = jou.validateString(CLIENT_ID_CONFIG);
- String clientSecret = jou.validateString(CLIENT_SECRET_CONFIG);
- String scope = jou.validateString(SCOPE_CONFIG, false);
-
- SSLSocketFactory sslSocketFactory = null;
-
- if (jou.shouldCreateSSLSocketFactory(tokenEndpointUrl))
- sslSocketFactory = jou.createSSLSocketFactory();
-
- boolean urlencodeHeader = validateUrlencodeHeader(cu);
-
- return new HttpAccessTokenRetriever(clientId,
- clientSecret,
- scope,
- sslSocketFactory,
- tokenEndpointUrl.toString(),
- cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MS),
- cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS),
- cu.validateInteger(SASL_LOGIN_CONNECT_TIMEOUT_MS, false),
- cu.validateInteger(SASL_LOGIN_READ_TIMEOUT_MS, false),
- urlencodeHeader);
- }
- }
-
- /**
- * In some cases, the incoming {@link Map} doesn't contain a value for
- * {@link SaslConfigs#SASL_OAUTHBEARER_HEADER_URLENCODE}. Returning {@code null} from {@link Map#get(Object)}
- * will cause a {@link NullPointerException} when it is later unboxed.
- *
- *
- *
- * This utility method ensures that we have a non-{@code null} value to use in the
- * {@link HttpAccessTokenRetriever} constructor.
- */
- static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) {
- Boolean urlencodeHeader = configurationUtils.validateBoolean(SASL_OAUTHBEARER_HEADER_URLENCODE, false);
-
- if (urlencodeHeader != null)
- return urlencodeHeader;
- else
- return DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
- }
-
-}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactory.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactory.java
deleted file mode 100644
index e4b39e5cc53c6..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorFactory.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.jose4j.keys.resolvers.VerificationKeyResolver;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME;
-
-public class AccessTokenValidatorFactory {
-
- public static AccessTokenValidator create(Map configs) {
- return create(configs, (String) null);
- }
-
- public static AccessTokenValidator create(Map configs, String saslMechanism) {
- ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
- String scopeClaimName = cu.get(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME);
- String subClaimName = cu.get(SASL_OAUTHBEARER_SUB_CLAIM_NAME);
- return new LoginAccessTokenValidator(scopeClaimName, subClaimName);
- }
-
- public static AccessTokenValidator create(Map configs,
- VerificationKeyResolver verificationKeyResolver) {
- return create(configs, null, verificationKeyResolver);
- }
-
- public static AccessTokenValidator create(Map configs,
- String saslMechanism,
- VerificationKeyResolver verificationKeyResolver) {
- ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
- Set expectedAudiences = null;
- List l = cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE);
-
- if (l != null)
- expectedAudiences = Set.copyOf(l);
-
- Integer clockSkew = cu.validateInteger(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, false);
- String expectedIssuer = cu.validateString(SASL_OAUTHBEARER_EXPECTED_ISSUER, false);
- String scopeClaimName = cu.validateString(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME);
- String subClaimName = cu.validateString(SASL_OAUTHBEARER_SUB_CLAIM_NAME);
-
- return new ValidatorAccessTokenValidator(clockSkew,
- expectedAudiences,
- expectedIssuer,
- verificationKeyResolver,
- scopeClaimName,
- subClaimName);
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerToken.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerToken.java
index 48ef89844070c..1ba872c0274b3 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerToken.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/BasicOAuthBearerToken.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
@@ -31,7 +30,6 @@
*
* @see RFC 7515: JSON Web Signature (JWS)
*/
-
public class BasicOAuthBearerToken implements OAuthBearerToken {
private final String token;
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java
new file mode 100644
index 0000000000000..0c20909f98695
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CachedFile.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.KafkaException;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.function.BiFunction;
+
+public class CachedFile {
+
+ public static final BiFunction NOOP_TRANSFORMER = (file, contents) -> contents;
+
+ protected final File file;
+ protected final BiFunction transformer;
+ protected final BiFunction, Boolean> cacheRefreshPolicy;
+ protected CachedFileInfo cachedFileInfo;
+
+ public CachedFile(File file,
+ BiFunction transformer,
+ BiFunction, Boolean> cacheRefreshPolicy) {
+ this.file = file;
+ this.transformer = transformer;
+ this.cacheRefreshPolicy = cacheRefreshPolicy;
+ }
+
+ public static BiFunction, Boolean> staticCacheRefreshPolicy() {
+ return (file, cachedFileInfo) -> cachedFileInfo == null;
+ }
+
+ public static BiFunction, Boolean> lastModifiedCacheRefreshPolicy() {
+ return (file, cachedFileInfo) -> cachedFileInfo == null || cachedFileInfo.lastModified() != file.lastModified();
+ }
+
+ public long size() {
+ return cachedFileInfo().size();
+ }
+
+ public long lastModified() {
+ return cachedFileInfo().lastModified();
+ }
+
+ public String contents() {
+ return cachedFileInfo().contents();
+ }
+
+ public T transformed() {
+ return cachedFileInfo().transformed();
+ }
+
+ protected CachedFileInfo cachedFileInfo() {
+ if (cacheRefreshPolicy.apply(file, cachedFileInfo)) {
+ long size = file.length();
+ long lastModified = file.lastModified();
+ String contents;
+
+ try {
+ contents = Files.readString(file.toPath());
+ } catch (IOException e) {
+ throw new KafkaException("Error reading the file contents of OAuth resource " + file.getPath() + " for caching");
+ }
+
+ T transformed = transformer.apply(file, contents);
+ cachedFileInfo = new CachedFileInfo<>(size, lastModified, contents, transformed);
+ }
+
+ return cachedFileInfo;
+ }
+
+ public static class CachedFileInfo {
+
+ private final long size;
+
+ private final long lastModified;
+
+ private final String contents;
+
+ private final T transformed;
+
+ public CachedFileInfo(long size, long lastModified, String contents, T transformed) {
+ this.size = size;
+ this.lastModified = lastModified;
+ this.contents = contents;
+ this.transformed = transformed;
+ }
+
+ public long size() {
+ return size;
+ }
+
+ public long lastModified() {
+ return lastModified;
+ }
+
+ public String contents() {
+ return contents;
+ }
+
+ public T transformed() {
+ return transformed;
+ }
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java
deleted file mode 100644
index 5bf5ef068ed0f..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClaimValidationUtils.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * Simple utility class to perform basic cleaning and validation on input values so that they're
- * performed consistently throughout the code base.
- */
-
-public class ClaimValidationUtils {
-
- /**
- * Validates that the scopes are valid, where invalid means any of
- * the following:
- *
- *
- * - Collection is
null
- * - Collection has duplicates
- * - Any of the elements in the collection are
null
- * - Any of the elements in the collection are zero length
- * - Any of the elements in the collection are whitespace only
- *
- *
- * @param scopeClaimName Name of the claim used for the scope values
- * @param scopes Collection of String scopes
- *
- * @return Unmodifiable {@link Set} that includes the values of the original set, but with
- * each value trimmed
- *
- * @throws ValidateException Thrown if the value is null
, contains duplicates, or
- * if any of the values in the set are null
, empty,
- * or whitespace only
- */
-
- public static Set validateScopes(String scopeClaimName, Collection scopes) throws ValidateException {
- if (scopes == null)
- throw new ValidateException(String.format("%s value must be non-null", scopeClaimName));
-
- Set copy = new HashSet<>();
-
- for (String scope : scopes) {
- scope = validateString(scopeClaimName, scope);
-
- if (copy.contains(scope))
- throw new ValidateException(String.format("%s value must not contain duplicates - %s already present", scopeClaimName, scope));
-
- copy.add(scope);
- }
-
- return Collections.unmodifiableSet(copy);
- }
-
- /**
- * Validates that the given lifetime is valid, where invalid means any of
- * the following:
- *
- *
- * null
- * - Negative
- *
- *
- * @param claimName Name of the claim
- * @param claimValue Expiration time (in milliseconds)
- *
- * @return Input parameter, as provided
- *
- * @throws ValidateException Thrown if the value is null
or negative
- */
-
- public static long validateExpiration(String claimName, Long claimValue) throws ValidateException {
- if (claimValue == null)
- throw new ValidateException(String.format("%s value must be non-null", claimName));
-
- if (claimValue < 0)
- throw new ValidateException(String.format("%s value must be non-negative; value given was \"%s\"", claimName, claimValue));
-
- return claimValue;
- }
-
- /**
- * Validates that the given claim value is valid, where invalid means any of
- * the following:
- *
- *
- * null
- * - Zero length
- * - Whitespace only
- *
- *
- * @param claimName Name of the claim
- * @param claimValue Name of the subject
- *
- * @return Trimmed version of the claimValue
parameter
- *
- * @throws ValidateException Thrown if the value is null
, empty, or whitespace only
- */
-
- public static String validateSubject(String claimName, String claimValue) throws ValidateException {
- return validateString(claimName, claimValue);
- }
-
- /**
- * Validates that the given issued at claim name is valid, where invalid means any of
- * the following:
- *
- *
- *
- * @param claimName Name of the claim
- * @param claimValue Start time (in milliseconds) or null
if not used
- *
- * @return Input parameter, as provided
- *
- * @throws ValidateException Thrown if the value is negative
- */
-
- public static Long validateIssuedAt(String claimName, Long claimValue) throws ValidateException {
- if (claimValue != null && claimValue < 0)
- throw new ValidateException(String.format("%s value must be null or non-negative; value given was \"%s\"", claimName, claimValue));
-
- return claimValue;
- }
-
- /**
- * Validates that the given claim name override is valid, where invalid means
- * any of the following:
- *
- *
- * null
- * - Zero length
- * - Whitespace only
- *
- *
- * @param name "Standard" name of the claim, e.g. sub
- * @param value "Override" name of the claim, e.g. email
- *
- * @return Trimmed version of the value
parameter
- *
- * @throws ValidateException Thrown if the value is null
, empty, or whitespace only
- */
-
- public static String validateClaimNameOverride(String name, String value) throws ValidateException {
- return validateString(name, value);
- }
-
- private static String validateString(String name, String value) throws ValidateException {
- if (value == null)
- throw new ValidateException(String.format("%s value must be non-null", name));
-
- if (value.isEmpty())
- throw new ValidateException(String.format("%s value must be non-empty", name));
-
- value = value.trim();
-
- if (value.isEmpty())
- throw new ValidateException(String.format("%s value must not contain only whitespace", name));
-
- return value;
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGenerator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGenerator.java
new file mode 100644
index 0000000000000..d338f18ef1967
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ClientCredentialsRequestGenerator.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.net.http.HttpRequest;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+public class ClientCredentialsRequestGenerator implements HttpRequestGenerator {
+
+ public static final String GRANT_TYPE = "client_credentials";
+
+ private final URL tokenEndpoint;
+ private final String clientId;
+ private final String clientSecret;
+ private final Optional scope;
+
+ public ClientCredentialsRequestGenerator(URL tokenEndpoint,
+ String clientId,
+ String clientSecret,
+ String scope,
+ boolean urlencodeHeader) {
+ this.tokenEndpoint = tokenEndpoint;
+
+ // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1
+ this.clientId = urlencodeHeader ? URLEncoder.encode(clientId, StandardCharsets.UTF_8) : clientId;
+ this.clientSecret = urlencodeHeader ? URLEncoder.encode(clientSecret, StandardCharsets.UTF_8) : clientSecret;
+ this.scope = Utils.isBlank(scope) ? Optional.empty() : Optional.of(URLEncoder.encode(scope, StandardCharsets.UTF_8));
+ }
+
+ @Override
+ public String generateBody() {
+ StringBuilder requestParameters = new StringBuilder();
+ requestParameters.append("grant_type=").append(GRANT_TYPE);
+ scope.ifPresent(s -> requestParameters.append("&scope=").append(s));
+ return requestParameters.toString();
+ }
+
+ @Override
+ public Map generateHeaders() {
+ String s = String.format("%s:%s", clientId, clientSecret);
+ // Per RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496.
+ String encoded = Base64.getEncoder().encodeToString(Utils.utf8(s));
+ String header = String.format("Basic %s", encoded);
+
+ Map headers = new HashMap<>();
+ headers.put("Accept", "application/json");
+ headers.put("Authorization", header);
+ headers.put("Cache-Control", "no-cache");
+ headers.put("Content-Type", "application/x-www-form-urlencoded");
+ return headers;
+ }
+
+ @Override
+ public HttpRequest generateRequest() {
+ HttpRequest.BodyPublisher bodyPublisher = HttpRequest.BodyPublishers.ofString(generateBody());
+
+ URI uri;
+
+ try {
+ uri = tokenEndpoint.toURI();
+ } catch (URISyntaxException e) {
+ throw new KafkaException("An error occurred formatting the OAuth token retrieval request", e);
+ }
+
+ HttpRequest.Builder builder = HttpRequest.newBuilder()
+ .uri(uri)
+ .POST(bodyPublisher);
+
+ for (Map.Entry header : generateHeaders().entrySet())
+ builder = builder.header(header.getKey(), header.getValue());
+
+ return builder.build();
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java
index bf8ca0cb82211..278f80f2fd835 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/CloseableVerificationKeyResolver.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallbackHandler;
@@ -22,7 +21,6 @@
import org.jose4j.keys.resolvers.VerificationKeyResolver;
import java.io.Closeable;
-import java.io.IOException;
/**
* The {@link OAuthBearerValidatorCallbackHandler} uses a {@link VerificationKeyResolver} as
@@ -36,18 +34,9 @@
* @see Closeable
*/
-public interface CloseableVerificationKeyResolver extends Initable, Closeable, VerificationKeyResolver {
-
- /**
- * Lifecycle method to perform a clean shutdown of the {@link VerificationKeyResolver}.
- * This must be performed by the caller to ensure the correct state, freeing up
- * and releasing any resources performed in {@link #init()}.
- *
- * @throws IOException Thrown on errors related to IO during closure
- */
+public interface CloseableVerificationKeyResolver extends VerificationKeyResolver, Closeable {
- default void close() throws IOException {
- // This method left intentionally blank.
+ default void close() {
+ // Do nothing...
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java
deleted file mode 100644
index 10f700826c8bd..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java
+++ /dev/null
@@ -1,252 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.config.ConfigException;
-import org.apache.kafka.common.network.ListenerName;
-
-import java.io.File;
-import java.net.MalformedURLException;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.nio.file.Path;
-import java.util.Arrays;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
-import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT;
-
-/**
- * ConfigurationUtils
is a utility class to perform basic configuration-related
- * logic and is separated out here for easier, more direct testing.
- */
-
-public class ConfigurationUtils {
-
- private final Map configs;
-
- private final String prefix;
-
- public ConfigurationUtils(Map configs) {
- this(configs, null);
- }
-
- public ConfigurationUtils(Map configs, String saslMechanism) {
- this.configs = configs;
-
- if (saslMechanism != null && !saslMechanism.trim().isEmpty())
- this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
- else
- this.prefix = null;
- }
-
- /**
- * Validates that, if a value is supplied, is a file that:
- *
- *
- *
- *
- *
- *
- *
- * If the value is null or an empty string, it is assumed to be an "empty" value and thus.
- * ignored. Any whitespace is trimmed off of the beginning and end.
- */
-
- public Path validateFile(String name) {
- URL url = validateUrl(name);
- File file;
-
- try {
- file = new File(url.toURI().getRawPath()).getAbsoluteFile();
- } catch (URISyntaxException e) {
- throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, url, e.getMessage()));
- }
-
- if (!file.exists())
- throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", name, file));
-
- if (!file.canRead())
- throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't have read permission", name, file));
-
- if (file.isDirectory())
- throw new ConfigException(String.format("The OAuth configuration option %s references a directory (%s), not a file", name, file));
-
- return file.toPath();
- }
-
- /**
- * Validates that, if a value is supplied, is a value that:
- *
- *
- *
- * has a value that is not less than the provided minimum value
- *
- *
- * If the value is null or an empty string, it is assumed to be an "empty" value and thus
- * ignored. Any whitespace is trimmed off of the beginning and end.
- */
-
- public Integer validateInteger(String name, boolean isRequired) {
- Integer value = get(name);
-
- if (value == null) {
- if (isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name));
- else
- return null;
- }
-
- return value;
- }
-
- /**
- * Validates that, if a value is supplied, is a value that:
- *
- *
- *
- * has a value that is not less than the provided minimum value
- *
- *
- * If the value is null or an empty string, it is assumed to be an "empty" value and thus
- * ignored. Any whitespace is trimmed off of the beginning and end.
- */
-
- public Long validateLong(String name) {
- return validateLong(name, true);
- }
-
- public Long validateLong(String name, boolean isRequired) {
- return validateLong(name, isRequired, null);
- }
-
- public Long validateLong(String name, boolean isRequired, Long min) {
- Long value = get(name);
-
- if (value == null) {
- if (isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name));
- else
- return null;
- }
-
- if (min != null && value < min)
- throw new ConfigException(String.format("The OAuth configuration option %s value must be at least %s", name, min));
-
- return value;
- }
-
- /**
- * Validates that the configured URL that:
- *
- *
- * - is well-formed
- * - contains a scheme
- * - uses either HTTP, HTTPS, or file protocols
- * - is in the allow-list
- *
- *
- * No effort is made to connect to the URL in the validation step.
- */
-
- public URL validateUrl(String name) {
- String value = validateString(name);
- URL url;
-
- try {
- url = new URL(value);
- } catch (MalformedURLException e) {
- throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, value, e.getMessage()));
- }
-
- String protocol = url.getProtocol();
-
- if (protocol == null || protocol.trim().isEmpty())
- throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is missing the protocol", name, value));
-
- protocol = protocol.toLowerCase(Locale.ROOT);
-
- if (!(protocol.equals("http") || protocol.equals("https") || protocol.equals("file")))
- throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that contains an invalid protocol (%s); only \"http\", \"https\", and \"file\" protocol are supported", name, value, protocol));
-
- throwIfURLIsNotAllowed(value);
-
- return url;
- }
-
- public String validateString(String name) throws ValidateException {
- return validateString(name, true);
- }
-
- public String validateString(String name, boolean isRequired) throws ValidateException {
- String value = get(name);
-
- if (value == null) {
- if (isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s value must be non-null", name));
- else
- return null;
- }
-
- value = value.trim();
-
- if (value.isEmpty()) {
- if (isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s value must not contain only whitespace", name));
- else
- return null;
- }
-
- return value;
- }
-
- public Boolean validateBoolean(String name, boolean isRequired) {
- Boolean value = get(name);
-
- if (value == null && isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name));
-
- return value;
- }
-
- @SuppressWarnings("unchecked")
- public T get(String name) {
- T value = (T) configs.get(prefix + name);
-
- if (value != null)
- return value;
-
- return (T) configs.get(name);
- }
-
- // visible for testing
- // make sure the url is in the "org.apache.kafka.sasl.oauthbearer.allowed.urls" system property
- void throwIfURLIsNotAllowed(String value) {
- Set allowedUrls = Arrays.stream(
- System.getProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT).split(","))
- .map(String::trim)
- .collect(Collectors.toSet());
- if (!allowedUrls.contains(value)) {
- throw new ConfigException(value + " is not allowed. Update system property '"
- + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG + "' to allow " + value);
- }
- }
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreator.java
new file mode 100644
index 0000000000000..1302bd7c66a0c
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/DefaultAssertionCreator.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.oauthbearer.AssertionCreator;
+import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.security.KeyFactory;
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivateKey;
+import java.security.Signature;
+import java.security.spec.PKCS8EncodedKeySpec;
+import java.time.Duration;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.BiFunction;
+
+import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.staticCacheRefreshPolicy;
+
+public class DefaultAssertionCreator implements AssertionCreator {
+
+ static final String TOKEN_SIGNING_ALGORITHM_RS256 = "RS256";
+ static final String TOKEN_SIGNING_ALGORITHM_ES256 = "ES256";
+
+ private static final BiFunction PRIVATE_KEY_TRANSFORMER = (file, privateKeySecret) -> {
+ try {
+ byte[] pkcs8EncodedBytes = Base64.getDecoder().decode(privateKeySecret);
+ PKCS8EncodedKeySpec keySpec = new PKCS8EncodedKeySpec(pkcs8EncodedBytes);
+ KeyFactory keyFactory = KeyFactory.getInstance("RSA");
+ return keyFactory.generatePrivate(keySpec);
+ } catch (GeneralSecurityException e) {
+ throw new KafkaException("An error occurred generating the OAuth assertion private key from " + file.getPath(), e);
+ }
+ };
+
+ private final Time time;
+ private final String algorithm;
+ private final CachedFile privateKeyFile;
+
+ public DefaultAssertionCreator(Time time, String algorithm, File privateKeyFile) {
+ this.time = time;
+ this.algorithm = algorithm;
+ this.privateKeyFile = new CachedFile<>(
+ privateKeyFile,
+ PRIVATE_KEY_TRANSFORMER,
+ staticCacheRefreshPolicy()
+ );
+ }
+
+ @Override
+ public String create(AssertionJwtTemplate template) throws GeneralSecurityException, IOException {
+ ObjectMapper mapper = new ObjectMapper();
+ Base64.Encoder encoder = Base64.getUrlEncoder().withoutPadding();
+ String header = encodeHeader(template, mapper, encoder);
+ String payload = encodePayload(template, mapper, encoder);
+ String content = header + "." + payload;
+ PrivateKey privateKey = privateKeyFile.transformed();
+ String signedContent = sign(privateKey, content);
+ return content + "." + signedContent;
+ }
+
+ String encodeHeader(AssertionJwtTemplate template,
+ ObjectMapper mapper,
+ Base64.Encoder encoder) throws IOException {
+ Map values = new HashMap<>(template.header());
+ values.put("alg", algorithm);
+ values.put("typ", "JWT");
+
+ String json = mapper.writeValueAsString(values);
+ return encoder.encodeToString(Utils.utf8(json));
+ }
+
+ String encodePayload(AssertionJwtTemplate template,
+ ObjectMapper mapper,
+ Base64.Encoder encoder) throws IOException {
+ long currentTimeSecs = time.milliseconds() / 1000L;
+ long expirationSecs = currentTimeSecs + Duration.ofMinutes(60).toSeconds();
+
+ Map values = new HashMap<>(template.payload());
+ values.put("iat", currentTimeSecs);
+ values.put("exp", expirationSecs);
+
+ String json = mapper.writeValueAsString(values);
+ return encoder.encodeToString(Utils.utf8(json));
+ }
+
+ Signature getSignature() throws GeneralSecurityException {
+ if (algorithm.equalsIgnoreCase(TOKEN_SIGNING_ALGORITHM_RS256)) {
+ return Signature.getInstance("SHA256withRSA");
+ } else if (algorithm.equalsIgnoreCase(TOKEN_SIGNING_ALGORITHM_ES256)) {
+ return Signature.getInstance("SHA256withECDSA");
+ } else {
+ throw new NoSuchAlgorithmException(String.format("Unsupported signing algorithm: %s", algorithm));
+ }
+ }
+
+ String sign(PrivateKey privateKey, String contentToSign) throws GeneralSecurityException {
+ Signature signature = getSignature();
+ signature.initSign(privateKey);
+ signature.update(contentToSign.getBytes(StandardCharsets.UTF_8));
+ byte[] signedContent = signature.sign();
+ return Base64.getUrlEncoder().withoutPadding().encodeToString(signedContent);
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileAssertionCreator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileAssertionCreator.java
new file mode 100644
index 0000000000000..692e3b7550d43
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileAssertionCreator.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.oauthbearer.AssertionCreator;
+import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate;
+
+import java.io.File;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+
+import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.NOOP_TRANSFORMER;
+import static org.apache.kafka.common.security.oauthbearer.internals.secured.CachedFile.staticCacheRefreshPolicy;
+
+public class FileAssertionCreator implements AssertionCreator {
+
+ private final CachedFile assertionFile;
+
+ public FileAssertionCreator(File assertionFile) {
+ try {
+ this.assertionFile = new CachedFile<>(assertionFile, NOOP_TRANSFORMER, staticCacheRefreshPolicy());
+ } catch (Exception e) {
+ throw new KafkaException("An error occurred reading the OAuth JWT from " + assertionFile);
+ }
+ }
+
+ @Override
+ public String create(AssertionJwtTemplate template) throws GeneralSecurityException, IOException {
+ return assertionFile.transformed();
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java
deleted file mode 100644
index c145cf7596959..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/FileTokenRetriever.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.utils.Utils;
-
-import java.io.IOException;
-import java.nio.file.Path;
-
-/**
- * FileTokenRetriever
is an {@link AccessTokenRetriever} that will load the contents,
- * interpreting them as a JWT access key in the serialized form.
- *
- * @see AccessTokenRetriever
- */
-
-public class FileTokenRetriever implements AccessTokenRetriever {
-
- private final Path accessTokenFile;
-
- private String accessToken;
-
- public FileTokenRetriever(Path accessTokenFile) {
- this.accessTokenFile = accessTokenFile;
- }
-
- @Override
- public void init() throws IOException {
- this.accessToken = Utils.readFileAsString(accessTokenFile.toFile().getPath());
- // always non-null; to remove any newline chars or backend will report err
- this.accessToken = this.accessToken.trim();
- }
-
- @Override
- public String retrieve() throws IOException {
- if (accessToken == null)
- throw new IllegalStateException("Access token is null; please call init() first");
-
- return accessToken;
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetriever.java
deleted file mode 100644
index fdc5707278a60..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpAccessTokenRetriever.java
+++ /dev/null
@@ -1,400 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.config.SaslConfigs;
-import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler;
-import org.apache.kafka.common.utils.Utils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.net.URLEncoder;
-import java.nio.charset.StandardCharsets;
-import java.util.Base64;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.SSLSocketFactory;
-
-/**
- * HttpAccessTokenRetriever
is an {@link AccessTokenRetriever} that will
- * communicate with an OAuth/OIDC provider directly via HTTP to post client credentials
- * ({@link OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG}/{@link OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG})
- * to a publicized token endpoint URL
- * ({@link SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL}).
- *
- * @see AccessTokenRetriever
- * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
- * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
- * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
- * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL
- */
-
-public class HttpAccessTokenRetriever implements AccessTokenRetriever {
-
- private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
-
- private static final Set UNRETRYABLE_HTTP_CODES;
-
- private static final int MAX_RESPONSE_BODY_LENGTH = 1000;
-
- public static final String AUTHORIZATION_HEADER = "Authorization";
-
- static {
- // This does not have to be an exhaustive list. There are other HTTP codes that
- // are defined in different RFCs (e.g. https://datatracker.ietf.org/doc/html/rfc6585)
- // that we won't worry about yet. The worst case if a status code is missing from
- // this set is that the request will be retried.
- UNRETRYABLE_HTTP_CODES = new HashSet<>();
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_BAD_REQUEST);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_UNAUTHORIZED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_PAYMENT_REQUIRED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_FORBIDDEN);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_NOT_FOUND);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_BAD_METHOD);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_NOT_ACCEPTABLE);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_PROXY_AUTH);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_CONFLICT);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_GONE);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_LENGTH_REQUIRED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_PRECON_FAILED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_ENTITY_TOO_LARGE);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_REQ_TOO_LONG);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_UNSUPPORTED_TYPE);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
- UNRETRYABLE_HTTP_CODES.add(HttpURLConnection.HTTP_VERSION);
- }
-
- private final String clientId;
-
- private final String clientSecret;
-
- private final String scope;
-
- private final SSLSocketFactory sslSocketFactory;
-
- private final String tokenEndpointUrl;
-
- private final long loginRetryBackoffMs;
-
- private final long loginRetryBackoffMaxMs;
-
- private final Integer loginConnectTimeoutMs;
-
- private final Integer loginReadTimeoutMs;
-
- private final boolean urlencodeHeader;
-
- public HttpAccessTokenRetriever(String clientId,
- String clientSecret,
- String scope,
- SSLSocketFactory sslSocketFactory,
- String tokenEndpointUrl,
- long loginRetryBackoffMs,
- long loginRetryBackoffMaxMs,
- Integer loginConnectTimeoutMs,
- Integer loginReadTimeoutMs,
- boolean urlencodeHeader) {
- this.clientId = Objects.requireNonNull(clientId);
- this.clientSecret = Objects.requireNonNull(clientSecret);
- this.scope = scope;
- this.sslSocketFactory = sslSocketFactory;
- this.tokenEndpointUrl = Objects.requireNonNull(tokenEndpointUrl);
- this.loginRetryBackoffMs = loginRetryBackoffMs;
- this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
- this.loginConnectTimeoutMs = loginConnectTimeoutMs;
- this.loginReadTimeoutMs = loginReadTimeoutMs;
- this.urlencodeHeader = urlencodeHeader;
- }
-
- /**
- * Retrieves a JWT access token in its serialized three-part form. The implementation
- * is free to determine how it should be retrieved but should not perform validation
- * on the result.
- *
- * Note: This is a blocking function and callers should be aware that the
- * implementation communicates over a network. The facility in the
- * {@link javax.security.auth.spi.LoginModule} from which this is ultimately called
- * does not provide an asynchronous approach.
- *
- * @return Non-null
JWT access token string
- *
- * @throws IOException Thrown on errors related to IO during retrieval
- */
-
- @Override
- public String retrieve() throws IOException {
- String authorizationHeader = formatAuthorizationHeader(clientId, clientSecret, urlencodeHeader);
- String requestBody = formatRequestBody(scope);
- Retry retry = new Retry<>(loginRetryBackoffMs, loginRetryBackoffMaxMs);
- Map headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
-
- String responseBody;
-
- try {
- responseBody = retry.execute(() -> {
- HttpURLConnection con = null;
-
- try {
- con = (HttpURLConnection) new URL(tokenEndpointUrl).openConnection();
-
- if (sslSocketFactory != null && con instanceof HttpsURLConnection)
- ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
-
- return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
- } catch (IOException e) {
- throw new ExecutionException(e);
- } finally {
- if (con != null)
- con.disconnect();
- }
- });
- } catch (ExecutionException e) {
- if (e.getCause() instanceof IOException)
- throw (IOException) e.getCause();
- else
- throw new KafkaException(e.getCause());
- }
-
- return parseAccessToken(responseBody);
- }
-
- public static String post(HttpURLConnection con,
- Map headers,
- String requestBody,
- Integer connectTimeoutMs,
- Integer readTimeoutMs)
- throws IOException, UnretryableException {
- handleInput(con, headers, requestBody, connectTimeoutMs, readTimeoutMs);
- return handleOutput(con);
- }
-
- private static void handleInput(HttpURLConnection con,
- Map headers,
- String requestBody,
- Integer connectTimeoutMs,
- Integer readTimeoutMs)
- throws IOException, UnretryableException {
- log.debug("handleInput - starting post for {}", con.getURL());
- con.setRequestMethod("POST");
- con.setRequestProperty("Accept", "application/json");
-
- if (headers != null) {
- for (Map.Entry header : headers.entrySet())
- con.setRequestProperty(header.getKey(), header.getValue());
- }
-
- con.setRequestProperty("Cache-Control", "no-cache");
-
- if (requestBody != null) {
- con.setRequestProperty("Content-Length", String.valueOf(requestBody.length()));
- con.setDoOutput(true);
- }
-
- con.setUseCaches(false);
-
- if (connectTimeoutMs != null)
- con.setConnectTimeout(connectTimeoutMs);
-
- if (readTimeoutMs != null)
- con.setReadTimeout(readTimeoutMs);
-
- log.debug("handleInput - preparing to connect to {}", con.getURL());
- con.connect();
-
- if (requestBody != null) {
- try (OutputStream os = con.getOutputStream()) {
- ByteArrayInputStream is = new ByteArrayInputStream(requestBody.getBytes(StandardCharsets.UTF_8));
- log.debug("handleInput - preparing to write request body to {}", con.getURL());
- copy(is, os);
- }
- }
- }
-
- static String handleOutput(final HttpURLConnection con) throws IOException {
- int responseCode = con.getResponseCode();
- log.debug("handleOutput - responseCode: {}", responseCode);
-
- // NOTE: the contents of the response should not be logged so that we don't leak any
- // sensitive data.
- String responseBody = null;
-
- // NOTE: It is OK to log the error response body and/or its formatted version as
- // per the OAuth spec, it doesn't include sensitive information.
- // See https://www.ietf.org/rfc/rfc6749.txt, section 5.2
- String errorResponseBody = null;
-
- try (InputStream is = con.getInputStream()) {
- ByteArrayOutputStream os = new ByteArrayOutputStream();
- log.debug("handleOutput - preparing to read response body from {}", con.getURL());
- copy(is, os);
- responseBody = os.toString(StandardCharsets.UTF_8);
- } catch (Exception e) {
- // there still can be useful error response from the servers, lets get it
- try (InputStream is = con.getErrorStream()) {
- ByteArrayOutputStream os = new ByteArrayOutputStream();
- log.debug("handleOutput - preparing to read error response body from {}", con.getURL());
- copy(is, os);
- errorResponseBody = os.toString(StandardCharsets.UTF_8);
- } catch (Exception e2) {
- log.warn("handleOutput - error retrieving error information", e2);
- }
- log.warn("handleOutput - error retrieving data", e);
- }
-
- if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
- log.debug("handleOutput - responseCode: {}, error response: {}", responseCode,
- errorResponseBody);
-
- if (responseBody == null || responseBody.isEmpty())
- throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %d from %s and error message %s",
- responseCode, con.getURL(), formatErrorMessage(errorResponseBody)));
-
- return responseBody;
- } else {
- log.warn("handleOutput - error response code: {}, error response body: {}", responseCode,
- formatErrorMessage(errorResponseBody));
-
- if (UNRETRYABLE_HTTP_CODES.contains(responseCode)) {
- // We know that this is a non-transient error, so let's not keep retrying the
- // request unnecessarily.
- throw new UnretryableException(new IOException(String.format("The response code %s and error response %s was encountered reading the token endpoint response; will not attempt further retries",
- responseCode, formatErrorMessage(errorResponseBody))));
- } else {
- // We don't know if this is a transient (retryable) error or not, so let's assume
- // it is.
- throw new IOException(String.format("The unexpected response code %s and error message %s was encountered reading the token endpoint response",
- responseCode, formatErrorMessage(errorResponseBody)));
- }
- }
- }
-
- static void copy(InputStream is, OutputStream os) throws IOException {
- byte[] buf = new byte[4096];
- int b;
-
- while ((b = is.read(buf)) != -1)
- os.write(buf, 0, b);
- }
-
- static String formatErrorMessage(String errorResponseBody) {
- // See https://www.ietf.org/rfc/rfc6749.txt, section 5.2 for the format
- // of this error message.
- if (errorResponseBody == null || errorResponseBody.trim().isEmpty()) {
- return "{}";
- }
- ObjectMapper mapper = new ObjectMapper();
- try {
- JsonNode rootNode = mapper.readTree(errorResponseBody);
- if (!rootNode.at("/error").isMissingNode()) {
- return String.format("{%s - %s}", rootNode.at("/error"), rootNode.at("/error_description"));
- } else if (!rootNode.at("/errorCode").isMissingNode()) {
- return String.format("{%s - %s}", rootNode.at("/errorCode"), rootNode.at("/errorSummary"));
- } else {
- return errorResponseBody;
- }
- } catch (Exception e) {
- log.warn("Error parsing error response", e);
- }
- return String.format("{%s}", errorResponseBody);
- }
-
- static String parseAccessToken(String responseBody) throws IOException {
- ObjectMapper mapper = new ObjectMapper();
- JsonNode rootNode = mapper.readTree(responseBody);
- JsonNode accessTokenNode = rootNode.at("/access_token");
-
- if (accessTokenNode == null) {
- // Only grab the first N characters so that if the response body is huge, we don't
- // blow up.
- String snippet = responseBody;
-
- if (snippet.length() > MAX_RESPONSE_BODY_LENGTH) {
- int actualLength = responseBody.length();
- String s = responseBody.substring(0, MAX_RESPONSE_BODY_LENGTH);
- snippet = String.format("%s (trimmed to first %d characters out of %d total)", s, MAX_RESPONSE_BODY_LENGTH, actualLength);
- }
-
- throw new IOException(String.format("The token endpoint response did not contain an access_token value. Response: (%s)", snippet));
- }
-
- return sanitizeString("the token endpoint response's access_token JSON attribute", accessTokenNode.textValue());
- }
-
- static String formatAuthorizationHeader(String clientId, String clientSecret, boolean urlencode) {
- clientId = sanitizeString("the token endpoint request client ID parameter", clientId);
- clientSecret = sanitizeString("the token endpoint request client secret parameter", clientSecret);
-
- // according to RFC-6749 clientId & clientSecret must be urlencoded, see https://tools.ietf.org/html/rfc6749#section-2.3.1
- if (urlencode) {
- clientId = URLEncoder.encode(clientId, StandardCharsets.UTF_8);
- clientSecret = URLEncoder.encode(clientSecret, StandardCharsets.UTF_8);
- }
-
- String s = String.format("%s:%s", clientId, clientSecret);
- // Per RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496.
- String encoded = Base64.getEncoder().encodeToString(Utils.utf8(s));
- return String.format("Basic %s", encoded);
- }
-
- static String formatRequestBody(String scope) {
- StringBuilder requestParameters = new StringBuilder();
- requestParameters.append("grant_type=client_credentials");
-
- if (scope != null && !scope.trim().isEmpty()) {
- scope = scope.trim();
- String encodedScope = URLEncoder.encode(scope, StandardCharsets.UTF_8);
- requestParameters.append("&scope=").append(encodedScope);
- }
-
- return requestParameters.toString();
- }
-
- private static String sanitizeString(String name, String value) {
- if (value == null)
- throw new IllegalArgumentException(String.format("The value for %s must be non-null", name));
-
- if (value.isEmpty())
- throw new IllegalArgumentException(String.format("The value for %s must be non-empty", name));
-
- value = value.trim();
-
- if (value.isEmpty())
- throw new IllegalArgumentException(String.format("The value for %s must not contain only whitespace", name));
-
- return value;
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/UnretryableException.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestGenerator.java
similarity index 74%
rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/UnretryableException.java
rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestGenerator.java
index a53ae6ec115c7..b5a7c34985969 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/UnretryableException.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/HttpRequestGenerator.java
@@ -14,15 +14,21 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
-import org.apache.kafka.common.KafkaException;
+import java.io.Closeable;
+import java.net.http.HttpRequest;
+import java.util.Map;
-public class UnretryableException extends KafkaException {
+public interface HttpRequestGenerator extends Closeable {
- public UnretryableException(Throwable cause) {
- super(cause);
- }
+ String generateBody();
+
+ Map generateHeaders();
+ HttpRequest generateRequest();
+
+ default void close() {
+ // Do nothing...
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java
deleted file mode 100644
index 3e49595dbc1b2..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JaasOptionsUtils.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.config.AbstractConfig;
-import org.apache.kafka.common.config.ConfigDef;
-import org.apache.kafka.common.config.ConfigException;
-import org.apache.kafka.common.network.ConnectionMode;
-import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
-import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
-import org.apache.kafka.common.security.ssl.SslFactory;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URL;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-
-import javax.net.ssl.SSLSocketFactory;
-import javax.security.auth.login.AppConfigurationEntry;
-
-/**
- * JaasOptionsUtils
is a utility class to perform logic for the JAAS options and
- * is separated out here for easier, more direct testing.
- */
-
-public class JaasOptionsUtils {
-
- private static final Logger log = LoggerFactory.getLogger(JaasOptionsUtils.class);
-
- private final Map options;
-
- public JaasOptionsUtils(Map options) {
- this.options = options;
- }
-
- public static Map getOptions(String saslMechanism, List jaasConfigEntries) {
- if (!OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(saslMechanism))
- throw new IllegalArgumentException(String.format("Unexpected SASL mechanism: %s", saslMechanism));
-
- if (Objects.requireNonNull(jaasConfigEntries).size() != 1 || jaasConfigEntries.get(0) == null)
- throw new IllegalArgumentException(String.format("Must supply exactly 1 non-null JAAS mechanism configuration (size was %d)", jaasConfigEntries.size()));
-
- return Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
- }
-
- public boolean shouldCreateSSLSocketFactory(URL url) {
- return url.getProtocol().equalsIgnoreCase("https");
- }
-
- public Map getSslClientConfig() {
- ConfigDef sslConfigDef = new ConfigDef();
- sslConfigDef.withClientSslSupport();
- AbstractConfig sslClientConfig = new AbstractConfig(sslConfigDef, options);
- return sslClientConfig.values();
- }
-
- public SSLSocketFactory createSSLSocketFactory() {
- Map sslClientConfig = getSslClientConfig();
- SslFactory sslFactory = new SslFactory(ConnectionMode.CLIENT);
- sslFactory.configure(sslClientConfig);
- SSLSocketFactory socketFactory = ((DefaultSslEngineFactory) sslFactory.sslEngineFactory()).sslContext().getSocketFactory();
- log.debug("Created SSLSocketFactory: {}", sslClientConfig);
- return socketFactory;
- }
-
- public String validateString(String name) throws ValidateException {
- return validateString(name, true);
- }
-
- public String validateString(String name, boolean isRequired) throws ValidateException {
- String value = (String) options.get(name);
-
- if (value == null) {
- if (isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s value must be non-null", name));
- else
- return null;
- }
-
- value = value.trim();
-
- if (value.isEmpty()) {
- if (isRequired)
- throw new ConfigException(String.format("The OAuth configuration option %s value must not contain only whitespace", name));
- else
- return null;
- }
-
- return value;
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java
index 27cdccb286c2c..e143a3e6beb8c 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwksFileVerificationKeyResolver.java
@@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.utils.Utils;
import org.jose4j.jwk.JsonWebKeySet;
@@ -24,15 +25,18 @@
import org.jose4j.jwx.JsonWebStructure;
import org.jose4j.keys.resolvers.JwksVerificationKeyResolver;
import org.jose4j.keys.resolvers.VerificationKeyResolver;
-import org.jose4j.lang.JoseException;
import org.jose4j.lang.UnresolvableKeyException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.File;
import java.io.IOException;
-import java.nio.file.Path;
import java.security.Key;
import java.util.List;
+import java.util.Map;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
+import static org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerUtils.validateFileUrl;
/**
* JwksFileVerificationKeyResolver
is a {@link VerificationKeyResolver} implementation
@@ -76,33 +80,28 @@
* Given that the JWKS is referenced by the JWT, the JWKS must be made available by the
* OAuth/OIDC provider so that a JWT can be validated.
*
- * @see org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL
* @see VerificationKeyResolver
*/
-
public class JwksFileVerificationKeyResolver implements CloseableVerificationKeyResolver {
private static final Logger log = LoggerFactory.getLogger(JwksFileVerificationKeyResolver.class);
- private final Path jwksFile;
-
- private VerificationKeyResolver delegate;
-
- public JwksFileVerificationKeyResolver(Path jwksFile) {
- this.jwksFile = jwksFile;
- }
-
- @Override
- public void init() throws IOException {
- log.debug("Starting creation of new VerificationKeyResolver from {}", jwksFile);
- String json = Utils.readFileAsString(jwksFile.toFile().getPath());
+ private final VerificationKeyResolver delegate;
+ public JwksFileVerificationKeyResolver(Map configs, String saslMechanism) {
+ OAuthBearerConfig oauthConfig = new OAuthBearerConfig(configs, saslMechanism);
+ File jwksFile = validateFileUrl(oauthConfig, SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
+ log.debug("Starting creation of new VerificationKeyResolver from {}", jwksFile.getPath());
JsonWebKeySet jwks;
try {
+ String json = Utils.readFileAsString(jwksFile.getPath());
jwks = new JsonWebKeySet(json);
- } catch (JoseException e) {
- throw new IOException(e);
+ } catch (IOException e) {
+ throw new KafkaException("An error occurred loading JWKS data from " + jwksFile.getPath(), e);
+ } catch (Exception e) {
+ throw new KafkaException(e);
}
delegate = new JwksVerificationKeyResolver(jwks.getJsonWebKeys());
@@ -110,10 +109,6 @@ public void init() throws IOException {
@Override
public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException {
- if (delegate == null)
- throw new UnresolvableKeyException("VerificationKeyResolver delegate is null; please call init() first");
-
return delegate.resolveKey(jws, nestingContext);
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGenerator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGenerator.java
new file mode 100644
index 0000000000000..ba51287a3eecd
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtBearerRequestGenerator.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.oauthbearer.AssertionCreator;
+import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate;
+import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException;
+import org.apache.kafka.common.utils.Utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.net.http.HttpRequest;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+public class JwtBearerRequestGenerator implements HttpRequestGenerator {
+
+ public static final String GRANT_TYPE = "urn:ietf:params:oauth:grant-type:jwt-bearer";
+
+ private final URL tokenEndpoint;
+ private final AssertionCreator assertionCreator;
+ private final AssertionJwtTemplate assertionJwtTemplate;
+
+ public JwtBearerRequestGenerator(URL tokenEndpoint,
+ AssertionCreator assertionCreator,
+ AssertionJwtTemplate assertionJwtTemplate) {
+ this.tokenEndpoint = tokenEndpoint;
+ this.assertionCreator = assertionCreator;
+ this.assertionJwtTemplate = assertionJwtTemplate;
+ }
+
+ @Override
+ public String generateBody() {
+ String assertion;
+
+ try {
+ assertion = assertionCreator.create(assertionJwtTemplate);
+ } catch (Exception e) {
+ throw new JwtRetrieverException("Error signing OAuth assertion with private key", e);
+ }
+
+ String encodedGrantType = URLEncoder.encode(GRANT_TYPE, StandardCharsets.UTF_8);
+ String encodedAssertion = URLEncoder.encode(assertion, StandardCharsets.UTF_8);
+ return String.format("grant_type=%s&assertion=%s", encodedGrantType, encodedAssertion);
+ }
+
+ @Override
+ public Map generateHeaders() {
+ Map headers = new HashMap<>();
+ headers.put("Accept", "application/json");
+ headers.put("Cache-Control", "no-cache");
+ headers.put("Content-Type", "application/x-www-form-urlencoded");
+ return headers;
+ }
+
+ @Override
+ public HttpRequest generateRequest() {
+ HttpRequest.BodyPublisher bodyPublisher = HttpRequest.BodyPublishers.ofString(generateBody());
+
+ URI uri;
+
+ try {
+ uri = tokenEndpoint.toURI();
+ } catch (URISyntaxException e) {
+ throw new KafkaException("An error occurred formatting the OAuth token retrieval request", e);
+ }
+
+ HttpRequest.Builder builder = HttpRequest.newBuilder()
+ .uri(uri)
+ .POST(bodyPublisher);
+
+ for (Map.Entry header : generateHeaders().entrySet())
+ builder = builder.header(header.getKey(), header.getValue());
+
+ return builder.build();
+ }
+
+ @Override
+ public void close() {
+ Utils.closeQuietly(assertionCreator, "assertionCreator");
+ Utils.closeQuietly(assertionJwtTemplate, "assertionJwtTemplate");
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpClient.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpClient.java
new file mode 100644
index 0000000000000..0bd855c02f917
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpClient.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.security.oauthbearer.JwtRetriever;
+import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException;
+import org.apache.kafka.common.utils.Time;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.http.HttpClient;
+import java.net.http.HttpRequest;
+import java.net.http.HttpResponse;
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * A {@link JwtRetriever} that will communicate with an OAuth/OIDC provider directly via HTTP.
+ *
+ * @see JwtRetriever
+ */
+public final class JwtHttpClient {
+
+ private static final Logger log = LoggerFactory.getLogger(JwtHttpClient.class);
+
+ public static final Set DEFAULT_SUCCESS_STATUS_CODES;
+
+ public static final Set DEFAULT_FATAL_STATUS_CODES;
+
+ static {
+ DEFAULT_SUCCESS_STATUS_CODES = Set.of(
+ HttpURLConnection.HTTP_OK,
+ HttpURLConnection.HTTP_CREATED
+ );
+
+ // This does not have to be an exhaustive list. There are other HTTP codes that
+ // are defined in different RFCs (e.g. https://datatracker.ietf.org/doc/html/rfc6585)
+ // that we won't worry about yet. The worst case if a status code is missing from
+ // this set is that the request will be retried.
+ DEFAULT_FATAL_STATUS_CODES = Set.of(
+ HttpURLConnection.HTTP_BAD_REQUEST,
+ HttpURLConnection.HTTP_UNAUTHORIZED,
+ HttpURLConnection.HTTP_PAYMENT_REQUIRED,
+ HttpURLConnection.HTTP_FORBIDDEN,
+ HttpURLConnection.HTTP_NOT_FOUND,
+ HttpURLConnection.HTTP_BAD_METHOD,
+ HttpURLConnection.HTTP_NOT_ACCEPTABLE,
+ HttpURLConnection.HTTP_PROXY_AUTH,
+ HttpURLConnection.HTTP_CONFLICT,
+ HttpURLConnection.HTTP_GONE,
+ HttpURLConnection.HTTP_LENGTH_REQUIRED,
+ HttpURLConnection.HTTP_PRECON_FAILED,
+ HttpURLConnection.HTTP_ENTITY_TOO_LARGE,
+ HttpURLConnection.HTTP_REQ_TOO_LONG,
+ HttpURLConnection.HTTP_UNSUPPORTED_TYPE,
+ HttpURLConnection.HTTP_NOT_IMPLEMENTED,
+ HttpURLConnection.HTTP_VERSION
+ );
+ }
+
+ private final Time time;
+ private final Set successStatusCodes;
+ private final Set fatalStatusCodes;
+
+ public JwtHttpClient(Time time) {
+ this(time, DEFAULT_SUCCESS_STATUS_CODES, DEFAULT_FATAL_STATUS_CODES);
+ }
+
+ public JwtHttpClient(Time time,
+ Set successStatusCodes,
+ Set fatalStatusCodes) {
+ this.time = time;
+ this.successStatusCodes = Collections.unmodifiableSet(successStatusCodes);
+ this.fatalStatusCodes = Collections.unmodifiableSet(fatalStatusCodes);
+ }
+
+ public String request(HttpClient client,
+ HttpRequest request,
+ HttpResponse.BodyHandler responseHandler,
+ long retryBackoffMs,
+ long retryBackoffMaxMs) throws JwtRetrieverException {
+ long endMs = time.milliseconds() + retryBackoffMaxMs;
+ int currAttempt = 0;
+
+ while (time.milliseconds() <= endMs) {
+ currAttempt++;
+
+ try {
+ HttpResponse response = client.send(request, responseHandler);
+ int statusCode = response.statusCode();
+
+ if (successStatusCodes.contains(statusCode)) {
+ return response.body();
+ } else if (fatalStatusCodes.contains(statusCode)) {
+ // This is a non-transient error, so don't bother retrying the request unnecessarily.
+ throw new JwtRetrieverException(
+ String.format(
+ "The fatal status code %s was encountered on attempt %s to retrieve the JWT from the OAuth token endpoint",
+ statusCode,
+ currAttempt
+ )
+ );
+ } else {
+ long waitMs = retryBackoffMs * (long) Math.pow(2, currAttempt - 1);
+ long diff = endMs - time.milliseconds();
+ waitMs = Math.min(waitMs, diff);
+
+ if (waitMs <= 0)
+ break;
+
+ log.warn(
+ "The status code {} was encountered on attempt {} to retrieve the JWT from the OAuth token endpoint; sleeping {} ms before attempting again",
+ statusCode,
+ currAttempt,
+ waitMs
+ );
+
+ time.sleep(waitMs);
+ }
+ } catch (IOException | InterruptedException e) {
+ throw new JwtRetrieverException(e);
+ }
+ }
+
+ throw new JwtRetrieverException(
+ String.format(
+ "%s failed attempts were made to retrieve the JWT from the OAuth token endpoint; will not attempt further",
+ currAttempt
+ )
+ );
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpResponseBodyHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpResponseBodyHandler.java
new file mode 100644
index 0000000000000..687784290c635
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/JwtHttpResponseBodyHandler.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.security.oauthbearer.JwtRetrieverException;
+import org.apache.kafka.common.utils.Utils;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.http.HttpResponse;
+import java.util.Collections;
+import java.util.List;
+
+public class JwtHttpResponseBodyHandler implements HttpResponse.BodyHandler {
+
+ public static final List DEFAULT_JSON_PATHS = List.of("/id_token", "/access_token");
+
+ private static final Logger log = LoggerFactory.getLogger(JwtHttpResponseBodyHandler.class);
+ private static final int MAX_RESPONSE_BODY_LENGTH = 1000;
+
+ private final List jsonPaths;
+
+ public JwtHttpResponseBodyHandler() {
+ this(DEFAULT_JSON_PATHS);
+ }
+
+ public JwtHttpResponseBodyHandler(List jsonPaths) {
+ this.jsonPaths = Collections.unmodifiableList(jsonPaths);
+ }
+
+ @Override
+ public HttpResponse.BodySubscriber apply(HttpResponse.ResponseInfo responseInfo) {
+ return HttpResponse.BodySubscribers.mapping(
+ HttpResponse.BodyHandlers.ofString().apply(responseInfo),
+ this::extractJwt
+ );
+ }
+
+ public String extractJwt(String responseBody) throws JwtRetrieverException {
+ ObjectMapper mapper = new ObjectMapper();
+ JsonNode rootNode;
+
+ try {
+ rootNode = mapper.readTree(responseBody);
+ } catch (Exception e) {
+ throw new JwtRetrieverException("An unexpected error occurred parsing the JSON from the OAuth token retrieval response", e);
+ }
+
+ for (String jsonPath : jsonPaths) {
+ JsonNode node = rootNode.at(jsonPath);
+
+ if (node == null) {
+ log.debug("The JSON path {} did not yield a node in the OAuth token retrieval response", jsonPath);
+ continue;
+ }
+
+ String jwt = node.textValue();
+
+ if (Utils.isBlank(jwt)) {
+ log.debug("The JSON path {} yielded a node in the OAuth token retrieval response, but the value was null, blank, or whitespace", jsonPath);
+ continue;
+ }
+
+ return jwt.trim();
+ }
+
+ // Only grab the first N characters so that if the response body is huge, we don't
+ // blow up.
+ String snippet = responseBody;
+
+ if (snippet.length() > MAX_RESPONSE_BODY_LENGTH) {
+ int actualLength = responseBody.length();
+ String s = responseBody.substring(0, MAX_RESPONSE_BODY_LENGTH);
+ snippet = String.format("%s (trimmed to first %d characters out of %d total)", s, MAX_RESPONSE_BODY_LENGTH, actualLength);
+ }
+
+ throw new JwtRetrieverException(String.format("The token endpoint response did not contain a JWT value. Response: (%s)", snippet));
+ }
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerAbstractConfig.java
similarity index 66%
rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java
rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerAbstractConfig.java
index 0a38f2b5094d5..3be0fb2830dad 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Initable.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerAbstractConfig.java
@@ -14,22 +14,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
-import java.io.IOException;
-public interface Initable {
+/**
+ * ConfigurationUtils
is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+public abstract class OAuthBearerAbstractConfig {
- /**
- * Lifecycle method to perform any one-time initialization of the retriever. This must
- * be performed by the caller to ensure the correct state before methods are invoked.
- *
- * @throws IOException Thrown on errors related to IO during initialization
- */
+ public abstract String getString(String key);
- default void init() throws IOException {
- // This method left intentionally blank.
- }
+ public abstract boolean containsKey(String key);
+ public abstract T get(String key);
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfig.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfig.java
new file mode 100644
index 0000000000000..8b972ab10f50b
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerConfig.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.utils.Utils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * OAuthBearerConfig
is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+public class OAuthBearerConfig extends OAuthBearerAbstractConfig {
+
+ private final Map configs;
+
+ private final String prefix;
+
+ public OAuthBearerConfig(Map configs, String saslMechanism) {
+ this.configs = configs;
+
+ if (!Utils.isBlank(saslMechanism))
+ this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
+ else
+ this.prefix = null;
+ }
+
+ public Short getShort(String key) {
+ return (Short) get(key);
+ }
+
+ public Integer getInt(String key) {
+ return (Integer) get(key);
+ }
+
+ public Optional maybeGetInt(String key) {
+ if (containsKey(key))
+ return Optional.of(getInt(key));
+
+ return Optional.empty();
+ }
+
+ public Long getLong(String key) {
+ return (Long) get(key);
+ }
+
+ public Double getDouble(String key) {
+ return (Double) get(key);
+ }
+
+ @SuppressWarnings("unchecked")
+ public List getList(String key) {
+ return (List) get(key);
+ }
+
+ public Boolean getBoolean(String key) {
+ return (Boolean) get(key);
+ }
+
+ public Optional maybeGetBoolean(String key) {
+ if (containsKey(key))
+ return Optional.of(getBoolean(key));
+
+ return Optional.empty();
+ }
+
+ @Override
+ public String getString(String key) {
+ String s = get(key);
+
+ if (Utils.isBlank(s)) {
+ throw new ConfigException("No value was found for the OAuth configuration " + key);
+ } else {
+ return s.trim();
+ }
+ }
+
+ public Optional maybeGetString(String key) {
+ if (containsKey(key))
+ return Optional.of(getString(key));
+
+ return Optional.empty();
+ }
+
+ @Override
+ public boolean containsKey(String key) {
+ return configs.get(key) != null || configs.get(prefix + key) != null;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public T get(String key) {
+ T value = (T) configs.get(prefix + key);
+
+ if (value != null)
+ return value;
+
+ value = (T) configs.get(key);
+
+ if (value != null)
+ return value;
+
+ throw new ConfigException("No value was found for the OAuth configuration " + key);
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerJaasConfig.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerJaasConfig.java
new file mode 100644
index 0000000000000..8b07bb073c5a9
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerJaasConfig.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.utils.Utils;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG;
+
+/**
+ * OAuthBearerJaasConfig
is a utility class to perform logic for the JAAS options and
+ * is separated out here for easier, more direct testing.
+ */
+public class OAuthBearerJaasConfig extends OAuthBearerAbstractConfig {
+
+ private final Map options;
+
+ public OAuthBearerJaasConfig(Map options) {
+ this.options = Collections.unmodifiableMap(options);
+ }
+
+ public Map options() {
+ return options;
+ }
+
+ @Override
+ public String getString(String key) {
+ String s = get(key);
+
+ if (Utils.isBlank(s)) {
+ throw new ConfigException("No value was found for the OAuth option " + key + " in " + SASL_JAAS_CONFIG);
+ } else {
+ return s.trim();
+ }
+ }
+
+ @Override
+ public boolean containsKey(String key) {
+ return options.get(key) != null;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public T get(String key) {
+ T value = (T) options.get(key);
+
+ if (value != null)
+ return value;
+
+ throw new ConfigException("No value was found for the OAuth option " + key + " in " + SASL_JAAS_CONFIG);
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtils.java
new file mode 100644
index 0000000000000..399050e06e4ea
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerUtils.java
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.security.oauthbearer.ClientCredentialsJwtRetriever;
+import org.apache.kafka.common.security.oauthbearer.JwtValidatorException;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerConfigurable;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.utils.Utils;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import javax.security.auth.login.AppConfigurationEntry;
+
+import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE;
+import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
+import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT;
+
+public class OAuthBearerUtils {
+
+ public static Map jaasOptions(String saslMechanism, List jaasConfigEntries) {
+ if (!OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(saslMechanism))
+ throw new ConfigException(String.format("Unexpected SASL mechanism: %s", saslMechanism));
+
+ if (Objects.requireNonNull(jaasConfigEntries).size() != 1 || jaasConfigEntries.get(0) == null)
+ throw new ConfigException(String.format("Must supply exactly 1 non-null JAAS mechanism configuration (size was %d)", jaasConfigEntries.size()));
+
+ return Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+ }
+
+ public static boolean protocolMatches(URL url, String protocol) {
+ return protocol.equalsIgnoreCase(url.getProtocol());
+ }
+
+ public static Optional maybeCreateSslResource(URL url, OAuthBearerJaasConfig jaasConfig) {
+ if (protocolMatches(url, "https")) {
+ Map sslClientConfig = getSslClientConfig(jaasConfig);
+ return Optional.of(SslResource.create(sslClientConfig));
+ } else {
+ return Optional.empty();
+ }
+ }
+
+ public static Map getSslClientConfig(OAuthBearerJaasConfig jaasConfig) {
+ ConfigDef sslConfigDef = new ConfigDef();
+ sslConfigDef.withClientSslSupport();
+ AbstractConfig sslClientConfig = new AbstractConfig(sslConfigDef, jaasConfig.options());
+ return sslClientConfig.values();
+ }
+
+ // visible for testing
+ // make sure the url is in the "org.apache.kafka.sasl.oauthbearer.allowed.urls" system property
+ public static void throwIfURLIsNotAllowed(String value) {
+ Set allowedUrls = Arrays.stream(
+ System.getProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, ALLOWED_SASL_OAUTHBEARER_URLS_DEFAULT).split(","))
+ .map(String::trim)
+ .collect(Collectors.toSet());
+ if (!allowedUrls.contains(value)) {
+ throw new ConfigException(value + " is not allowed. Update system property '"
+ + ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG + "' to allow " + value);
+ }
+ }
+
+ public static T getConfiguredInstanceOrDefault(Map configs,
+ String saslMechanism,
+ List jaasConfigEntries,
+ String configName,
+ Class clazz) {
+ Object classOrClassName = configs.get(configName);
+ Object o;
+
+ if (classOrClassName instanceof String) {
+ try {
+ o = Utils.newInstance((String) classOrClassName, clazz);
+ } catch (ClassNotFoundException e) {
+ throw new KafkaException("Class " + classOrClassName + " cannot be found", e);
+ }
+ } else if (classOrClassName instanceof Class>) {
+ o = Utils.newInstance((Class>) classOrClassName);
+ } else {
+ throw new KafkaException("Unexpected element of type " + classOrClassName.getClass().getName() + ", expected String or Class");
+ }
+
+ if (!clazz.isInstance(o))
+ throw new KafkaException(classOrClassName + " is not an instance of " + clazz.getName());
+
+ try {
+ if (o instanceof OAuthBearerConfigurable)
+ ((OAuthBearerConfigurable) o).configure(configs, saslMechanism, jaasConfigEntries);
+ } catch (Exception e) {
+ Utils.closeQuietly((AutoCloseable) o, "AutoCloseable object constructed and configured during failed call to configure()");
+ throw e;
+ }
+
+ return clazz.cast(o);
+ }
+
+ /**
+ * In some cases, the incoming {@link Map} doesn't contain a value for
+ * {@link SaslConfigs#SASL_OAUTHBEARER_HEADER_URLENCODE}. Returning {@code null} from {@link Map#get(Object)}
+ * will cause a {@link NullPointerException} when it is later unboxed.
+ *
+ *
+ *
+ * This utility method ensures that we have a non-{@code null} value to use in the
+ * {@link ClientCredentialsJwtRetriever} constructor.
+ */
+ public static boolean urlencodeHeader(OAuthBearerConfig oauthConfig) {
+ if (oauthConfig.containsKey(SASL_OAUTHBEARER_HEADER_URLENCODE))
+ return oauthConfig.getBoolean(SASL_OAUTHBEARER_HEADER_URLENCODE);
+ else
+ return DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE;
+ }
+
+ /**
+ * Validates that the scopes are valid, where invalid means any of
+ * the following:
+ *
+ *
+ * - Collection is
null
+ * - Collection has duplicates
+ * - Any of the elements in the collection are
null
+ * - Any of the elements in the collection are zero length
+ * - Any of the elements in the collection are whitespace only
+ *
+ *
+ * @param scopeClaimName Name of the claim used for the scope values
+ * @param scopes Collection of String scopes
+ *
+ * @return Unmodifiable {@link Set} that includes the values of the original set, but with
+ * each value trimmed
+ *
+ * @throws JwtValidatorException Thrown if the value is null
, contains duplicates, or
+ * if any of the values in the set are null
, empty,
+ * or whitespace only
+ */
+
+ public static Set validateClaimScopes(String scopeClaimName, Collection scopes) throws JwtValidatorException {
+ if (scopes == null)
+ throw new JwtValidatorException(String.format("%s value must be non-null", scopeClaimName));
+
+ Set copy = new HashSet<>();
+
+ for (String scope : scopes) {
+ scope = validateClaimValue(scopeClaimName, scope);
+
+ if (copy.contains(scope))
+ throw new JwtValidatorException(String.format("%s value must not contain duplicates - %s already present", scopeClaimName, scope));
+
+ copy.add(scope);
+ }
+
+ return Collections.unmodifiableSet(copy);
+ }
+
+ /**
+ * Validates that the given lifetime is valid, where invalid means any of
+ * the following:
+ *
+ *
+ * null
+ * - Negative
+ *
+ *
+ * @param claimName Name of the claim
+ * @param claimValue Expiration time (in milliseconds)
+ *
+ * @return Input parameter, as provided
+ *
+ * @throws JwtValidatorException Thrown if the value is null
or negative
+ */
+ public static long validateClaimExpiration(String claimName, Long claimValue) throws JwtValidatorException {
+ if (claimValue == null)
+ throw new JwtValidatorException(String.format("%s value must be non-null", claimName));
+
+ if (claimValue < 0)
+ throw new JwtValidatorException(String.format("%s value must be non-negative; value given was \"%s\"", claimName, claimValue));
+
+ return claimValue;
+ }
+
+ /**
+ * Validates that the given claim value is valid, where invalid means any of
+ * the following:
+ *
+ *
+ * null
+ * - Zero length
+ * - Whitespace only
+ *
+ *
+ * @param claimName Name of the claim
+ * @param claimValue Name of the subject
+ *
+ * @return Trimmed version of the claimValue
parameter
+ *
+ * @throws JwtValidatorException Thrown if the value is null
, empty, or whitespace only
+ */
+
+ public static String validateClaimSubject(String claimName, String claimValue) throws JwtValidatorException {
+ return validateClaimValue(claimName, claimValue);
+ }
+
+ /**
+ * Validates that the given issued at claim name is valid, where invalid means any of
+ * the following:
+ *
+ *
+ *
+ * @param claimName Name of the claim
+ * @param claimValue Start time (in milliseconds) or null
if not used
+ *
+ * @return Input parameter, as provided
+ *
+ * @throws JwtValidatorException Thrown if the value is negative
+ */
+
+ public static Long validateClaimIssuedAt(String claimName, Long claimValue) throws JwtValidatorException {
+ if (claimValue != null && claimValue < 0)
+ throw new JwtValidatorException(String.format("%s value must be null or non-negative; value given was \"%s\"", claimName, claimValue));
+
+ return claimValue;
+ }
+
+ /**
+ * Validates that the given claim name override is valid, where invalid means
+ * any of the following:
+ *
+ *
+ * null
+ * - Zero length
+ * - Whitespace only
+ *
+ *
+ * @param name "Standard" name of the claim, e.g. sub
+ * @param value "Override" name of the claim, e.g. email
+ *
+ * @return Trimmed version of the value
parameter
+ *
+ * @throws JwtValidatorException Thrown if the value is null
, empty, or whitespace only
+ */
+
+ public static String validateClaimNameOverride(String name, String value) throws JwtValidatorException {
+ return validateClaimValue(name, value);
+ }
+
+ public static String validateClaimValue(String name, String value) throws JwtValidatorException {
+ if (Utils.isBlank(value))
+ throw new JwtValidatorException(String.format("The value of the OAuth claim %s must be non-null, non-empty, and non-whitespace", name));
+
+ return value.trim();
+ }
+
+ /**
+ * Validates that, if a value is supplied, is a file that:
+ *
+ *
+ *
+ *
+ *
+ *
+ *
+ * If the value is null or an empty string, it is assumed to be an "empty" value and thus.
+ * ignored. Any whitespace is trimmed off of the beginning and end.
+ */
+ public static File validateFile(OAuthBearerAbstractConfig config, String key) {
+ String fileName = config.getString(key);
+ File file = new File(fileName);
+ return validateFile(key, file);
+ }
+
+ /**
+ * Validates that, if a value is supplied, is a file that:
+ *
+ *
+ *
+ *
+ *
+ *
+ *
+ * If the value is null or an empty string, it is assumed to be an "empty" value and thus.
+ * ignored. Any whitespace is trimmed off of the beginning and end.
+ */
+ public static File validateFileUrl(OAuthBearerAbstractConfig config, String key) {
+ URL url = validateUrl(config, key);
+ File file;
+
+ try {
+ file = new File(url.toURI().getRawPath()).getAbsoluteFile();
+ } catch (URISyntaxException e) {
+ throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", key, url, e.getMessage()));
+ }
+
+ return validateFile(key, file);
+ }
+
+ /**
+ * Validates that the file:
+ *
+ *
+ *
+ *
+ *
+ *
+ */
+ public static File validateFile(String key, File file) {
+ if (!file.exists())
+ throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", key, file));
+
+ if (!file.canRead())
+ throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't have read permission", key, file));
+
+ if (file.isDirectory())
+ throw new ConfigException(String.format("The OAuth configuration option %s references a directory (%s), not a file", key, file));
+
+ return file;
+ }
+
+ /**
+ * Validates that the configured URL that:
+ *
+ *
+ * - is well-formed
+ * - contains a scheme
+ * - uses either HTTP, HTTPS, or file protocols
+ * - is in the allow-list
+ *
+ *
+ * No effort is made to connect to the URL in the validation step.
+ */
+ public static URL validateUrl(OAuthBearerAbstractConfig config, String key) {
+ String value = config.getString(key);
+ URL url;
+
+ try {
+ url = new URL(value);
+ } catch (MalformedURLException e) {
+ throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", key, value, e.getMessage()));
+ }
+
+ String protocol = url.getProtocol();
+
+ if (!protocolMatches(url, "https") && !protocolMatches(url, "http") && !protocolMatches(url, "file"))
+ throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that contains an invalid protocol (%s); only \"http\", \"https\", and \"file\" protocol are supported", key, value, protocol));
+
+ throwIfURLIsNotAllowed(value);
+
+ return url;
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefCountingMap.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefCountingMap.java
new file mode 100644
index 0000000000000..20aab1851f53f
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefCountingMap.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.utils.Utils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+
+public class RefCountingMap {
+
+ private final Map values = new HashMap<>();
+
+ private final Map counters = new HashMap<>();
+
+ public synchronized V get(K key, Function super K, ? extends V> mappingFunction) {
+ V value = values.computeIfAbsent(key, mappingFunction);
+ counters.computeIfAbsent(key, k -> new AtomicInteger()).incrementAndGet();
+ return value;
+ }
+
+ public synchronized void release(K key) {
+ int currentCount = counters.computeIfAbsent(key, k -> new AtomicInteger()).decrementAndGet();
+
+ if (currentCount == 0) {
+ V value = values.remove(key);
+
+ // maybeCloseQuietly includes checks for AutoCloseable and null.
+ Utils.maybeCloseQuietly(value, "value for " + key);
+ }
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java
index 62261fed58df8..4306829813799 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwks.java
@@ -14,11 +14,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.oauthbearer.DefaultJwtValidator;
import org.apache.kafka.common.utils.Time;
+import org.jose4j.http.Get;
import org.jose4j.jwk.HttpsJwks;
import org.jose4j.jwk.JsonWebKey;
import org.jose4j.lang.JoseException;
@@ -31,14 +33,15 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
+import java.util.Optional;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.net.ssl.SSLContext;
+
/**
* Implementation of {@link HttpsJwks} that will periodically refresh the JWKS cache to reduce or
* even prevent HTTP/HTTPS traffic in the hot path of validation. It is assumed that it's
@@ -49,15 +52,14 @@
* This instance is created and provided to the
* {@link org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver} that is used when using
* an HTTP-/HTTPS-based {@link org.jose4j.keys.resolvers.VerificationKeyResolver}, which is then
- * provided to the {@link ValidatorAccessTokenValidator} to use in validating the signature of
+ * provided to the {@link DefaultJwtValidator} to use in validating the signature of
* a JWT.
*
* @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
* @see org.jose4j.keys.resolvers.VerificationKeyResolver
- * @see ValidatorAccessTokenValidator
+ * @see DefaultJwtValidator
*/
-
-public final class RefreshingHttpsJwks implements Initable, Closeable {
+public class RefreshingHttpsJwks implements Closeable {
private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
@@ -71,6 +73,8 @@ public final class RefreshingHttpsJwks implements Initable, Closeable {
private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+ private final Time time;
+
/**
* {@link HttpsJwks} does the actual work of contacting the OAuth/OIDC endpoint to get the
* JWKS. In some cases, the call to {@link HttpsJwks#getJsonWebKeys()} will trigger a call
@@ -81,7 +85,7 @@ public final class RefreshingHttpsJwks implements Initable, Closeable {
* perform any operation (directly or indirectly) that could cause blocking. This is because
* the JWKS logic is part of the larger authentication logic which operates on Kafka's network
* thread. It's OK to execute {@link HttpsJwks#getJsonWebKeys()} (which calls
- * {@link HttpsJwks#refresh()}) from within {@link #init()} as that method is called only at
+ * {@link HttpsJwks#refresh()}) from within the constructor as it's called only at
* startup, and we can afford the blocking hit there.
*/
@@ -89,143 +93,97 @@ public final class RefreshingHttpsJwks implements Initable, Closeable {
private final ScheduledExecutorService executorService;
- private final Time time;
-
- private final long refreshMs;
-
private final long refreshRetryBackoffMs;
private final long refreshRetryBackoffMaxMs;
- /**
- * Protects {@link #missingKeyIds} and {@link #jsonWebKeys}.
- */
-
- private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
-
private final Map missingKeyIds;
/**
* Flag to prevent concurrent refresh invocations.
*/
-
private final AtomicBoolean refreshInProgressFlag = new AtomicBoolean(false);
+ /**
+ * Protects {@link #missingKeyIds} and {@link #jsonWebKeys}.
+ */
+ private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
/**
* As mentioned in the comments for {@link #httpsJwks}, we cache the JWKS ourselves so that
* we can return the list immediately without any network I/O. They are only cached within
* calls to {@link #refresh()}.
*/
-
private List jsonWebKeys;
- private boolean isInitialized;
-
- /**
- * Creates a RefreshingHttpsJwks
. It should only be used for testing to pass in a mock executor
- * service. Otherwise the constructor below should be used.
- */
-
- // VisibleForTesting
- RefreshingHttpsJwks(Time time,
- HttpsJwks httpsJwks,
- long refreshMs,
- long refreshRetryBackoffMs,
- long refreshRetryBackoffMaxMs,
- ScheduledExecutorService executorService) {
- if (refreshMs <= 0)
- throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
-
- this.httpsJwks = httpsJwks;
+ public RefreshingHttpsJwks(Time time,
+ HttpsJwks httpsJwks,
+ Optional sslContext,
+ ScheduledExecutorService executorService,
+ long refreshMs,
+ long refreshRetryBackoffMs,
+ long refreshRetryBackoffMaxMs) {
this.time = time;
- this.refreshMs = refreshMs;
+ this.httpsJwks = httpsJwks;
+ this.executorService = executorService;
this.refreshRetryBackoffMs = refreshRetryBackoffMs;
this.refreshRetryBackoffMaxMs = refreshRetryBackoffMaxMs;
- this.executorService = executorService;
this.missingKeyIds = new LinkedHashMap<>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
@Override
protected boolean removeEldestEntry(Map.Entry eldest) {
return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
}
};
- }
- /**
- * Creates a RefreshingHttpsJwks
that will be used by the
- * {@link RefreshingHttpsJwksVerificationKeyResolver} to resolve new key IDs in JWTs.
- *
- * @param time {@link Time} instance
- * @param httpsJwks {@link HttpsJwks} instance from which to retrieve the JWKS
- * based on the OAuth/OIDC standard
- * @param refreshMs The number of milliseconds between refresh passes to connect
- * to the OAuth/OIDC JWKS endpoint to retrieve the latest set
- * @param refreshRetryBackoffMs Time for delay after initial failed attempt to retrieve JWKS
- * @param refreshRetryBackoffMaxMs Maximum time to retrieve JWKS
- */
+ sslContext.ifPresent(c -> {
+ Get get = new Get();
+ get.setSslSocketFactory(c.getSocketFactory());
+ httpsJwks.setSimpleHttpGet(get);
+ });
- public RefreshingHttpsJwks(Time time,
- HttpsJwks httpsJwks,
- long refreshMs,
- long refreshRetryBackoffMs,
- long refreshRetryBackoffMaxMs) {
- this(time, httpsJwks, refreshMs, refreshRetryBackoffMs, refreshRetryBackoffMaxMs, Executors.newSingleThreadScheduledExecutor());
- }
-
- @Override
- public void init() throws IOException {
- try {
- log.debug("init started");
+ this.httpsJwks.setDefaultCacheDuration(refreshMs);
- List localJWKs;
+ List localJWKs;
- try {
- localJWKs = httpsJwks.getJsonWebKeys();
- } catch (JoseException e) {
- throw new IOException("Could not refresh JWKS", e);
- }
-
- try {
- refreshLock.writeLock().lock();
- jsonWebKeys = Collections.unmodifiableList(localJWKs);
- } finally {
- refreshLock.writeLock().unlock();
- }
-
- // Since we just grabbed the keys (which will have invoked a HttpsJwks.refresh()
- // internally), we can delay our first invocation by refreshMs.
- //
- // Note: we refer to this as a _scheduled_ refresh.
- executorService.scheduleAtFixedRate(this::refresh,
- refreshMs,
- refreshMs,
- TimeUnit.MILLISECONDS);
+ try {
+ localJWKs = httpsJwks.getJsonWebKeys();
+ } catch (Exception e) {
+ throw new KafkaException("Could not refresh JWKS", e);
+ }
- log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+ try {
+ refreshLock.writeLock().lock();
+ jsonWebKeys = Collections.unmodifiableList(localJWKs);
} finally {
- isInitialized = true;
-
- log.debug("init completed");
+ refreshLock.writeLock().unlock();
}
+
+ // Since we just grabbed the keys (which will have invoked a HttpsJwks.refresh()
+ // internally), we can delay our first invocation by refreshMs.
+ //
+ // Note: we refer to this as a _scheduled_ refresh.
+ executorService.scheduleAtFixedRate(
+ this::refresh,
+ refreshMs,
+ refreshMs,
+ TimeUnit.MILLISECONDS
+ );
+
+ log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
}
@Override
public void close() {
try {
- log.debug("close started");
-
- try {
- log.debug("JWKS validation key refresh thread shutting down");
- executorService.shutdown();
+ log.debug("JWKS validation key refresh thread shutting down");
+ executorService.shutdown();
- if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
- log.warn("JWKS validation key refresh thread termination did not end after {} {}",
- SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
- }
- } catch (InterruptedException e) {
- log.warn("JWKS validation key refresh thread error during close", e);
+ if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+ log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+ SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
}
- } finally {
- log.debug("close completed");
+ } catch (Exception e) {
+ log.warn("JWKS validation key refresh thread error during close", e);
}
}
@@ -234,7 +192,7 @@ public void close() {
* sometimes called internal to {@link HttpsJwks#getJsonWebKeys()}. We want to avoid any
* blocking I/O as this code is running in the authentication path on the Kafka network thread.
*
- * The list may be stale up to {@link #refreshMs}.
+ * The list may be stale up to the refresh interval.
*
* @return {@link List} of {@link JsonWebKey} instances
* @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
@@ -242,9 +200,6 @@ public void close() {
*/
public List getJsonWebKeys() throws JoseException, IOException {
- if (!isInitialized)
- throw new IllegalStateException("Please call init() first");
-
try {
refreshLock.readLock().lock();
return jsonWebKeys;
@@ -257,6 +212,17 @@ public String getLocation() {
return httpsJwks.getLocation();
}
+ /**
+ * Call the actual refresh implementation that will more than likely issue HTTP(S) calls over the network.
+ */
+ private List refreshJsonWebKeys() throws JoseException, IOException {
+ log.debug("JWKS validation key calling refresh of {} starting", httpsJwks.getLocation());
+ httpsJwks.refresh();
+ List jwks = httpsJwks.getJsonWebKeys();
+ log.debug("JWKS validation key refresh of {} complete", httpsJwks.getLocation());
+ return jwks;
+ }
+
/**
*
* refresh
is an internal method that will refresh the JWKS cache and is
@@ -269,13 +235,12 @@ public String getLocation() {
*
*
*
- * The scheduled refresh is scheduled in {@link #init()} and runs every
- * {@link #refreshMs} milliseconds. An expedited refresh is performed when an
+ * The scheduled refresh is scheduled in the constructor and runs at interval
+ * defined by the refresh configuration. An expedited refresh is performed when an
* incoming JWT refers to a key ID that isn't in our JWKS cache ({@link #jsonWebKeys})
* and we try to perform a refresh sooner than the next scheduled refresh.
*
*/
-
private void refresh() {
if (!refreshInProgressFlag.compareAndSet(false, true)) {
log.debug("OAuth JWKS refresh is already in progress; ignoring concurrent refresh");
@@ -284,20 +249,46 @@ private void refresh() {
try {
log.info("OAuth JWKS refresh of {} starting", httpsJwks.getLocation());
- Retry> retry = new Retry<>(refreshRetryBackoffMs, refreshRetryBackoffMaxMs);
- List localJWKs = retry.execute(() -> {
+
+ List localJWKs = null;
+ long endMs = time.milliseconds() + refreshRetryBackoffMaxMs;
+ int currAttempt = 0;
+
+ while (time.milliseconds() <= endMs) {
+ currAttempt++;
+
try {
- log.debug("JWKS validation key calling refresh of {} starting", httpsJwks.getLocation());
- // Call the *actual* refresh implementation that will more than likely issue
- // HTTP(S) calls over the network.
- httpsJwks.refresh();
- List jwks = httpsJwks.getJsonWebKeys();
- log.debug("JWKS validation key refresh of {} complete", httpsJwks.getLocation());
- return jwks;
- } catch (Exception e) {
- throw new ExecutionException(e);
+ localJWKs = refreshJsonWebKeys();
+ break;
+ } catch (IOException | JoseException e) {
+ long waitMs = refreshRetryBackoffMs * (long) Math.pow(2, currAttempt - 1);
+ long diff = endMs - time.milliseconds();
+ waitMs = Math.min(waitMs, diff);
+
+ if (waitMs <= 0)
+ break;
+
+ log.warn(
+ "An error was encountered on attempt {} to retrieve the OAuth JWKS from {}; sleeping {} ms before attempting again",
+ currAttempt,
+ httpsJwks.getLocation(),
+ waitMs,
+ e
+ );
+
+ time.sleep(waitMs);
}
- });
+ }
+
+ if (localJWKs == null) {
+ log.warn(
+ "{} attempts were made to refresh the OAuth JWKS from {}, but none were successful; not updating local JWKS cache",
+ currAttempt,
+ httpsJwks.getLocation()
+ );
+
+ return;
+ }
try {
refreshLock.writeLock().lock();
@@ -311,8 +302,6 @@ private void refresh() {
}
log.info("OAuth JWKS refresh of {} complete", httpsJwks.getLocation());
- } catch (ExecutionException e) {
- log.warn("OAuth JWKS refresh of {} encountered an error; not updating local JWKS cache", httpsJwks.getLocation(), e);
} finally {
refreshInProgressFlag.set(false);
}
@@ -321,7 +310,7 @@ private void refresh() {
/**
*
* maybeExpediteRefresh
is a public method that will trigger a refresh of
- * the JWKS cache if all of the following conditions are met:
+ * the JWKS cache if all the following conditions are met:
*
*
* - The given
keyId
parameter is <e; the
@@ -336,7 +325,6 @@ private void refresh() {
* @param keyId JWT key ID
* @return true
if an expedited refresh was scheduled, false
otherwise
*/
-
public boolean maybeExpediteRefresh(String keyId) {
if (keyId.length() > MISSING_KEY_ID_MAX_KEY_LENGTH) {
// Although there's no limit on the length of the key ID, they're generally
@@ -375,5 +363,4 @@ public boolean maybeExpediteRefresh(String keyId) {
}
}
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java
index 52d0c6c39785f..b9a4c518cdec5 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/RefreshingHttpsJwksVerificationKeyResolver.java
@@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.utils.Utils;
+
import org.jose4j.jwk.HttpsJwks;
import org.jose4j.jwk.JsonWebKey;
import org.jose4j.jwk.VerificationJwkSelector;
@@ -80,51 +81,26 @@
* @see RefreshingHttpsJwks
* @see HttpsJwks
*/
-
public class RefreshingHttpsJwksVerificationKeyResolver implements CloseableVerificationKeyResolver {
private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwksVerificationKeyResolver.class);
- private final RefreshingHttpsJwks refreshingHttpsJwks;
-
private final VerificationJwkSelector verificationJwkSelector;
- private boolean isInitialized;
+ private final RefreshingHttpsJwks refreshingHttpsJwks;
public RefreshingHttpsJwksVerificationKeyResolver(RefreshingHttpsJwks refreshingHttpsJwks) {
this.refreshingHttpsJwks = refreshingHttpsJwks;
this.verificationJwkSelector = new VerificationJwkSelector();
}
- @Override
- public void init() throws IOException {
- try {
- log.debug("init started");
-
- refreshingHttpsJwks.init();
- } finally {
- isInitialized = true;
-
- log.debug("init completed");
- }
- }
-
@Override
public void close() {
- try {
- log.debug("close started");
-
- refreshingHttpsJwks.close();
- } finally {
- log.debug("close completed");
- }
+ Utils.closeQuietly(refreshingHttpsJwks, "refreshingHttpsJwks");
}
@Override
public Key resolveKey(JsonWebSignature jws, List nestingContext) throws UnresolvableKeyException {
- if (!isInitialized)
- throw new IllegalStateException("Please call init() first");
-
try {
List jwks = refreshingHttpsJwks.getJsonWebKeys();
JsonWebKey jwk = verificationJwkSelector.select(jws, jwks);
@@ -148,5 +124,4 @@ public Key resolveKey(JsonWebSignature jws, List nestingContex
throw new UnresolvableKeyException(sb, e);
}
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retry.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retry.java
deleted file mode 100644
index 0da92e4fc7f72..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retry.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.utils.Time;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ExecutionException;
-
-/**
- * Retry encapsulates the mechanism to perform a retry and then exponential
- * backoff using provided wait times between attempts.
- *
- * @param Result type
- */
-
-public class Retry {
-
- private static final Logger log = LoggerFactory.getLogger(Retry.class);
-
- private final Time time;
-
- private final long retryBackoffMs;
-
- private final long retryBackoffMaxMs;
-
- public Retry(long retryBackoffMs, long retryBackoffMaxMs) {
- this(Time.SYSTEM, retryBackoffMs, retryBackoffMaxMs);
- }
-
- public Retry(Time time, long retryBackoffMs, long retryBackoffMaxMs) {
- this.time = time;
- this.retryBackoffMs = retryBackoffMs;
- this.retryBackoffMaxMs = retryBackoffMaxMs;
-
- if (this.retryBackoffMs < 0)
- throw new IllegalArgumentException(String.format("retryBackoffMs value (%d) must be non-negative", retryBackoffMs));
-
- if (this.retryBackoffMaxMs < 0)
- throw new IllegalArgumentException(String.format("retryBackoffMaxMs value (%d) must be non-negative", retryBackoffMaxMs));
-
- if (this.retryBackoffMaxMs < this.retryBackoffMs)
- throw new IllegalArgumentException(String.format("retryBackoffMaxMs value (%d) is less than retryBackoffMs value (%d)", retryBackoffMaxMs, retryBackoffMs));
- }
-
- public R execute(Retryable retryable) throws ExecutionException {
- long endMs = time.milliseconds() + retryBackoffMaxMs;
- int currAttempt = 0;
- ExecutionException error = null;
-
- while (time.milliseconds() <= endMs) {
- currAttempt++;
-
- try {
- return retryable.call();
- } catch (UnretryableException e) {
- // We've deemed this error to not be worth retrying, so collect the error and
- // fail immediately.
- if (error == null)
- error = new ExecutionException(e);
-
- break;
- } catch (ExecutionException e) {
- log.warn("Error during retry attempt {}", currAttempt, e);
-
- if (error == null)
- error = e;
-
- long waitMs = retryBackoffMs * (long) Math.pow(2, currAttempt - 1);
- long diff = endMs - time.milliseconds();
- waitMs = Math.min(waitMs, diff);
-
- if (waitMs <= 0)
- break;
-
- String message = String.format("Attempt %d to make call resulted in an error; sleeping %d ms before retrying",
- currAttempt, waitMs);
- log.warn(message, e);
-
- time.sleep(waitMs);
- }
- }
-
- if (error == null)
- // Really shouldn't ever get to here, but...
- error = new ExecutionException(new IllegalStateException("Exhausted all retry attempts but no attempt returned value or encountered exception"));
-
- throw error;
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java
index f45865fa63848..2cc8b8ae69940 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SerializedJwt.java
@@ -14,14 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.oauthbearer.JwtValidatorException;
+
/**
* SerializedJwt provides a modicum of structure and validation around a JWT's serialized form by
* splitting and making the three sections (header, payload, and signature) available to the user.
*/
-
public class SerializedJwt {
private final String token;
@@ -39,12 +39,12 @@ public SerializedJwt(String token) {
token = token.trim();
if (token.isEmpty())
- throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)");
+ throw new JwtValidatorException("Malformed JWT provided; expected three sections (header, payload, and signature)");
String[] splits = token.split("\\.");
if (splits.length != 3)
- throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)");
+ throw new JwtValidatorException("Malformed JWT provided; expected three sections (header, payload, and signature)");
this.token = token.trim();
this.header = validateSection(splits[0]);
@@ -92,13 +92,12 @@ public String getSignature() {
return signature;
}
- private String validateSection(String section) throws ValidateException {
+ private String validateSection(String section) throws JwtValidatorException {
section = section.trim();
if (section.isEmpty())
- throw new ValidateException("Malformed JWT provided; expected three sections (header, payload, and signature)");
+ throw new JwtValidatorException("Malformed JWT provided; expected three sections (header, payload, and signature)");
return section;
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SslResource.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SslResource.java
new file mode 100644
index 0000000000000..4de532bf5937b
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/SslResource.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer.internals.secured;
+
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.network.ConnectionMode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.apache.kafka.common.utils.Utils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+
+import javax.net.ssl.SSLContext;
+
+/**
+ * {@code SslResource} couples the {@link SslFactory} and {@link SSLContext} so that
+ * {@link #sslFactory} can be properly {@link SslFactory#close() closed} during closing of the overall
+ * OAuth login/validation module. The {@link SSLContext} API is what the HTTP clients use, so the two
+ * need to be kept closely together.
+ */
+public class SslResource implements Closeable {
+
+ private final SslFactory sslFactory;
+
+ private final SSLContext sslContext;
+
+ public SslResource(SslFactory sslFactory, SSLContext sslContext) {
+ this.sslFactory = sslFactory;
+ this.sslContext = sslContext;
+ }
+
+ public static SslResource create(Map configs) {
+ SslFactory sslFactory = new SslFactory(ConnectionMode.CLIENT);
+ sslFactory.configure(configs);
+
+ if (!((sslFactory.sslEngineFactory()) instanceof DefaultSslEngineFactory)) {
+ String message = String.format(
+ "The OAuth %s configuration includes a custom SSL factory class (%s) which is not a supported JAAS option for OAuth",
+ SaslConfigs.SASL_JAAS_CONFIG,
+ SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG
+ );
+ throw new ConfigException(message);
+ }
+
+ SSLContext sslContext = ((DefaultSslEngineFactory) sslFactory.sslEngineFactory()).sslContext();
+ return new SslResource(sslFactory, sslContext);
+ }
+
+ public SSLContext sslContext() {
+ return sslContext;
+ }
+
+ @Override
+ public void close() throws IOException {
+ Utils.closeQuietly(sslFactory, "sslFactory");
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retryable.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/StaticAssertionJwtTemplate.java
similarity index 54%
rename from clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retryable.java
rename to clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/StaticAssertionJwtTemplate.java
index 46752f3ea6020..9dede2ad32c3f 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/Retryable.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/StaticAssertionJwtTemplate.java
@@ -14,32 +14,28 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security.oauthbearer.internals.secured;
-import java.util.concurrent.ExecutionException;
+import org.apache.kafka.common.security.oauthbearer.AssertionJwtTemplate;
-/**
- * Simple interface to abstract out the call that is made so that it can be retried.
- *
- * @param Result type
- *
- * @see Retry
- * @see UnretryableException
- */
+import java.util.Collections;
+import java.util.Map;
+
+public class StaticAssertionJwtTemplate implements AssertionJwtTemplate {
-public interface Retryable {
+ private final Map payload;
- /**
- * Perform the operation and return the data from the response.
- *
- * @return Return response data, formatted in the given data type
- *
- * @throws ExecutionException Thrown on errors connecting, writing, reading, timeouts, etc.
- * that can likely be tried again
- * @throws UnretryableException Thrown on errors that we can determine should not be tried again
- */
+ public StaticAssertionJwtTemplate(Map payload) {
+ this.payload = Collections.unmodifiableMap(payload);
+ }
- R call() throws ExecutionException, UnretryableException;
+ @Override
+ public Map header() {
+ return Collections.emptyMap();
+ }
+ @Override
+ public Map payload() {
+ return payload;
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidator.java
deleted file mode 100644
index c7ae8edae9d93..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidator.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
-
-import org.jose4j.jwt.JwtClaims;
-import org.jose4j.jwt.MalformedClaimException;
-import org.jose4j.jwt.NumericDate;
-import org.jose4j.jwt.ReservedClaimNames;
-import org.jose4j.jwt.consumer.InvalidJwtException;
-import org.jose4j.jwt.consumer.JwtConsumer;
-import org.jose4j.jwt.consumer.JwtConsumerBuilder;
-import org.jose4j.jwt.consumer.JwtContext;
-import org.jose4j.keys.resolvers.VerificationKeyResolver;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Set;
-
-import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
-
-/**
- * ValidatorAccessTokenValidator is an implementation of {@link AccessTokenValidator} that is used
- * by the broker to perform more extensive validation of the JWT access token that is received
- * from the client, but ultimately from posting the client credentials to the OAuth/OIDC provider's
- * token endpoint.
- *
- * The validation steps performed (primary by the jose4j library) are:
- *
- *
- * -
- * Basic structural validation of the
b64token
value as defined in
- * RFC 6750 Section 2.1
- *
- * - Basic conversion of the token into an in-memory data structure
- * -
- * Presence of scope,
exp
, subject, iss
, and
- * iat
claims
- *
- * -
- * Signature matching validation against the
kid
and those provided by
- * the OAuth/OIDC provider's JWKS
- *
- *
- */
-
-public class ValidatorAccessTokenValidator implements AccessTokenValidator {
-
- private static final Logger log = LoggerFactory.getLogger(ValidatorAccessTokenValidator.class);
-
- private final JwtConsumer jwtConsumer;
-
- private final String scopeClaimName;
-
- private final String subClaimName;
-
- /**
- * Creates a new ValidatorAccessTokenValidator that will be used by the broker for more
- * thorough validation of the JWT.
- *
- * @param clockSkew The optional value (in seconds) to allow for differences
- * between the time of the OAuth/OIDC identity provider and
- * the broker. If null
is provided, the broker
- * and the OAUth/OIDC identity provider are assumed to have
- * very close clock settings.
- * @param expectedAudiences The (optional) set the broker will use to verify that
- * the JWT was issued for one of the expected audiences.
- * The JWT will be inspected for the standard OAuth
- * aud
claim and if this value is set, the
- * broker will match the value from JWT's aud
- * claim to see if there is an exact match. If there is no
- * match, the broker will reject the JWT and authentication
- * will fail. May be null
to not perform any
- * check to verify the JWT's aud
claim matches any
- * fixed set of known/expected audiences.
- * @param expectedIssuer The (optional) value for the broker to use to verify that
- * the JWT was created by the expected issuer. The JWT will
- * be inspected for the standard OAuth iss
claim
- * and if this value is set, the broker will match it
- * exactly against what is in the JWT's iss
- * claim. If there is no match, the broker will reject the JWT
- * and authentication will fail. May be null
to not
- * perform any check to verify the JWT's iss
claim
- * matches a specific issuer.
- * @param verificationKeyResolver jose4j-based {@link VerificationKeyResolver} that is used
- * to validate the signature matches the contents of the header
- * and payload
- * @param scopeClaimName Name of the scope claim to use; must be non-null
- * @param subClaimName Name of the subject claim to use; must be
- * non-null
- *
- * @see JwtConsumerBuilder
- * @see JwtConsumer
- * @see VerificationKeyResolver
- */
-
- public ValidatorAccessTokenValidator(Integer clockSkew,
- Set expectedAudiences,
- String expectedIssuer,
- VerificationKeyResolver verificationKeyResolver,
- String scopeClaimName,
- String subClaimName) {
- final JwtConsumerBuilder jwtConsumerBuilder = new JwtConsumerBuilder();
-
- if (clockSkew != null)
- jwtConsumerBuilder.setAllowedClockSkewInSeconds(clockSkew);
-
- if (expectedAudiences != null && !expectedAudiences.isEmpty())
- jwtConsumerBuilder.setExpectedAudience(expectedAudiences.toArray(new String[0]));
-
- if (expectedIssuer != null)
- jwtConsumerBuilder.setExpectedIssuer(expectedIssuer);
-
- this.jwtConsumer = jwtConsumerBuilder
- .setJwsAlgorithmConstraints(DISALLOW_NONE)
- .setRequireExpirationTime()
- .setRequireIssuedAt()
- .setVerificationKeyResolver(verificationKeyResolver)
- .build();
- this.scopeClaimName = scopeClaimName;
- this.subClaimName = subClaimName;
- }
-
- /**
- * Accepts an OAuth JWT access token in base-64 encoded format, validates, and returns an
- * OAuthBearerToken.
- *
- * @param accessToken Non-null
JWT access token
- * @return {@link OAuthBearerToken}
- * @throws ValidateException Thrown on errors performing validation of given token
- */
-
- @SuppressWarnings("unchecked")
- public OAuthBearerToken validate(String accessToken) throws ValidateException {
- SerializedJwt serializedJwt = new SerializedJwt(accessToken);
-
- JwtContext jwt;
-
- try {
- jwt = jwtConsumer.process(serializedJwt.getToken());
- } catch (InvalidJwtException e) {
- throw new ValidateException(String.format("Could not validate the access token: %s", e.getMessage()), e);
- }
-
- JwtClaims claims = jwt.getJwtClaims();
-
- Object scopeRaw = getClaim(() -> claims.getClaimValue(scopeClaimName), scopeClaimName);
- Collection scopeRawCollection;
-
- if (scopeRaw instanceof String)
- scopeRawCollection = Collections.singletonList((String) scopeRaw);
- else if (scopeRaw instanceof Collection)
- scopeRawCollection = (Collection) scopeRaw;
- else
- scopeRawCollection = Collections.emptySet();
-
- NumericDate expirationRaw = getClaim(claims::getExpirationTime, ReservedClaimNames.EXPIRATION_TIME);
- String subRaw = getClaim(() -> claims.getStringClaimValue(subClaimName), subClaimName);
- NumericDate issuedAtRaw = getClaim(claims::getIssuedAt, ReservedClaimNames.ISSUED_AT);
-
- Set scopes = ClaimValidationUtils.validateScopes(scopeClaimName, scopeRawCollection);
- long expiration = ClaimValidationUtils.validateExpiration(ReservedClaimNames.EXPIRATION_TIME,
- expirationRaw != null ? expirationRaw.getValueInMillis() : null);
- String sub = ClaimValidationUtils.validateSubject(subClaimName, subRaw);
- Long issuedAt = ClaimValidationUtils.validateIssuedAt(ReservedClaimNames.ISSUED_AT,
- issuedAtRaw != null ? issuedAtRaw.getValueInMillis() : null);
-
- return new BasicOAuthBearerToken(accessToken,
- scopes,
- expiration,
- sub,
- issuedAt);
- }
-
- private T getClaim(ClaimSupplier supplier, String claimName) throws ValidateException {
- try {
- T value = supplier.get();
- log.debug("getClaim - {}: {}", claimName, value);
- return value;
- } catch (MalformedClaimException e) {
- throw new ValidateException(String.format("Could not extract the '%s' claim from the access token", claimName), e);
- }
- }
-
- public interface ClaimSupplier {
-
- T get() throws MalformedClaimException;
-
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java
deleted file mode 100644
index 0422045fc029d..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/VerificationKeyResolverFactory.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.utils.Time;
-
-import org.jose4j.http.Get;
-import org.jose4j.jwk.HttpsJwks;
-
-import java.net.URL;
-import java.nio.file.Path;
-import java.util.Locale;
-import java.util.Map;
-
-import javax.net.ssl.SSLSocketFactory;
-
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS;
-import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
-
-public class VerificationKeyResolverFactory {
-
- /**
- * Create an {@link AccessTokenRetriever} from the given
- * {@link org.apache.kafka.common.config.SaslConfigs}.
- *
- * Note: the returned CloseableVerificationKeyResolver
is not
- * initialized here and must be done by the caller.
- *
- * Primarily exposed here for unit testing.
- *
- * @param configs SASL configuration
- *
- * @return Non-null
{@link CloseableVerificationKeyResolver}
- */
- public static CloseableVerificationKeyResolver create(Map configs,
- Map jaasConfig) {
- return create(configs, null, jaasConfig);
- }
-
- public static CloseableVerificationKeyResolver create(Map configs,
- String saslMechanism,
- Map jaasConfig) {
- ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
- URL jwksEndpointUrl = cu.validateUrl(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
-
- if (jwksEndpointUrl.getProtocol().toLowerCase(Locale.ROOT).equals("file")) {
- Path p = cu.validateFile(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL);
- return new JwksFileVerificationKeyResolver(p);
- } else {
- long refreshIntervalMs = cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, true, 0L);
- JaasOptionsUtils jou = new JaasOptionsUtils(jaasConfig);
- SSLSocketFactory sslSocketFactory = null;
-
- if (jou.shouldCreateSSLSocketFactory(jwksEndpointUrl))
- sslSocketFactory = jou.createSSLSocketFactory();
-
- HttpsJwks httpsJwks = new HttpsJwks(jwksEndpointUrl.toString());
- httpsJwks.setDefaultCacheDuration(refreshIntervalMs);
-
- if (sslSocketFactory != null) {
- Get get = new Get();
- get.setSslSocketFactory(sslSocketFactory);
- httpsJwks.setSimpleHttpGet(get);
- }
-
- RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(Time.SYSTEM,
- httpsJwks,
- refreshIntervalMs,
- cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS),
- cu.validateLong(SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS));
- return new RefreshingHttpsJwksVerificationKeyResolver(refreshingHttpsJwks);
- }
- }
-
-}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidatorTest.java
similarity index 58%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidatorTest.java
index 4db20e9ee10d6..7f2ce543e9922 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ValidatorAccessTokenValidatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidatorTest.java
@@ -14,30 +14,44 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+package org.apache.kafka.common.security.oauthbearer;
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
-
-import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver;
+import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder;
import org.jose4j.jwk.PublicJsonWebKey;
import org.jose4j.jws.AlgorithmIdentifiers;
import org.jose4j.lang.InvalidAlgorithmException;
import org.junit.jupiter.api.Test;
+import java.security.Key;
import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
-public class ValidatorAccessTokenValidatorTest extends AccessTokenValidatorTest {
+public class BrokerJwtValidatorTest extends JwtValidatorTest {
@Override
- protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) {
- return new ValidatorAccessTokenValidator(30,
- Collections.emptySet(),
- null,
- (jws, nestingContext) -> builder.jwk().getKey(),
- builder.scopeClaimName(),
- builder.subjectClaimName());
+ protected JwtValidator createValidator(JwtBuilder builder) throws Exception {
+ Key key = builder.jwk() != null ? builder.jwk().getKey() : null;
+ CloseableVerificationKeyResolver keyResolver = mock(CloseableVerificationKeyResolver.class);
+ when(keyResolver.resolveKey(any(), any())).thenReturn(key);
+
+ return new BrokerJwtValidator() {
+ @Override
+ public void configure(Map configs, String saslMechanism, List jaasConfigEntries) {
+ super.configure(keyResolver, configs, saslMechanism);
+ }
+ };
}
@Test
@@ -66,13 +80,15 @@ public void testMissingSubShouldBeValid() throws Exception {
String subClaimName = "client_id";
String subject = "otherSub";
PublicJsonWebKey jwk = createRsaJwk();
- AccessTokenBuilder tokenBuilder = new AccessTokenBuilder()
+ JwtBuilder tokenBuilder = new JwtBuilder()
.jwk(jwk)
.alg(AlgorithmIdentifiers.RSA_USING_SHA256)
.addCustomClaim(subClaimName, subject)
.subjectClaimName(subClaimName)
.subject(null);
- AccessTokenValidator validator = createAccessTokenValidator(tokenBuilder);
+ JwtValidator validator = createValidator(tokenBuilder);
+ Map configs = Collections.singletonMap(SASL_OAUTHBEARER_SUB_CLAIM_NAME, tokenBuilder.subjectClaimName());
+ validator.configure(getSaslConfigs(configs), OAUTHBEARER_MECHANISM, List.of());
// Validation should succeed (e.g. signature verification) even if sub claim is missing
OAuthBearerToken token = validator.validate(tokenBuilder.build());
@@ -81,10 +97,11 @@ public void testMissingSubShouldBeValid() throws Exception {
}
private void testEncryptionAlgorithm(PublicJsonWebKey jwk, String alg) throws Exception {
- AccessTokenBuilder builder = new AccessTokenBuilder().jwk(jwk).alg(alg);
- AccessTokenValidator validator = createAccessTokenValidator(builder);
- String accessToken = builder.build();
- OAuthBearerToken token = validator.validate(accessToken);
+ JwtBuilder builder = new JwtBuilder().jwk(jwk).alg(alg);
+ JwtValidator validator = createValidator(builder);
+ validator.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
+ String jwt = builder.build();
+ OAuthBearerToken token = validator.validate(jwt);
assertEquals(builder.subject(), token.principalName());
assertEquals(builder.issuedAtSeconds() * 1000, token.startTimeMs());
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.java
similarity index 70%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.java
index fc2e3d2a2e83a..72c925fe2f9c0 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/LoginAccessTokenValidatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/ClientJwtValidatorTest.java
@@ -14,14 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+package org.apache.kafka.common.security.oauthbearer;
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder;
-public class LoginAccessTokenValidatorTest extends AccessTokenValidatorTest {
+public class ClientJwtValidatorTest extends JwtValidatorTest {
@Override
- protected AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder builder) {
- return new LoginAccessTokenValidator(builder.scopeClaimName(), builder.subjectClaimName());
+ protected JwtValidator createValidator(JwtBuilder builder) {
+ return new ClientJwtValidator();
}
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java
new file mode 100644
index 0000000000000..4d971af44a20e
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/DefaultJwtRetrieverTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer;
+
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.test.TestUtils;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
+import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class DefaultJwtRetrieverTest extends OAuthBearerTest {
+
+ @AfterEach
+ public void tearDown() throws Exception {
+ System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG);
+ }
+
+ @Test
+ public void testConfigureRefreshingFileJwtRetriever() throws Exception {
+ String expected = "{}";
+
+ File jwtFile = createTempFile("jwt-", ".json", expected);
+
+ System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, jwtFile.toURI().toString());
+ Map configs = Collections.singletonMap(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString());
+
+ try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) {
+ jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of());
+ assertEquals(expected, jwtRetriever.retrieve());
+ }
+ }
+
+ @Test
+ public void testConfigureRefreshingFileJwtRetrieverWithInvalidDirectory() {
+ // Should fail because the parent path doesn't exist.
+ String file = new File("/tmp/this-directory-does-not-exist/foo.json").toURI().toString();
+ System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, file);
+ Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, file);
+
+ try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) {
+ assertThrowsWithMessage(ConfigException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), "that doesn't exist");
+ }
+ }
+
+ @Test
+ public void testConfigureRefreshingFileJwtRetrieverWithInvalidFile() throws Exception {
+ // Should fail because while the parent path exists, the file itself doesn't.
+ File jwtFile = new File(TestUtils.tempDirectory(), "this-file-does-not-exist.json");
+ System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, jwtFile.toURI().toString());
+ Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString());
+
+ try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) {
+ assertThrowsWithMessage(ConfigException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), "that doesn't exist");
+ }
+ }
+
+ @Test
+ public void testSaslOauthbearerTokenEndpointUrlIsNotAllowed() throws Exception {
+ // Should fail if the URL is not allowed
+ File jwtFile = new File("not_allowed.json");
+ Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString());
+
+ try (JwtRetriever jwtRetriever = new DefaultJwtRetriever()) {
+ assertThrowsWithMessage(ConfigException.class, () -> jwtRetriever.configure(configs, OAUTHBEARER_MECHANISM, List.of()), ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG);
+ }
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetrieverTest.java
new file mode 100644
index 0000000000000..3df6155f6f12e
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/FileJwtRetrieverTest.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.oauthbearer;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.login.AppConfigurationEntry;
+
+import static javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag.OPTIONAL;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
+import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class FileJwtRetrieverTest extends OAuthBearerTest {
+
+ @Test
+ public void testFileTokenRetrieverHandlesNewline() throws Exception {
+ Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+ long cur = cal.getTimeInMillis() / 1000;
+ String exp = "" + (cur + 60 * 60); // 1 hour in future
+ String iat = "" + cur;
+
+ String expected = createAccessKey("{}", String.format("{\"exp\":%s, \"iat\":%s, \"sub\":\"subj\"}", exp, iat), "sign");
+ String withNewline = expected + "\n";
+
+ File jwtFile = createTempFile("jwt-", ".json", withNewline);
+
+ List jaasConfigEntries = new ArrayList<>();
+ jaasConfigEntries.add(new AppConfigurationEntry("dummy", OPTIONAL, Collections.emptyMap()));
+
+ System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, jwtFile.toURI().toString());
+ Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString());
+
+ try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler()) {
+ handler.configure(configs, OAUTHBEARER_MECHANISM, jaasConfigEntries);
+ OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+ handler.handle(new Callback[]{callback});
+ assertEquals(callback.token().value(), expected);
+ }
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorTest.java
similarity index 50%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorTest.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorTest.java
index 0adaf34bbbeea..0d67a4e81d73d 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenValidatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/JwtValidatorTest.java
@@ -14,8 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+package org.apache.kafka.common.security.oauthbearer;
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
+import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder;
import org.jose4j.jws.AlgorithmIdentifiers;
import org.jose4j.jwx.HeaderParameterNames;
@@ -23,70 +24,74 @@
import org.junit.jupiter.api.TestInstance;
import org.junit.jupiter.api.TestInstance.Lifecycle;
+import java.util.List;
+
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertThrows;
@TestInstance(Lifecycle.PER_CLASS)
-public abstract class AccessTokenValidatorTest extends OAuthBearerTest {
+public abstract class JwtValidatorTest extends OAuthBearerTest {
- protected abstract AccessTokenValidator createAccessTokenValidator(AccessTokenBuilder accessTokenBuilder) throws Exception;
+ protected abstract JwtValidator createValidator(JwtBuilder jwtBuilder) throws Exception;
- protected AccessTokenValidator createAccessTokenValidator() throws Exception {
- AccessTokenBuilder builder = new AccessTokenBuilder();
- return createAccessTokenValidator(builder);
+ protected JwtValidator createValidator() throws Exception {
+ JwtBuilder builder = new JwtBuilder();
+ JwtValidator validator = createValidator(builder);
+ validator.configure(getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
+ return validator;
}
@Test
public void testNull() throws Exception {
- AccessTokenValidator validator = createAccessTokenValidator();
- assertThrowsWithMessage(ValidateException.class, () -> validator.validate(null), "Malformed JWT provided; expected three sections (header, payload, and signature)");
+ JwtValidator validator = createValidator();
+ assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(null), "Malformed JWT provided; expected three sections (header, payload, and signature)");
}
@Test
public void testEmptyString() throws Exception {
- AccessTokenValidator validator = createAccessTokenValidator();
- assertThrowsWithMessage(ValidateException.class, () -> validator.validate(""), "Malformed JWT provided; expected three sections (header, payload, and signature)");
+ JwtValidator validator = createValidator();
+ assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(""), "Malformed JWT provided; expected three sections (header, payload, and signature)");
}
@Test
public void testWhitespace() throws Exception {
- AccessTokenValidator validator = createAccessTokenValidator();
- assertThrowsWithMessage(ValidateException.class, () -> validator.validate(" "), "Malformed JWT provided; expected three sections (header, payload, and signature)");
+ JwtValidator validator = createValidator();
+ assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(" "), "Malformed JWT provided; expected three sections (header, payload, and signature)");
}
@Test
public void testEmptySections() throws Exception {
- AccessTokenValidator validator = createAccessTokenValidator();
- assertThrowsWithMessage(ValidateException.class, () -> validator.validate(".."), "Malformed JWT provided; expected three sections (header, payload, and signature)");
+ JwtValidator validator = createValidator();
+ assertThrowsWithMessage(JwtValidatorException.class, () -> validator.validate(".."), "Malformed JWT provided; expected three sections (header, payload, and signature)");
}
@Test
public void testMissingHeader() throws Exception {
- AccessTokenValidator validator = createAccessTokenValidator();
+ JwtValidator validator = createValidator();
String header = "";
String payload = createBase64JsonJwtSection(node -> { });
String signature = "";
- String accessToken = String.format("%s.%s.%s", header, payload, signature);
- assertThrows(ValidateException.class, () -> validator.validate(accessToken));
+ String jwt = String.format("%s.%s.%s", header, payload, signature);
+ assertThrows(JwtValidatorException.class, () -> validator.validate(jwt));
}
@Test
public void testMissingPayload() throws Exception {
- AccessTokenValidator validator = createAccessTokenValidator();
+ JwtValidator validator = createValidator();
String header = createBase64JsonJwtSection(node -> node.put(HeaderParameterNames.ALGORITHM, AlgorithmIdentifiers.NONE));
String payload = "";
String signature = "";
- String accessToken = String.format("%s.%s.%s", header, payload, signature);
- assertThrows(ValidateException.class, () -> validator.validate(accessToken));
+ String jwt = String.format("%s.%s.%s", header, payload, signature);
+ assertThrows(JwtValidatorException.class, () -> validator.validate(jwt));
}
@Test
public void testMissingSignature() throws Exception {
- AccessTokenValidator validator = createAccessTokenValidator();
+ JwtValidator validator = createValidator();
String header = createBase64JsonJwtSection(node -> node.put(HeaderParameterNames.ALGORITHM, AlgorithmIdentifiers.NONE));
String payload = createBase64JsonJwtSection(node -> { });
String signature = "";
- String accessToken = String.format("%s.%s.%s", header, payload, signature);
- assertThrows(ValidateException.class, () -> validator.validate(accessToken));
+ String jwt = String.format("%s.%s.%s", header, payload, signature);
+ assertThrows(JwtValidatorException.class, () -> validator.validate(jwt));
}
-
}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java
index 5b1b2976662b6..4d1848ca2b2da 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java
@@ -17,17 +17,11 @@
package org.apache.kafka.common.security.oauthbearer;
+import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenRetriever;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidator;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidatorFactory;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.FileTokenRetriever;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.HttpAccessTokenRetriever;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest;
-import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder;
import org.jose4j.jws.AlgorithmIdentifiers;
import org.junit.jupiter.api.AfterEach;
@@ -35,29 +29,31 @@
import java.io.File;
import java.io.IOException;
-import java.util.Base64;
-import java.util.Calendar;
import java.util.Collections;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
-import java.util.TimeZone;
import javax.security.auth.callback.Callback;
import javax.security.auth.callback.UnsupportedCallbackException;
import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
import static org.apache.kafka.common.config.internals.BrokerSecurityConfigs.ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
-import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest {
+
@AfterEach
public void tearDown() throws Exception {
System.clearProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG);
@@ -65,27 +61,23 @@ public void tearDown() throws Exception {
@Test
public void testHandleTokenCallback() throws Exception {
- Map configs = getSaslConfigs();
- AccessTokenBuilder builder = new AccessTokenBuilder()
+ JwtBuilder builder = new JwtBuilder()
.jwk(createRsaJwk())
.alg(AlgorithmIdentifiers.RSA_USING_SHA256);
- String accessToken = builder.build();
- AccessTokenRetriever accessTokenRetriever = () -> accessToken;
-
- OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs);
+ String jwt = builder.build();
+ JwtRetriever jwtRetriever = mock(JwtRetriever.class);
+ when(jwtRetriever.retrieve()).thenReturn(jwt);
- try {
+ try (OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever)) {
OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
- handler.handle(new Callback[] {callback});
+ handler.handle(new Callback[]{callback});
assertNotNull(callback.token());
OAuthBearerToken token = callback.token();
- assertEquals(accessToken, token.value());
+ assertEquals(jwt, token.value());
assertEquals(builder.subject(), token.principalName());
assertEquals(builder.expirationSeconds() * 1000, token.lifetimeMs());
assertEquals(builder.issuedAtSeconds() * 1000, token.startTimeMs());
- } finally {
- handler.close();
}
}
@@ -95,8 +87,8 @@ public void testHandleSaslExtensionsCallback() throws Exception {
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, "http://www.example.com");
Map jaasConfig = new HashMap<>();
- jaasConfig.put(CLIENT_ID_CONFIG, "an ID");
- jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret");
+ jaasConfig.put("clientId", "an ID");
+ jaasConfig.put("clientSecret", "a secret");
jaasConfig.put("extension_foo", "1");
jaasConfig.put("extension_bar", 2);
jaasConfig.put("EXTENSION_baz", "3");
@@ -125,8 +117,8 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() {
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, "http://www.example.com");
Map jaasConfig = new HashMap<>();
- jaasConfig.put(CLIENT_ID_CONFIG, "an ID");
- jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret");
+ jaasConfig.put("clientId", "an ID");
+ jaasConfig.put("clientSecret", "a secret");
jaasConfig.put(illegalKey, "this key isn't allowed per OAuthBearerClientInitialResponse.validateExtensions");
configureHandler(handler, configs, jaasConfig);
@@ -141,92 +133,90 @@ public void testHandleSaslExtensionsCallbackWithInvalidExtension() {
}
@Test
- public void testInvalidCallbackGeneratesUnsupportedCallbackException() {
- Map configs = getSaslConfigs();
- OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- AccessTokenRetriever accessTokenRetriever = () -> "foo";
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
- handler.init(accessTokenRetriever, accessTokenValidator);
+ public void testConfigureThrowsExceptionOnJwtValidatorConfigure() {
+ try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+ JwtRetriever jwtRetriever = mock(JwtRetriever.class);
+ JwtValidator jwtValidator = mock(JwtValidator.class)) {
- try {
- Callback unsupportedCallback = new Callback() { };
- assertThrows(UnsupportedCallbackException.class, () -> handler.handle(new Callback[]{unsupportedCallback}));
- } finally {
- handler.close();
+ doThrow(new KafkaException("Forced failure")).when(jwtValidator).configure(any(), any(), any());
+
+ assertThrows(
+ KafkaException.class,
+ () -> handler.configure(jwtRetriever, jwtValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of())
+ );
}
}
@Test
- public void testInvalidAccessToken() throws Exception {
- testInvalidAccessToken("this isn't valid", "Malformed JWT provided");
- testInvalidAccessToken("this.isn't.valid", "malformed Base64 URL encoded value");
- testInvalidAccessToken(createAccessKey("this", "isn't", "valid"), "malformed JSON");
- testInvalidAccessToken(createAccessKey("{}", "{}", "{}"), "exp value must be non-null");
+ public void testConfigureThrowsExceptionOnJwtValidatorClose() {
+ try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+ JwtRetriever jwtRetriever = mock(JwtRetriever.class)) {
+ JwtValidator jwtValidator = mock(JwtValidator.class);
+ doThrow(new KafkaException("Forced failure")).when(jwtValidator).close();
+ handler.configure(jwtRetriever, jwtValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
+ assertDoesNotThrow(handler::close);
+ }
}
@Test
- public void testMissingAccessToken() {
- AccessTokenRetriever accessTokenRetriever = () -> {
- throw new IOException("The token endpoint response access_token value must be non-null");
- };
- Map configs = getSaslConfigs();
- OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs);
+ public void testInvalidCallbackGeneratesUnsupportedCallbackException() throws IOException {
+ OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+ JwtRetriever jwtRetriever = mock(JwtRetriever.class);
+ when(jwtRetriever.retrieve()).thenReturn("foo");
+ JwtValidator jwtValidator = new ClientJwtValidator();
+ handler.configure(jwtRetriever, jwtValidator, getSaslConfigs(), OAUTHBEARER_MECHANISM, List.of());
try {
- OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
- assertThrowsWithMessage(IOException.class,
- () -> handler.handle(new Callback[]{callback}),
- "token endpoint response access_token value must be non-null");
+ Callback unsupportedCallback = new Callback() { };
+ assertThrows(UnsupportedCallbackException.class, () -> handler.handle(new Callback[]{unsupportedCallback}));
} finally {
handler.close();
}
}
@Test
- public void testFileTokenRetrieverHandlesNewline() throws IOException {
- Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
- long cur = cal.getTimeInMillis() / 1000;
- String exp = "" + (cur + 60 * 60); // 1 hour in future
- String iat = "" + cur;
-
- String expected = createAccessKey("{}", String.format("{\"exp\":%s, \"iat\":%s, \"sub\":\"subj\"}", exp, iat), "sign");
- String withNewline = expected + "\n";
+ public void testInvalidJwt() throws Exception {
+ testInvalidJwt("this isn't valid", "Malformed JWT provided");
+ testInvalidJwt("this.isn't.valid", "malformed Base64 URL encoded value");
+ testInvalidJwt(createAccessKey("this", "isn't", "valid"), "malformed JSON");
+ testInvalidJwt(createAccessKey("{}", "{}", "{}"), "exp value must be non-null");
+ }
- File tmpDir = createTempDir("access-token");
- File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", withNewline);
+ @Test
+ public void testMissingJwt() throws IOException {
+ JwtRetriever jwtRetriever = mock(JwtRetriever.class);
+ when(jwtRetriever.retrieve()).thenThrow(new JwtRetrieverException("The token endpoint response id_token value must be non-null"));
- Map configs = getSaslConfigs();
- OAuthBearerLoginCallbackHandler handler = createHandler(new FileTokenRetriever(accessTokenFile.toPath()), configs);
- OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
- try {
- handler.handle(new Callback[]{callback});
- assertEquals(callback.token().value(), expected);
- } catch (Exception e) {
- fail(e);
- } finally {
- handler.close();
+ try (OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever)) {
+ OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+ assertThrowsWithMessage(
+ JwtRetrieverException.class,
+ () -> handler.handle(new Callback[]{callback}),
+ "token endpoint response id_token value must be non-null"
+ );
}
}
@Test
public void testNotConfigured() {
- OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure or init method");
+ try (OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler()) {
+ assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure method");
+ }
}
@Test
- public void testConfigureWithAccessTokenFile() throws Exception {
+ public void testConfigureWithJwtFile() throws Exception {
String expected = "{}";
- File tmpDir = createTempDir("access-token");
- File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", expected);
- System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, accessTokenFile.toURI().toString());
+ File jwtFile = createTempFile("jwt-", ".json", expected);
+ System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, jwtFile.toURI().toString());
OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString());
- Map jaasConfigs = Collections.emptyMap();
- configureHandler(handler, configs, jaasConfigs);
- assertInstanceOf(FileTokenRetriever.class, handler.getAccessTokenRetriever());
+ Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, jwtFile.toURI().toString());
+ Map jaasConfig = Collections.emptyMap();
+ configureHandler(handler, configs, jaasConfig);
+ assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever);
+ assertInstanceOf(FileJwtRetriever.class, ((DefaultJwtRetriever) handler.jwtRetriever).delegate());
}
@Test
@@ -234,18 +224,19 @@ public void testConfigureWithAccessClientCredentials() {
OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
System.setProperty(ALLOWED_SASL_OAUTHBEARER_URLS_CONFIG, "http://www.example.com");
- Map jaasConfigs = new HashMap<>();
- jaasConfigs.put(CLIENT_ID_CONFIG, "an ID");
- jaasConfigs.put(CLIENT_SECRET_CONFIG, "a secret");
- configureHandler(handler, configs, jaasConfigs);
- assertInstanceOf(HttpAccessTokenRetriever.class, handler.getAccessTokenRetriever());
+ Map jaasConfig = new HashMap<>();
+ jaasConfig.put("clientId", "an ID");
+ jaasConfig.put("clientSecret", "a secret");
+ configureHandler(handler, configs, jaasConfig);
+ assertInstanceOf(DefaultJwtRetriever.class, handler.jwtRetriever);
+ assertInstanceOf(ClientCredentialsJwtRetriever.class, ((DefaultJwtRetriever) handler.jwtRetriever).delegate());
}
- private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception {
- Map configs = getSaslConfigs();
- OAuthBearerLoginCallbackHandler handler = createHandler(() -> accessToken, configs);
+ private void testInvalidJwt(String jwt, String expectedMessageSubstring) throws Exception {
+ JwtRetriever jwtRetriever = mock(JwtRetriever.class);
+ when(jwtRetriever.retrieve()).thenReturn(jwt);
- try {
+ try (OAuthBearerLoginCallbackHandler handler = createHandler(jwtRetriever)) {
OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
handler.handle(new Callback[]{callback});
@@ -255,24 +246,14 @@ private void testInvalidAccessToken(String accessToken, String expectedMessageSu
assertTrue(actualMessage.contains(expectedMessageSubstring), String.format(
"The error message \"%s\" didn't contain the expected substring \"%s\"",
actualMessage, expectedMessageSubstring));
- } finally {
- handler.close();
}
}
- private String createAccessKey(String header, String payload, String signature) {
- Base64.Encoder enc = Base64.getEncoder();
- header = enc.encodeToString(Utils.utf8(header));
- payload = enc.encodeToString(Utils.utf8(payload));
- signature = enc.encodeToString(Utils.utf8(signature));
- return String.format("%s.%s.%s", header, payload, signature);
- }
-
- private OAuthBearerLoginCallbackHandler createHandler(AccessTokenRetriever accessTokenRetriever, Map configs) {
+ protected OAuthBearerLoginCallbackHandler createHandler(JwtRetriever jwtRetriever) {
+ Map configs = getSaslConfigs();
OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
- AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
- handler.init(accessTokenRetriever, accessTokenValidator);
+ JwtValidator jwtValidator = new ClientJwtValidator();
+ handler.configure(jwtRetriever, jwtValidator, configs, OAUTHBEARER_MECHANISM, List.of());
return handler;
}
-
}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerTest.java
similarity index 68%
rename from clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java
rename to clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerTest.java
index 7f20b9464faea..1b101ba0a9b80 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerTest.java
@@ -14,15 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
-package org.apache.kafka.common.security.oauthbearer.internals.secured;
+package org.apache.kafka.common.security.oauthbearer;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
import org.apache.kafka.common.security.authenticator.TestJaasConfig;
-import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
@@ -37,34 +38,28 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
import java.io.File;
-import java.io.FileWriter;
import java.io.IOException;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.util.Arrays;
+import java.nio.file.Files;
import java.util.Base64;
import java.util.Collections;
-import java.util.Iterator;
import java.util.Map;
-import java.util.concurrent.ExecutionException;
import java.util.function.Consumer;
import javax.security.auth.login.AppConfigurationEntry;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
@TestInstance(Lifecycle.PER_CLASS)
public abstract class OAuthBearerTest {
protected final Logger log = LoggerFactory.getLogger(getClass());
+ protected final Time time = new MockTime();
+
protected ObjectMapper mapper = new ObjectMapper();
protected void assertThrowsWithMessage(Class extends Exception> clazz,
@@ -88,10 +83,18 @@ protected void configureHandler(AuthenticateCallbackHandler handler,
AppConfigurationEntry kafkaClient = config.getAppConfigurationEntry("KafkaClient")[0];
handler.configure(configs,
- OAuthBearerLoginModule.OAUTHBEARER_MECHANISM,
+ OAUTHBEARER_MECHANISM,
Collections.singletonList(kafkaClient));
}
+ protected String createAccessKey(String header, String payload, String signature) {
+ Base64.Encoder enc = Base64.getEncoder();
+ header = enc.encodeToString(Utils.utf8(header));
+ payload = enc.encodeToString(Utils.utf8(payload));
+ signature = enc.encodeToString(Utils.utf8(signature));
+ return String.format("%s.%s.%s", header, payload, signature);
+ }
+
protected String createBase64JsonJwtSection(Consumer c) {
String json = createJsonJwtSection(c);
@@ -119,61 +122,10 @@ protected String createJsonJwtSection(Consumer c) {
}
}
- protected Retryable createRetryable(Exception[] attempts) {
- Iterator i = Arrays.asList(attempts).iterator();
-
- return () -> {
- Exception e = i.hasNext() ? i.next() : null;
-
- if (e == null) {
- return "success!";
- } else {
- if (e instanceof IOException)
- throw new ExecutionException(e);
- else if (e instanceof RuntimeException)
- throw (RuntimeException) e;
- else
- throw new RuntimeException(e);
- }
- };
- }
-
- protected HttpURLConnection createHttpURLConnection(String response) throws IOException {
- HttpURLConnection mockedCon = mock(HttpURLConnection.class);
- when(mockedCon.getURL()).thenReturn(new URL("https://www.example.com"));
- when(mockedCon.getResponseCode()).thenReturn(200);
- when(mockedCon.getOutputStream()).thenReturn(new ByteArrayOutputStream());
- when(mockedCon.getInputStream()).thenReturn(new ByteArrayInputStream(Utils.utf8(response)));
- return mockedCon;
- }
-
- protected File createTempDir(String directory) throws IOException {
- File tmpDir = new File(System.getProperty("java.io.tmpdir"));
-
- if (directory != null)
- tmpDir = new File(tmpDir, directory);
-
- if (!tmpDir.exists() && !tmpDir.mkdirs())
- throw new IOException("Could not create " + tmpDir);
-
- tmpDir.deleteOnExit();
- log.debug("Created temp directory {}", tmpDir);
- return tmpDir;
- }
-
- protected File createTempFile(File tmpDir,
- String prefix,
- String suffix,
- String contents)
- throws IOException {
- File file = File.createTempFile(prefix, suffix, tmpDir);
+ protected File createTempFile(String prefix, String suffix, String contents) throws IOException {
+ File file = TestUtils.tempFile(prefix, suffix);
+ Files.writeString(file.toPath(), contents);
log.debug("Created new temp file {}", file);
- file.deleteOnExit();
-
- try (FileWriter writer = new FileWriter(file)) {
- writer.write(contents);
- }
-
return file;
}
@@ -211,5 +163,4 @@ protected PublicJsonWebKey createEcJwk() throws JoseException {
jwk.setKeyId("key-1");
return jwk;
}
-
}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java
index d682a05ec11cc..1d074c9f2760e 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandlerTest.java
@@ -17,24 +17,19 @@
package org.apache.kafka.common.security.oauthbearer;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenBuilder;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidator;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.AccessTokenValidatorFactory;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.CloseableVerificationKeyResolver;
-import org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerTest;
-import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.security.oauthbearer.internals.secured.JwtBuilder;
import org.jose4j.jws.AlgorithmIdentifiers;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
-import java.util.Base64;
import java.util.List;
import java.util.Map;
import javax.security.auth.callback.Callback;
import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE;
+import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule.OAUTHBEARER_MECHANISM;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@@ -46,74 +41,56 @@ public class OAuthBearerValidatorCallbackHandlerTest extends OAuthBearerTest {
public void testBasic() throws Exception {
String expectedAudience = "a";
List allAudiences = Arrays.asList(expectedAudience, "b", "c");
- AccessTokenBuilder builder = new AccessTokenBuilder()
+ JwtBuilder builder = new JwtBuilder()
.audience(expectedAudience)
.jwk(createRsaJwk())
.alg(AlgorithmIdentifiers.RSA_USING_SHA256);
- String accessToken = builder.build();
+ String jwt = builder.build();
Map configs = getSaslConfigs(SASL_OAUTHBEARER_EXPECTED_AUDIENCE, allAudiences);
- OAuthBearerValidatorCallbackHandler handler = createHandler(configs, builder);
- try {
- OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(accessToken);
+ try (OAuthBearerValidatorCallbackHandler handler = createHandler(configs, builder)) {
+ OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(jwt);
handler.handle(new Callback[]{callback});
assertNotNull(callback.token());
OAuthBearerToken token = callback.token();
- assertEquals(accessToken, token.value());
+ assertEquals(jwt, token.value());
assertEquals(builder.subject(), token.principalName());
assertEquals(builder.expirationSeconds() * 1000, token.lifetimeMs());
assertEquals(builder.issuedAtSeconds() * 1000, token.startTimeMs());
- } finally {
- handler.close();
}
}
@Test
- public void testInvalidAccessToken() throws Exception {
+ public void testInvalidJwt() throws Exception {
// There aren't different error messages for the validation step, so these are all the
// same :(
String substring = "invalid_token";
- assertInvalidAccessTokenFails("this isn't valid", substring);
- assertInvalidAccessTokenFails("this.isn't.valid", substring);
- assertInvalidAccessTokenFails(createAccessKey("this", "isn't", "valid"), substring);
- assertInvalidAccessTokenFails(createAccessKey("{}", "{}", "{}"), substring);
+ assertInvalidJwtFails("this isn't valid", substring);
+ assertInvalidJwtFails("this.isn't.valid", substring);
+ assertInvalidJwtFails(createAccessKey("this", "isn't", "valid"), substring);
+ assertInvalidJwtFails(createAccessKey("{}", "{}", "{}"), substring);
}
- private void assertInvalidAccessTokenFails(String accessToken, String expectedMessageSubstring) throws Exception {
+ private void assertInvalidJwtFails(String jwt, String expectedMessageSubstring) throws Exception {
Map