You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/08/30 18:50:16 UTC

[GitHub] [kafka] kirktrue opened a new pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

kirktrue opened a new pull request #11284:
URL: https://github.com/apache/kafka/pull/11284


   This task is to provide a concrete implementation of the interfaces defined in KIP-255 to allow Kafka to connect to an OAuth/OIDC identity provider for authentication and token retrieval. While KIP-255 provides an unsecured JWT example for development, this will fill in the gap and provide a production-grade implementation.
   
   The OAuth/OIDC work will allow out-of-the-box configuration by any Apache Kafka users to connect to an external identity provider service (e.g. Okta, Auth0, Azure, etc.). The code will implement the standard OAuth clientcredentials grant type.
   
   The proposed change is largely composed of a pair of AuthenticateCallbackHandler implementations: one to login on the client and one to validate on the broker.
   
   See the following for more detail:
   
   * [KIP-768](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186877575)
   * [KAFKA-13202](https://issues.apache.org/jira/browse/KAFKA-13202)
   
   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734089125



##########
File path: build.gradle
##########
@@ -829,6 +829,7 @@ project(':core') {
     implementation libs.jacksonDataformatCsv
     implementation libs.jacksonJDK8Datatypes
     implementation libs.joptSimple
+    implementation libs.jose4j

Review comment:
       Clients intentionally *doesn't* have the jose4j dependency (to keep client dependencies to a minimum), so we have to explicitly include it here. 
   
   Suggestions welcome on a better way to handle it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732088812



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        configure(verificationKeyResolver, accessTokenValidator);
+    }
+
+    public void configure(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);
+        }
+
+        isConfigured = true;

Review comment:
       nit: should we move the flag setting to the end of main configure() function?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734083140



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/PemDirectoryVerificationKeyResolver.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.security.Key;
+import java.security.PublicKey;
+import java.security.interfaces.ECPublicKey;
+import java.security.interfaces.RSAPublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.kafka.common.utils.Utils;
+import org.jose4j.jwk.EllipticCurveJsonWebKey;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.jwk.RsaJsonWebKey;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.keys.EcKeyUtil;
+import org.jose4j.keys.RsaKeyUtil;
+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;
+
+/**
+ * <code>PemVerificationKeyResolver</code> is a {@link VerificationKeyResolver} implementation
+ * that will load the PEM files from the given file system directory.
+ *
+ * The instance is configured with the directory name that contains one or more
+ * <a href=\"https://en.wikipedia.org/wiki/Privacy-Enhanced_Mail\">public key files</a>

Review comment:
       That said, I will strip this out as it's not needed for a first version.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732341161



##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/AccessTokenValidatorFactoryTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.kafka.common.KafkaException;
+import org.junit.jupiter.api.Test;
+
+public class AccessTokenValidatorFactoryTest extends OAuthBearerTest {
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorInit() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void init() throws IOException {
+                throw new IOException("My init had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+
+        assertThrowsWithMessage(
+            KafkaException.class, () -> handler.configure(accessTokenRetriever, accessTokenValidator), "encountered an error when initializing");
+    }
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void close() throws IOException {
+                throw new IOException("My close had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+        handler.configure(accessTokenRetriever, accessTokenValidator);
+
+        // Basically asserting this doesn't throw an exception :(
+        handler.close();
+    }
+
+    private OAuthBearerLoginCallbackHandler createHandler(AccessTokenRetriever accessTokenRetriever, Map<String, ?> configs) {

Review comment:
       This function is never used?

##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/AccessTokenValidatorFactoryTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.kafka.common.KafkaException;
+import org.junit.jupiter.api.Test;
+
+public class AccessTokenValidatorFactoryTest extends OAuthBearerTest {
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorInit() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void init() throws IOException {
+                throw new IOException("My init had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+
+        assertThrowsWithMessage(
+            KafkaException.class, () -> handler.configure(accessTokenRetriever, accessTokenValidator), "encountered an error when initializing");
+    }
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void close() throws IOException {
+                throw new IOException("My close had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+        handler.configure(accessTokenRetriever, accessTokenValidator);
+
+        // Basically asserting this doesn't throw an exception :(
+        handler.close();
+    }
+
+    private OAuthBearerLoginCallbackHandler createHandler(AccessTokenRetriever accessTokenRetriever, Map<String, ?> configs) {

Review comment:
       This function is never used?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736894948



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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 of set of configuration
+     * may have multiple instances, we want to reuse the single instance.
+     */
+
+    private static final Map<VkrKey, CloseableVerificationKeyResolver> VKR_CACHE = new HashMap<>();
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        Map<String, Object> moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries);
+        CloseableVerificationKeyResolver verificationKeyResolver;
+
+        // Here's the logic which keeps our VKRs down to a single instance.
+        synchronized (VKR_CACHE) {
+            VkrKey key = new VkrKey(configs, saslMechanism, moduleOptions);
+            verificationKeyResolver = VKR_CACHE.computeIfAbsent(key, k -> {
+                CloseableVerificationKeyResolver vkr = VerificationKeyResolverFactory.create(configs, saslMechanism, moduleOptions);
+                return new RefCountingVkr(vkr);
+            });
+        }
+
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        init(verificationKeyResolver, accessTokenValidator);
+    }
+
+    /*
+     * Package-visible for testing.
+     */
+
+    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;
+    }
+
+    @Override
+    public void close() {
+        if (verificationKeyResolver != null) {
+            try {
+                verificationKeyResolver.close();
+            } catch (Exception e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkInitialized();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerValidatorCallback) {
+                handleValidatorCallback((OAuthBearerValidatorCallback) callback);
+            } else if (callback instanceof OAuthBearerExtensionsValidatorCallback) {
+                handleExtensionsValidatorCallback((OAuthBearerExtensionsValidatorCallback) callback);
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    private void handleValidatorCallback(OAuthBearerValidatorCallback callback) {
+        checkInitialized();
+
+        OAuthBearerToken token;
+
+        try {
+            token = accessTokenValidator.validate(callback.tokenValue());
+            log.debug("handle - token: {}", token);
+            callback.token(token);
+        } catch (ValidateException e) {
+            log.warn(e.getMessage(), e);
+            callback.error("invalid_token", null, null);
+        }
+    }
+
+    private void handleExtensionsValidatorCallback(OAuthBearerExtensionsValidatorCallback extensionsValidatorCallback) {
+        checkInitialized();
+
+        extensionsValidatorCallback.inputExtensions().map().forEach((extensionName, v) -> extensionsValidatorCallback.valid(extensionName));
+    }
+
+    private void checkInitialized() {
+        if (!isInitialized)
+            throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName()));
+    }
+
+    /**
+     * <code>VkrKey</code> 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 VkrKey {

Review comment:
       What does vkr stand for?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    /**
+     * {@link HttpsJwks} does the actual work of contacting the OAuth/OIDC endpoint to get the
+     * JWKS. In some cases, the call to {@link HttpsJwks#getJsonWebKeys()} will trigger a call
+     * to {@link HttpsJwks#refresh()} which will block the current thread in network I/O. We cache
+     * the JWKS ourselves (see {@link #jsonWebKeys}) to avoid the network I/O.
+     *
+     * We want to be very careful where we use the {@link HttpsJwks} instance so that we don't
+     * perform any operation (directly or indirectly) that could cause blocking. This is because
+     * the JWKS logic is part of the larger authentication logic which operates on Kafka's network
+     * thread. It's OK to execute {@link HttpsJwks#getJsonWebKeys()} (which calls
+     * {@link HttpsJwks#refresh()}) from within {@link #init()} as that method is called only at
+     * startup, and we can afford the blocking hit there.
+     */
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    /**
+     * Protects {@link #missingKeyIds} and {@link #jsonWebKeys}.
+     */
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    /**
+     * 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<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> 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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Our implementation avoids the blocking call within {@link HttpsJwks#refresh()} that is
+     * sometimes called internal to {@link HttpsJwks#getJsonWebKeys()}. We want to avoid any
+     * blocking I/O is this code is running in the authentication path on the Kafka network thread.

Review comment:
       is this code => as this code

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler

Review comment:
       The KIP uses listener.name.<listener name>.oauthbearer.sasl.login.callback.handler.class, could we fix that?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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 of set of configuration
+     * may have multiple instances, we want to reuse the single instance.
+     */
+
+    private static final Map<VkrKey, CloseableVerificationKeyResolver> VKR_CACHE = new HashMap<>();
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        Map<String, Object> moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries);
+        CloseableVerificationKeyResolver verificationKeyResolver;
+
+        // Here's the logic which keeps our VKRs down to a single instance.
+        synchronized (VKR_CACHE) {
+            VkrKey key = new VkrKey(configs, saslMechanism, moduleOptions);

Review comment:
       Do we need saslMechanism as part of the key? It seems that saslMechanism should always be oauthbearer. Perhaps we could assert that?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    /**
+     * {@link HttpsJwks} does the actual work of contacting the OAuth/OIDC endpoint to get the
+     * JWKS. In some cases, the call to {@link HttpsJwks#getJsonWebKeys()} will trigger a call
+     * to {@link HttpsJwks#refresh()} which will block the current thread in network I/O. We cache
+     * the JWKS ourselves (see {@link #jsonWebKeys}) to avoid the network I/O.
+     *
+     * We want to be very careful where we use the {@link HttpsJwks} instance so that we don't
+     * perform any operation (directly or indirectly) that could cause blocking. This is because
+     * the JWKS logic is part of the larger authentication logic which operates on Kafka's network
+     * thread. It's OK to execute {@link HttpsJwks#getJsonWebKeys()} (which calls
+     * {@link HttpsJwks#refresh()}) from within {@link #init()} as that method is called only at
+     * startup, and we can afford the blocking hit there.
+     */
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    /**
+     * Protects {@link #missingKeyIds} and {@link #jsonWebKeys}.
+     */
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    /**
+     * 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<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> 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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Our implementation avoids the blocking call within {@link HttpsJwks#refresh()} that is
+     * sometimes called internal to {@link HttpsJwks#getJsonWebKeys()}. We want to avoid any
+     * blocking I/O is this code is running in the authentication path on the Kafka network thread.
+     *
+     * The list may be stale up to {@link #refreshMs}.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache in a background thread.
+     *
+     * This method may be called as part of the <i>normally</i>-scheduled refresh call
+     * (via {@link #executorService}, but it may have

Review comment:
       "but it may have" seems incomplete.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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 of set of configuration

Review comment:
       each of set => each set

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.
+                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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, this refresh method
+        // will still be called again within RETRY_BACKOFF_MS :)
+        //
+        // Note: timeBeforeNextRefresh is negative when we're in the midst of executing refresh
+        // in a scheduled fashion. ScheduledFuture.getDelay will reset *after* the method has

Review comment:
       If timeBeforeNextRefresh <= 0, it seems that we will trigger a refresh. However, in that case, we're in the midst of executing refresh. So, another refresh seems unnecessary?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736006401



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.
+                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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, this refresh method
+        // will still be called again within RETRY_BACKOFF_MS :)
+        //
+        // Note: timeBeforeNextRefresh is negative when we're in the midst of executing refresh
+        // in a scheduled fashion. ScheduledFuture.getDelay will reset *after* the method has

Review comment:
       I'm not sure I fully understand the question, sorry. Can you rephrase?
   
   `refresh` can get called in one of two ways:
   
   1. Scheduled refresh
   2. Expedited refresh
   
   An expedited refresh is performed because the JWT included a key ID that we don't have in our cache. The idea I implemented was to skip execution of an expedited refresh if the amount of time before the next scheduled refresh was less than some set amount. Basically: if we're going to do a scheduled refresh in the next N seconds or so, let's just wait for that. Kind of a cheap batching idea.
   
   Another option is to just perform the refresh without any of that checking, but have a flag (`isCurrentlyRefreshing`) that is checked at the start of a refresh and returns if there's already one executing. At least that's how I'm interpreting your comment.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734106778



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/PemDirectoryVerificationKeyResolver.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.security.Key;
+import java.security.PublicKey;
+import java.security.interfaces.ECPublicKey;
+import java.security.interfaces.RSAPublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.kafka.common.utils.Utils;
+import org.jose4j.jwk.EllipticCurveJsonWebKey;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.jwk.RsaJsonWebKey;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.keys.EcKeyUtil;
+import org.jose4j.keys.RsaKeyUtil;
+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;
+
+/**
+ * <code>PemVerificationKeyResolver</code> is a {@link VerificationKeyResolver} implementation

Review comment:
       It was going to be a format that we support, but it doesn't need to be in the first version.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734905502



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> that will be used by the
+     * {@link RefreshingHttpsJwksVerificationKeyResolver} to resolve new key IDs in JWTs.
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = Collections.unmodifiableList(localJWKs);

Review comment:
       Sure. I'll remove `this.`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734120121



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/JwksFileVerificationKeyResolver.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.security.Key;
+import java.util.List;
+import org.apache.kafka.common.utils.Utils;
+import org.jose4j.jwk.JsonWebKeySet;
+import org.jose4j.jws.JsonWebSignature;
+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;
+
+/**
+ * <code>JwksFileVerificationKeyResolver</code> is a {@link VerificationKeyResolver} implementation
+ * that will load the JWKS from the given file system directory.

Review comment:
       Done. Take a look and let me know if there's more that needs to be said.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734096345



##########
File path: clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
##########
@@ -75,30 +78,111 @@
     public static final String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = "Login refresh thread will sleep until the specified window factor relative to the"
             + " credential's lifetime has been reached, at which time it will try to refresh the credential."
             + " Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a default value of 0.8 (80%) is used"
-            + " if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR = 0.80;
 
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER = "sasl.login.refresh.window.jitter";
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = "The maximum amount of random jitter relative to the credential's lifetime"
             + " that is added to the login refresh thread's sleep time. Legal values are between 0 and 0.25 (25%) inclusive;"
-            + " a default value of 0.05 (5%) is used if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " a default value of 0.05 (5%) is used if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_JITTER = 0.05;
 
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS = "sasl.login.refresh.min.period.seconds";
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = "The desired minimum time for the login refresh thread to wait before refreshing a credential,"
             + " in seconds. Legal values are between 0 and 900 (15 minutes); a default value of 60 (1 minute) is used if no value is specified.  This value and "
             + " sasl.login.refresh.buffer.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS = 60;
 
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS = "sasl.login.refresh.buffer.seconds";
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = "The amount of buffer time before credential expiration to maintain when refreshing a credential,"
             + " in seconds. If a refresh would otherwise occur closer to expiration than the number of buffer seconds then the refresh will be moved up to maintain"
             + " as much of the buffer time as possible. Legal values are between 0 and 3600 (1 hour); a default value of  300 (5 minutes) is used if no value is specified."
             + " This value and sasl.login.refresh.min.period.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS = 300;
 
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS = "sasl.login.connect.timeout.ms";
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider connection timeout."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS = "sasl.login.read.timeout.ms";
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider read timeout."
+            + OAUTHBEARER_NOTE;
+
+    // These are only specified here outside their normal groupings so that they can be
+    // forward referencing.
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MS = "sasl.login.retry.backoff.ms";
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS = "sasl.login.retry.backoff.max.ms";
+
+    private static final String EXPONENTIAL_BACKOFF_NOTE = " Login uses an exponential backoff algorithm with an initial wait based on the"
+            + " " + SASL_LOGIN_RETRY_BACKOFF_MS
+            + " setting and will double in wait length between attempts up to a maximum wait length specified by the"
+            + " " + SASL_LOGIN_RETRY_BACKOFF_MAX_MS
+            + " setting.";

Review comment:
       I have added the `OAUTHBEARER_NOTE` to the `EXPONENTIAL_BACKOFF_NOTE` so that it is included in the documentation for the `sasl.login.retry.backoff.ms` and `sasl.login.retry.backoff.max.ms` configuration options.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732339345



##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/AccessTokenValidatorFactoryTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.kafka.common.KafkaException;
+import org.junit.jupiter.api.Test;
+
+public class AccessTokenValidatorFactoryTest extends OAuthBearerTest {
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorInit() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void init() throws IOException {
+                throw new IOException("My init had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+
+        assertThrowsWithMessage(
+            KafkaException.class, () -> handler.configure(accessTokenRetriever, accessTokenValidator), "encountered an error when initializing");
+    }
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void close() throws IOException {
+                throw new IOException("My close had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+        handler.configure(accessTokenRetriever, accessTokenValidator);
+
+        // Basically asserting this doesn't throw an exception :(

Review comment:
       dummy question: why we don't want to throw exception when close fails?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733895025



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory data structure</li>
+ *     <li>
+ *         Presence of scope, <code>exp</code>, subject, <code>iss</code>, and
+ *         <code>iat</code> claims
+ *     </li>
+ *     <li>
+ *         Signature matching validation against the <code>kid</code> and those provided b

Review comment:
       Thank ou for noticing that 😄 . Fixed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733850703



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {
+        this.configs = configs;
+
+        if (saslMechanism != null && !saslMechanism.trim().isEmpty())
+            this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
+        else
+            this.prefix = null;
+    }
+
+    public Map<String, ?> getSslClientConfig(String uriConfigName) {
+        String urlConfigValue = get(uriConfigName);
+
+        if (urlConfigValue == null || urlConfigValue.trim().isEmpty())
+            throw new ConfigException(String.format("The OAuth configuration option %s is required", uriConfigName));
+
+        URL url;
+
+        try {
+            url = new URL(urlConfigValue);
+        } catch (IOException e) {
+            throw new ConfigException(String.format("The OAuth configuration option %s was not a valid URL (%s)", uriConfigName, urlConfigValue));
+        }
+
+        if (!url.getProtocol().equalsIgnoreCase("https")) {
+            log.warn("Not creating SSL socket factory as URL for {} ({}) is not SSL-/TLS-based", uriConfigName, url);
+            return null;
+        }
+
+        ConfigDef sslConfigDef = new ConfigDef();
+        sslConfigDef.withClientSslSupport();
+        AbstractConfig sslClientConfig = new AbstractConfig(sslConfigDef, configs);
+        return sslClientConfig.values();
+    }
+
+    public SSLSocketFactory createSSLSocketFactory(String uriConfigName) {
+        Map<String, ?> sslClientConfig = getSslClientConfig(uriConfigName);
+
+        if (sslClientConfig == null) {
+            log.warn("Requesting SSL client socket factory but SSL configs were null");

Review comment:
       I refactored the code a bit, so this has been removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732259427



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        configure(verificationKeyResolver, accessTokenValidator);
+    }
+
+    public void configure(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);
+        }
+
+        isConfigured = true;
+    }
+
+    @Override
+    public void close() {
+        if (verificationKeyResolver != null) {
+            try {
+                verificationKeyResolver.close();
+            } catch (Exception e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkConfigured();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerValidatorCallback) {
+                handle((OAuthBearerValidatorCallback) callback);
+            } else if (callback instanceof OAuthBearerExtensionsValidatorCallback) {
+                OAuthBearerExtensionsValidatorCallback extensionsCallback = (OAuthBearerExtensionsValidatorCallback) callback;
+                extensionsCallback.inputExtensions().map().forEach((extensionName, v) -> extensionsCallback.valid(extensionName));

Review comment:
       Correct.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734095192



##########
File path: clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
##########
@@ -75,30 +78,111 @@
     public static final String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = "Login refresh thread will sleep until the specified window factor relative to the"
             + " credential's lifetime has been reached, at which time it will try to refresh the credential."
             + " Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a default value of 0.8 (80%) is used"
-            + " if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR = 0.80;
 
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER = "sasl.login.refresh.window.jitter";
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = "The maximum amount of random jitter relative to the credential's lifetime"
             + " that is added to the login refresh thread's sleep time. Legal values are between 0 and 0.25 (25%) inclusive;"
-            + " a default value of 0.05 (5%) is used if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " a default value of 0.05 (5%) is used if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_JITTER = 0.05;
 
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS = "sasl.login.refresh.min.period.seconds";
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = "The desired minimum time for the login refresh thread to wait before refreshing a credential,"
             + " in seconds. Legal values are between 0 and 900 (15 minutes); a default value of 60 (1 minute) is used if no value is specified.  This value and "
             + " sasl.login.refresh.buffer.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS = 60;
 
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS = "sasl.login.refresh.buffer.seconds";
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = "The amount of buffer time before credential expiration to maintain when refreshing a credential,"
             + " in seconds. If a refresh would otherwise occur closer to expiration than the number of buffer seconds then the refresh will be moved up to maintain"
             + " as much of the buffer time as possible. Legal values are between 0 and 3600 (1 hour); a default value of  300 (5 minutes) is used if no value is specified."
             + " This value and sasl.login.refresh.min.period.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS = 300;
 
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS = "sasl.login.connect.timeout.ms";
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider connection timeout."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS = "sasl.login.read.timeout.ms";
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider read timeout."
+            + OAUTHBEARER_NOTE;
+
+    // These are only specified here outside their normal groupings so that they can be
+    // forward referencing.
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MS = "sasl.login.retry.backoff.ms";
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS = "sasl.login.retry.backoff.max.ms";
+
+    private static final String EXPONENTIAL_BACKOFF_NOTE = " Login uses an exponential backoff algorithm with an initial wait based on the"
+            + " " + SASL_LOGIN_RETRY_BACKOFF_MS
+            + " setting and will double in wait length between attempts up to a maximum wait length specified by the"
+            + " " + SASL_LOGIN_RETRY_BACKOFF_MAX_MS
+            + " setting.";
+
+    public static final long DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS = 10000;
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC = "The (optional) value in milliseconds for the maximum wait between login attempts to the"
+            + " external authentication provider."
+            + EXPONENTIAL_BACKOFF_NOTE;
+
+    public static final long DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS = 100;
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MS_DOC = "The (optional) value in milliseconds for the initial wait between login attempts to the external"
+            + " authentication provider."
+            + EXPONENTIAL_BACKOFF_NOTE;
+
+    public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME = "sasl.oauthbearer.scope.claim.name";
+    public static final String DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME = "scope";
+    public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC = "The OAuth claim for the scope is often named \"" + DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME + "\", but this (optional)"
+            + " setting can provide a different name to use for the scope included in the JWT payload's claims if the OAuth/OIDC provider uses a different"
+            + " name for that claim.";
+
+    public static final String SASL_OAUTHBEARER_SUB_CLAIM_NAME = "sasl.oauthbearer.sub.claim.name";
+    public static final String DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME = "sub";
+    public static final String SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC = "The OAuth claim for the subject is often named \"" + DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME + "\", but this (optional)"
+            + " setting can provide a different name to use for the subject included in the JWT payload's claims if the OAuth/OIDC provider uses a different"
+            + " name for that claim.";
+
+    public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI = "sasl.oauthbearer.token.endpoint.uri";
+    public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI_DOC = "The URL for the OAuth/OIDC identity provider. If the URL is HTTP(S)-based, it is the issuer's token"
+            + " endpoint URI to which requests will be made to login based on the configuration in " + SASL_JAAS_CONFIG + ". If the URL is file-based, it"
+            + " specifies a file containing an access token (in JWT serialized form) issued by the OAuth/OIDC identity provider to use for authorization.";
+
+    // These are only specified here outside their normal groupings so that they can be
+    // forward referencing.
+    public static final String SASL_OAUTHBEARER_JWKS_ENDPOINT_URI = "sasl.oauthbearer.jwks.endpoint.uri";
+    public static final String SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS = "sasl.oauthbearer.jwks.endpoint.refresh.ms";

Review comment:
       I have rearranged the ordering to be consistent with the existing structure. This meant that I had to reference the name in a string literal vs. using a reference to the name, but 🤷 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734081204



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {

Review comment:
       Yes, I have added appropriate JavaDoc describing this callback handler and contrasting it with the other.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736041929



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {

Review comment:
       I implemented the change you requested by adding the appropriate code to `OAuthBearerValidatorCallbackHandler` reuse shared instances.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735883615



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> that will be used by the
+     * {@link RefreshingHttpsJwksVerificationKeyResolver} to resolve new key IDs in JWTs.
+     *
+     * @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
+     */
+
+    public RefreshingHttpsJwks(HttpsJwks httpsJwks, long refreshMs) {
+        this(Time.SYSTEM, httpsJwks, refreshMs);
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> 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();
+            }
+
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                0,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, refreshInternal will

Review comment:
       Will update comment. Good catch.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732068187



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class);
+
+    public static final String CLIENT_ID_CONFIG = "clientId";
+    public static final String CLIENT_SECRET_CONFIG = "clientSecret";
+    public static final String SCOPE_CONFIG = "scope";
+
+    public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client ID to uniquely identify the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " +
+        "account and identifies the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " +
+        "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI + ") may need to specify an " +
+        "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " +
+        "include with the login request.";
+
+    private static final String EXTENSION_PREFIX = "extension_";
+
+    private Map<String, Object> moduleOptions;
+
+    private AccessTokenRetriever accessTokenRetriever;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> 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()));
+
+        moduleOptions = Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+        AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism);
+        configure(accessTokenRetriever, accessTokenValidator);
+    }
+
+    public void configure(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator accessTokenValidator) {

Review comment:
       nit: should we name this func as init()?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732068749



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class);
+
+    public static final String CLIENT_ID_CONFIG = "clientId";
+    public static final String CLIENT_SECRET_CONFIG = "clientSecret";
+    public static final String SCOPE_CONFIG = "scope";
+
+    public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client ID to uniquely identify the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " +
+        "account and identifies the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " +
+        "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI + ") may need to specify an " +
+        "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " +
+        "include with the login request.";
+
+    private static final String EXTENSION_PREFIX = "extension_";
+
+    private Map<String, Object> moduleOptions;
+
+    private AccessTokenRetriever accessTokenRetriever;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> 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()));
+
+        moduleOptions = Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+        AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism);
+        configure(accessTokenRetriever, accessTokenValidator);
+    }
+
+    public void configure(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator accessTokenValidator) {
+        this.accessTokenRetriever = accessTokenRetriever;
+        this.accessTokenValidator = accessTokenValidator;
+
+        try {
+            this.accessTokenRetriever.init();
+        } catch (IOException e) {
+            throw new KafkaException("The OAuth login configuration encountered an error when initializing the AccessTokenRetriever", e);
+        }
+
+        isConfigured = true;
+    }
+
+    @Override
+    public void close() {
+        if (accessTokenRetriever != null) {
+            try {
+                this.accessTokenRetriever.close();
+            } catch (IOException e) {
+                log.warn("The OAuth login configuration encountered an error when closing the AccessTokenRetriever", e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkConfigured();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerTokenCallback) {
+                handle((OAuthBearerTokenCallback) callback);
+            } else if (callback instanceof SaslExtensionsCallback) {
+                handle((SaslExtensionsCallback) callback);
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    void handle(OAuthBearerTokenCallback callback) throws IOException {

Review comment:
       nit: Should we name these two handle functions separetely? like handleCallback() and handleExtensionsCallback()?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732258376



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class);
+
+    public static final String CLIENT_ID_CONFIG = "clientId";
+    public static final String CLIENT_SECRET_CONFIG = "clientSecret";
+    public static final String SCOPE_CONFIG = "scope";
+
+    public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client ID to uniquely identify the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " +
+        "account and identifies the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " +
+        "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI + ") may need to specify an " +
+        "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " +
+        "include with the login request.";
+
+    private static final String EXTENSION_PREFIX = "extension_";
+
+    private Map<String, Object> moduleOptions;
+
+    private AccessTokenRetriever accessTokenRetriever;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> 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()));
+
+        moduleOptions = Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+        AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism);
+        configure(accessTokenRetriever, accessTokenValidator);
+    }
+
+    public void configure(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator accessTokenValidator) {

Review comment:
       Sure! I changed it to `init`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736971828



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    /**
+     * {@link HttpsJwks} does the actual work of contacting the OAuth/OIDC endpoint to get the
+     * JWKS. In some cases, the call to {@link HttpsJwks#getJsonWebKeys()} will trigger a call
+     * to {@link HttpsJwks#refresh()} which will block the current thread in network I/O. We cache
+     * the JWKS ourselves (see {@link #jsonWebKeys}) to avoid the network I/O.
+     *
+     * We want to be very careful where we use the {@link HttpsJwks} instance so that we don't
+     * perform any operation (directly or indirectly) that could cause blocking. This is because
+     * the JWKS logic is part of the larger authentication logic which operates on Kafka's network
+     * thread. It's OK to execute {@link HttpsJwks#getJsonWebKeys()} (which calls
+     * {@link HttpsJwks#refresh()}) from within {@link #init()} as that method is called only at
+     * startup, and we can afford the blocking hit there.
+     */
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    /**
+     * Protects {@link #missingKeyIds} and {@link #jsonWebKeys}.
+     */
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    /**
+     * 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<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> 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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Our implementation avoids the blocking call within {@link HttpsJwks#refresh()} that is
+     * sometimes called internal to {@link HttpsJwks#getJsonWebKeys()}. We want to avoid any
+     * blocking I/O is this code is running in the authentication path on the Kafka network thread.

Review comment:
       Changed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734107280



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class);
+
+    public static final String CLIENT_ID_CONFIG = "clientId";
+    public static final String CLIENT_SECRET_CONFIG = "clientSecret";
+    public static final String SCOPE_CONFIG = "scope";
+
+    public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client ID to uniquely identify the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " +
+        "account and identifies the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " +
+        "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI + ") may need to specify an " +
+        "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " +
+        "include with the login request.";
+
+    private static final String EXTENSION_PREFIX = "extension_";
+
+    private Map<String, Object> moduleOptions;
+
+    private AccessTokenRetriever accessTokenRetriever;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> 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()));
+
+        moduleOptions = Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+        AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions);

Review comment:
       The login callback handler can be used from the broker side too.
   
   The SASL mechanism is passed in for the case of the broker side. The later configuration lookups will look for the optional SASL mechanism prefix and use it if the configuration is there, or fall back to the top-level configuration.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733259234



##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/AccessTokenValidatorFactoryTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.kafka.common.KafkaException;
+import org.junit.jupiter.api.Test;
+
+public class AccessTokenValidatorFactoryTest extends OAuthBearerTest {
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorInit() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void init() throws IOException {
+                throw new IOException("My init had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+
+        assertThrowsWithMessage(
+            KafkaException.class, () -> handler.configure(accessTokenRetriever, accessTokenValidator), "encountered an error when initializing");
+    }
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void close() throws IOException {
+                throw new IOException("My close had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+        handler.configure(accessTokenRetriever, accessTokenValidator);
+
+        // Basically asserting this doesn't throw an exception :(
+        handler.close();
+    }
+
+    private OAuthBearerLoginCallbackHandler createHandler(AccessTokenRetriever accessTokenRetriever, Map<String, ?> configs) {

Review comment:
       Correct. I will remove it in the next revision.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733882914



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");

Review comment:
       I was/am a _little_ leery of including the response verbatim, only because the response could be huge, especially if the URL provided in the configuration is incorrect.
   
   The code is now logging the entire response, but arbitrarily limiting the length to 1000 characters for the exception message.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734082935



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/PemDirectoryVerificationKeyResolver.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.security.Key;
+import java.security.PublicKey;
+import java.security.interfaces.ECPublicKey;
+import java.security.interfaces.RSAPublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.kafka.common.utils.Utils;
+import org.jose4j.jwk.EllipticCurveJsonWebKey;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.jwk.RsaJsonWebKey;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.keys.EcKeyUtil;
+import org.jose4j.keys.RsaKeyUtil;
+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;
+
+/**
+ * <code>PemVerificationKeyResolver</code> is a {@link VerificationKeyResolver} implementation
+ * that will load the PEM files from the given file system directory.
+ *
+ * The instance is configured with the directory name that contains one or more
+ * <a href=\"https://en.wikipedia.org/wiki/Privacy-Enhanced_Mail\">public key files</a>

Review comment:
       I believe so 🤷 
   
   From that Wikipedia page:
   
   > Privacy-Enhanced Mail (PEM) is a de facto file format for storing and sending cryptographic keys, certificates, and other data.
   
   and...
   
   > PEM data is commonly stored in files with a ".pem" suffix, a ".cer" or ".crt" suffix (for certificates), or a ".key" suffix (for public or private keys).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734096643



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/Retry.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Retry encapsulates the mechanism to perform a retry and then exponential
+ * backoff using provided wait times between attempts.
+ *
+ * @param <R> Result type
+ */
+
+public class Retry<R> {

Review comment:
       I'm a little hesitant to do so, only because I want to have freedom to refactor as needed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733928342



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/AccessTokenRetrieverFactory.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG;
+
+import java.net.URI;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+
+public class AccessTokenRetrieverFactory  {
+
+    /**
+     * Create an {@link AccessTokenRetriever} from the given SASL and JAAS configuration.
+     *
+     * <b>Note</b>: the returned <code>AccessTokenRetriever</code> is <em>not</em> initialized
+     * here and must be done by the caller prior to use.
+     *
+     * @param configs    SASL configuration
+     * @param jaasConfig JAAS configuration
+     *
+     * @return Non-<code>null</code> {@link AccessTokenRetriever}
+     */
+
+    public static AccessTokenRetriever create(Map<String, ?> configs, Map<String, Object> jaasConfig) {
+        return create(configs, null, jaasConfig);
+    }
+
+    public static AccessTokenRetriever create(Map<String, ?> configs, String saslMechanism, Map<String, Object> jaasConfig) {
+        ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
+        URI tokenEndpointUri = cu.validateUri(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI);
+
+        if (tokenEndpointUri.getScheme().toLowerCase(Locale.ROOT).equals("file")) {
+            return new FileTokenRetriever(cu.validateFile(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI));
+        } else {
+            ConfigurationUtils jaasCu = new ConfigurationUtils(jaasConfig);
+            String clientId = jaasCu.validateString(CLIENT_ID_CONFIG);
+            String clientSecret = jaasCu.validateString(CLIENT_SECRET_CONFIG);
+            String scope = jaasCu.get(SCOPE_CONFIG);
+
+            SSLSocketFactory sslSocketFactory = cu.createSSLSocketFactory(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI);
+
+            return new HttpAccessTokenRetriever(clientId,
+                clientSecret,
+                scope,
+                sslSocketFactory,
+                tokenEndpointUri.toString(),
+                cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MS),
+                cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS),
+                cu.validateInteger(SASL_LOGIN_CONNECT_TIMEOUT_MS, false, null),

Review comment:
       Changed to use the two-parameter version of `validateInteger`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734111852



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {

Review comment:
       Per the `AuthenticateCallbackHandler` interface, the `configure` method parameters do not include the listener name.
   
   Based on my research, it seemed like the upstream code only passed in the configuration that's either listener-based or top-level.
   
   The SASL mechanism is passed in for the case of the broker side. The later configuration lookups will look for the optional SASL mechanism prefix and use it if the configuration is there, or fall back to the top-level configuration.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734111966



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {

Review comment:
       Or should I change the `AuthenticateCallbackHandler` interface's `configure` method to take the listener name too?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734111966



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {

Review comment:
       Or should I change the `AuthenticateCallbackHandler` interface's, `configure` method to take the listener name too?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734908199



##########
File path: build.gradle
##########
@@ -1200,6 +1201,7 @@ project(':clients') {
 
     compileOnly libs.jacksonDatabind // for SASL/OAUTHBEARER bearer token parsing
     compileOnly libs.jacksonJDK8Datatypes
+    compileOnly libs.jose4j

Review comment:
       Done!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733897632



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory data structure</li>
+ *     <li>
+ *         Presence of scope, <code>exp</code>, subject, <code>iss</code>, and
+ *         <code>iat</code> claims
+ *     </li>
+ *     <li>
+ *         Signature matching validation against the <code>kid</code> and those provided b
+ *         the OAuth/OIDC provider's JWKS
+ *     </li>
+ * </ol>
+ */
+
+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 <code>null</code> 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
+     *                                <code>aud</code> claim and if this value is set, the
+     *                                broker will match the value from JWT's <code>aud</code>
+     *                                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 <code>null</code> to not perform an

Review comment:
       Completed the thought so that it now reads:
   
   ```
   May be <code>null</code> to not perform any check to verify the JWT's audience matches any fixed set of known/expected audiences.
   ```
   
   Thanks for catching that.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
junrao commented on pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#issuecomment-953357814


   @YiDing-Duke : Any more comments from you?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732258128



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/Retry.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Retry encapsulates the mechanism to perform a retry and then exponential
+ * backoff using provided wait times between attempts.
+ *
+ * @param <R> Result type
+ */
+
+public class Retry<R> {
+
+    private static final Logger log = LoggerFactory.getLogger(Retry.class);
+
+    private final Time time;
+
+    private final long retryBackoffMs;
+
+    private final long 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 %s must be non-negative", retryBackoffMs));
+
+        if (this.retryBackoffMaxMs < 0)
+            throw new IllegalArgumentException(String.format("retryBackoffMaxMs %s value must be non-negative", retryBackoffMaxMs));
+
+        if (this.retryBackoffMaxMs < this.retryBackoffMs)
+            throw new IllegalArgumentException(String.format("retryBackoffMaxMs %s is less than retryBackoffMs %s", retryBackoffMaxMs, retryBackoffMs));
+    }
+
+    public R execute(Retryable<R> retryable) throws IOException {
+        int currAttempt = 0;
+        long end = time.milliseconds() + retryBackoffMaxMs;
+        IOException error = null;
+
+        while (time.milliseconds() <= end) {
+            currAttempt++;
+
+            try {
+                return retryable.call();
+            } catch (IOException e) {
+                if (error == null)
+                    error = e;
+
+                long waitMs = retryBackoffMs * (long) Math.pow(2, currAttempt - 1);
+                long diff = end - time.milliseconds();
+                waitMs = Math.min(waitMs, diff);
+
+                if (waitMs <= 0)

Review comment:
       Added logging for error.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r731496881



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = null;
+
+        try (InputStream is = con.getInputStream()) {
+            ByteArrayOutputStream os = new ByteArrayOutputStream();
+            log.debug("handleOutput - preparing to read request body to {}", con.getURL());

Review comment:
       nit: should we say "response body" rather than "request body"?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735884051



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> that will be used by the
+     * {@link RefreshingHttpsJwksVerificationKeyResolver} to resolve new key IDs in JWTs.
+     *
+     * @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
+     */
+
+    public RefreshingHttpsJwks(HttpsJwks httpsJwks, long refreshMs) {
+        this(Time.SYSTEM, httpsJwks, refreshMs);
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> 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();
+            }
+
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                0,

Review comment:
       Yes. I will set the `initialDelay` parameter value to `refreshMs`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735884322



##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwksTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.secured;
+
+import static org.apache.kafka.common.security.oauthbearer.secured.RefreshingHttpsJwks.MISSING_KEY_ID_CACHE_IN_FLIGHT_MS;
+import static org.apache.kafka.common.security.oauthbearer.secured.RefreshingHttpsJwks.MISSING_KEY_ID_MAX_KEY_LENGTH;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.kafka.common.utils.MockTime;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.junit.jupiter.api.Test;
+
+public class RefreshingHttpsJwksTest extends OAuthBearerTest {
+
+    public static final int REFRESH_MS = 10000;
+
+    /**
+     * Test that a key not previously scheduled for refresh will be scheduled without a refresh.
+     */
+
+    @Test
+    public void test() throws Exception {
+        String keyId = "abc123";
+        MockTime time = new MockTime();
+        HttpsJwks httpsJwks = mockHttpsJwks();
+
+        try (RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS)) {
+            refreshingHttpsJwks.init();
+            assertTrue(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+            verify(httpsJwks, times(0)).refresh();
+        }
+    }
+
+    /**
+     * Test that a key previously scheduled for refresh will <b>not</b> be scheduled a second time
+     * if it's requested right away.
+     */
+
+    @Test
+    public void testScheduleRefreshForMissingKeyNoDelay() throws Exception {
+        String keyId = "abc123";
+        MockTime time = new MockTime();
+        HttpsJwks httpsJwks = mockHttpsJwks();
+
+        try (RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS)) {
+            refreshingHttpsJwks.init();
+            assertTrue(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+            assertFalse(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+        }
+    }
+
+    /**
+     * Test that a key previously scheduled for refresh <b>will</b> be scheduled a second time
+     * if it's requested after the delay.
+     */
+
+    @Test
+    public void testScheduleRefreshForMissingKeyDelays() throws Exception {
+        assertScheduleRefreshForMissingKeyWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS - 1, false);
+        assertScheduleRefreshForMissingKeyWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS, true);
+        assertScheduleRefreshForMissingKeyWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS + 1, true);
+    }
+
+    @Test
+    public void testLongKey() throws Exception {
+        char[] keyIdChars = new char[MISSING_KEY_ID_MAX_KEY_LENGTH + 1];
+        Arrays.fill(keyIdChars, '0');
+        String keyId = new String(keyIdChars);
+
+        MockTime time = new MockTime();
+        HttpsJwks httpsJwks = mockHttpsJwks();
+
+        try (RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS)) {
+            refreshingHttpsJwks.init();
+            assertFalse(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+            verify(httpsJwks, times(0)).refresh();
+        }
+    }
+
+    private void assertScheduleRefreshForMissingKeyWithDelay(long sleepDelay, boolean shouldBeScheduled) throws Exception {
+        String keyId = "abc123";
+        MockTime time = new MockTime();
+        HttpsJwks httpsJwks = mockHttpsJwks();
+
+        try (RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS)) {
+            refreshingHttpsJwks.init();
+            assertTrue(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+            time.sleep(sleepDelay);
+            assertEquals(shouldBeScheduled, refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+        }
+    }
+
+    private HttpsJwks mockHttpsJwks() throws Exception {
+        return mockHttpsJwks(Collections.emptyList());
+    }
+
+    private HttpsJwks mockHttpsJwks(List<JsonWebKey> objects) throws Exception {
+        HttpsJwks httpsJwks = mock(HttpsJwks.class);
+        when(httpsJwks.getJsonWebKeys()).thenReturn(objects);
+        when(httpsJwks.getLocation()).thenReturn("https://www.example.com");
+        return httpsJwks;
+    }
+
+    private JsonWebKey mockJsonWebKey(String keyId) {

Review comment:
       It was, then wasn't, then was, and now isn't again. I will remove it. Thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734905531



##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerTest.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.secured;
+
+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;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+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.util.Base64;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Random;
+import java.util.function.Consumer;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.authenticator.TestJaasConfig;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.utils.Utils;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.TestInstance.Lifecycle;
+import org.junit.jupiter.api.function.Executable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@TestInstance(Lifecycle.PER_CLASS)
+public abstract class OAuthBearerTest {
+
+    protected final Logger log = LoggerFactory.getLogger(getClass());
+
+    protected ObjectMapper mapper = new ObjectMapper();
+
+    protected void assertThrowsWithMessage(Class<? extends Exception> clazz,
+        Executable executable,
+        String substring) {
+        boolean failed = false;
+
+        try {
+            executable.execute();
+        } catch (Throwable t) {
+            failed = true;
+            assertTrue(clazz.isInstance(t), String.format("Test failed by exception %s, but expected %s", t.getClass(), clazz));
+
+            assertErrorMessageContains(t.getMessage(), substring);
+        }
+
+        if (!failed)
+            fail("Expected test to fail with " + clazz + " that contains the string " + substring);
+    }
+
+    protected void assertErrorMessageContains(String actual, String expectedSubstring) {
+        assertTrue(actual.contains(expectedSubstring),
+            String.format("Expected exception message (\"%s\") to contain substring (\"%s\")",
+                actual,
+                expectedSubstring));
+    }
+
+    protected void configureHandler(AuthenticateCallbackHandler handler,
+        Map<String, ?> configs,
+        Map<String, Object> jaasConfig) {
+        TestJaasConfig config = new TestJaasConfig();
+        config.createOrUpdateEntry("KafkaClient", OAuthBearerLoginModule.class.getName(), jaasConfig);
+        AppConfigurationEntry kafkaClient = config.getAppConfigurationEntry("KafkaClient")[0];
+
+        handler.configure(configs,
+            OAuthBearerLoginModule.OAUTHBEARER_MECHANISM,
+            Collections.singletonList(kafkaClient));
+    }
+
+    protected String createBase64JsonJwtSection(Consumer<ObjectNode> c) {
+        String json = createJsonJwtSection(c);
+
+        try {
+            return Utils.utf8(Base64.getEncoder().encode(Utils.utf8(json)));
+        } catch (Throwable t) {
+            fail(t);
+
+            // Shouldn't get to here...
+            return null;
+        }
+    }
+
+    protected String createJsonJwtSection(Consumer<ObjectNode> c) {
+        ObjectNode node = mapper.createObjectNode();
+        c.accept(node);
+
+        try {
+            return mapper.writeValueAsString(node);
+        } catch (Throwable t) {
+            fail(t);
+
+            // Shouldn't get to here...
+            return null;
+        }
+    }
+
+    protected Retryable<String> createRetryable(Exception[] attempts) {
+        Iterator<Exception> i = Arrays.asList(attempts).iterator();
+
+        return () -> {
+            Exception e = i.hasNext() ? i.next() : null;
+
+            if (e == null) {
+                return "success!";
+            } else {
+                if (e instanceof IOException)
+                    throw (IOException) 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 createTempPemDir() throws IOException {

Review comment:
       It is, and it will be removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734905551



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/LoginAccessTokenValidator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
+import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerIllegalTokenException;
+import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredJws;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * LoginAccessTokenValidator is an implementation of {@link AccessTokenValidator} that is used
+ * by the client to perform some rudimentary validation of the JWT access token that is received
+ * as part of the response from posting the client credentials to the OAuth/OIDC provider's
+ * token endpoint.
+ *
+ * The validation steps performed are:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory map</li>
+ *     <li>Presence of scope, <code>exp</code>, subject, and <code>iat</code> claims</li>
+ * </ol>
+ */
+
+public class LoginAccessTokenValidator implements AccessTokenValidator {
+
+    private static final Logger log = LoggerFactory.getLogger(LoginAccessTokenValidator.class);
+
+    public static final String EXPIRATION_CLAIM_NAME = "exp";
+
+    public static final String ISSUED_AT_CLAIM_NAME = "iat";
+
+    private final String scopeClaimName;
+
+    private final String subClaimName;
+
+    /**
+     * Creates a new LoginAccessTokenValidator that will be used by the client for lightweight
+     * validation of the JWT.
+     */
+
+    public LoginAccessTokenValidator() {

Review comment:
       Correct. It has been removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r737758788



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.
+                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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, this refresh method
+        // will still be called again within RETRY_BACKOFF_MS :)
+        //
+        // Note: timeBeforeNextRefresh is negative when we're in the midst of executing refresh
+        // in a scheduled fashion. ScheduledFuture.getDelay will reset *after* the method has

Review comment:
       Changed the logic to be simpler: if we're in the middle of a `refresh` in thread `A` and thread `B` starts to execute `refresh` (either scheduled or expedited), we simply let thread `B` return. This is implemented via a quick CAS at the start of `refresh` using an `AtomicBoolean`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732258641



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class);
+
+    public static final String CLIENT_ID_CONFIG = "clientId";
+    public static final String CLIENT_SECRET_CONFIG = "clientSecret";
+    public static final String SCOPE_CONFIG = "scope";
+
+    public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client ID to uniquely identify the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " +
+        "account and identifies the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " +
+        "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI + ") may need to specify an " +
+        "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " +
+        "include with the login request.";
+
+    private static final String EXTENSION_PREFIX = "extension_";
+
+    private Map<String, Object> moduleOptions;
+
+    private AccessTokenRetriever accessTokenRetriever;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> 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()));
+
+        moduleOptions = Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+        AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism);
+        configure(accessTokenRetriever, accessTokenValidator);
+    }
+
+    public void configure(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator accessTokenValidator) {
+        this.accessTokenRetriever = accessTokenRetriever;
+        this.accessTokenValidator = accessTokenValidator;
+
+        try {
+            this.accessTokenRetriever.init();
+        } catch (IOException e) {
+            throw new KafkaException("The OAuth login configuration encountered an error when initializing the AccessTokenRetriever", e);
+        }
+
+        isConfigured = true;
+    }
+
+    @Override
+    public void close() {
+        if (accessTokenRetriever != null) {
+            try {
+                this.accessTokenRetriever.close();
+            } catch (IOException e) {
+                log.warn("The OAuth login configuration encountered an error when closing the AccessTokenRetriever", e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkConfigured();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerTokenCallback) {
+                handle((OAuthBearerTokenCallback) callback);
+            } else if (callback instanceof SaslExtensionsCallback) {
+                handle((SaslExtensionsCallback) callback);
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    void handle(OAuthBearerTokenCallback callback) throws IOException {

Review comment:
       Sure! I changed it to be more specific.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r731343661



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {

Review comment:
       saslMechanism seems to be used for the listener prefix. 
   (1) Could we name it as listener name?
   (2) The listener name is only relevant for the broker. Could we avoid passing in that for the client?
   (3) Listener name is not always the sasl mechanism name. Should we pass in the listener name?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());
+                requestParameters.append("&scope=").append(encodedScope);
+            }
+
+            return requestParameters.toString();
+        } catch (UnsupportedEncodingException e) {
+            // The world has gone crazy!
+            throw new IOException(String.format("Encoding %s not supported", StandardCharsets.UTF_8.name()));
+        }
+    }
+
+    private static String sanitizeString(String name, String value) throws IOException {
+        if (value == null)
+            throw new IOException(String.format("%s value must be non-null", name));
+
+        if (value.isEmpty())
+            throw new IOException(String.format("%s value must be non-empty", name));
+
+        value = value.trim();
+
+        if (value.isEmpty())
+            throw new IOException(String.format("%s value must not contain only whitespace", name));

Review comment:
       name -> value ?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/PemDirectoryVerificationKeyResolver.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.security.Key;
+import java.security.PublicKey;
+import java.security.interfaces.ECPublicKey;
+import java.security.interfaces.RSAPublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.kafka.common.utils.Utils;
+import org.jose4j.jwk.EllipticCurveJsonWebKey;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.jwk.RsaJsonWebKey;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.keys.EcKeyUtil;
+import org.jose4j.keys.RsaKeyUtil;
+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;
+
+/**
+ * <code>PemVerificationKeyResolver</code> is a {@link VerificationKeyResolver} implementation

Review comment:
       So, PemVerificationKeyResolver is only used for testing and not for production use?

##########
File path: build.gradle
##########
@@ -1604,6 +1608,7 @@ project(':tools') {
     implementation libs.slf4jApi
     implementation libs.log4j
 
+    implementation libs.jose4j

Review comment:
       Is this needed since tools depends on clients, which already has this dependency?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {
+        this.configs = configs;
+
+        if (saslMechanism != null && !saslMechanism.trim().isEmpty())
+            this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
+        else
+            this.prefix = null;
+    }
+
+    public Map<String, ?> getSslClientConfig(String uriConfigName) {
+        String urlConfigValue = get(uriConfigName);
+
+        if (urlConfigValue == null || urlConfigValue.trim().isEmpty())
+            throw new ConfigException(String.format("The OAuth configuration option %s is required", uriConfigName));
+
+        URL url;
+
+        try {
+            url = new URL(urlConfigValue);
+        } catch (IOException e) {
+            throw new ConfigException(String.format("The OAuth configuration option %s was not a valid URL (%s)", uriConfigName, urlConfigValue));
+        }
+
+        if (!url.getProtocol().equalsIgnoreCase("https")) {
+            log.warn("Not creating SSL socket factory as URL for {} ({}) is not SSL-/TLS-based", uriConfigName, url);
+            return null;
+        }
+
+        ConfigDef sslConfigDef = new ConfigDef();
+        sslConfigDef.withClientSslSupport();
+        AbstractConfig sslClientConfig = new AbstractConfig(sslConfigDef, configs);
+        return sslClientConfig.values();
+    }
+
+    public SSLSocketFactory createSSLSocketFactory(String uriConfigName) {
+        Map<String, ?> sslClientConfig = getSslClientConfig(uriConfigName);
+
+        if (sslClientConfig == null) {
+            log.warn("Requesting SSL client socket factory but SSL configs were null");
+            return null;
+        }
+
+        SslFactory sslFactory = new SslFactory(Mode.CLIENT);
+        sslFactory.configure(sslClientConfig);
+        SSLSocketFactory socketFactory = ((DefaultSslEngineFactory) sslFactory.sslEngineFactory()).sslContext().getSocketFactory();
+        log.debug("Created SSLSocketFactory: {}", sslClientConfig);
+        return socketFactory;
+    }
+
+    /**
+     * Validates that, if a value is supplied, is a file that:
+     *
+     * <li>
+     *     <ul>exists</ul>
+     *     <ul>has read permission</ul>
+     *     <ul>points to a file</ul>
+     * </li>
+     *
+     * 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) {
+        URI uri = validateUri(name);
+        File file = new File(uri.getRawPath()).getAbsoluteFile();
+
+        if (!file.exists())
+            throw new ConfigException(name, file, String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", name, file));
+
+        if (!file.canRead())
+            throw new ConfigException(name, file, 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(name, file, 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:
+     *
+     * <li>
+     *     <ul>is an Integer</ul>
+     *     <ul>has a value that is not less than the provided minimum value</ul>
+     * </li>
+     *
+     * 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) {
+        return validateInteger(name, true);
+    }
+
+    public Integer validateInteger(String name, boolean isRequired) {
+        return validateInteger(name, isRequired, null);
+    }
+
+    public Integer validateInteger(String name, boolean isRequired, Integer min) {
+        Integer value = get(name);
+
+        if (value == null) {
+            if (isRequired)
+                throw new ConfigException(name, null, String.format("The OAuth configuration option %s must be non-null", name));
+            else
+                return null;
+        }
+
+        if (min != null && value < min)
+            throw new ConfigException(name, value, String.format("The OAuth configuration option %s value must be at least %s", name, min));
+
+        return value;
+    }
+
+    /**
+     * Validates that, if a value is supplied, is a value that:
+     *
+     * <li>
+     *     <ul>is an Integer</ul>
+     *     <ul>has a value that is not less than the provided minimum value</ul>
+     * </li>
+     *
+     * 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(name, null, String.format("The OAuth configuration option %s must be non-null", name));
+            else
+                return null;
+        }
+
+        if (min != null && value < min)
+            throw new ConfigException(name, value, String.format("The OAuth configuration option %s value must be at least %s", name, min));
+
+        return value;
+    }
+
+    /**
+     * Validates that the configured URI that:
+     *
+     * <li>
+     *     <ul>is well-formed</ul>
+     *     <ul>contains a scheme</ul>
+     *     <ul>uses either HTTP, HTTPS, or file protocols</ul>
+     * </li>
+     *
+     * No effort is made to contact the URL in the validation step.
+     */
+
+    public URI validateUri(String name) {
+        String value = validateString(name);
+        URI uri;
+
+        try {
+            uri = new URI(value.trim());

Review comment:
       value is already trimmed in validateString(). So, no need to trim it again.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {
+        this.configs = configs;
+
+        if (saslMechanism != null && !saslMechanism.trim().isEmpty())
+            this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
+        else
+            this.prefix = null;
+    }
+
+    public Map<String, ?> getSslClientConfig(String uriConfigName) {
+        String urlConfigValue = get(uriConfigName);
+
+        if (urlConfigValue == null || urlConfigValue.trim().isEmpty())
+            throw new ConfigException(String.format("The OAuth configuration option %s is required", uriConfigName));
+
+        URL url;
+
+        try {
+            url = new URL(urlConfigValue);
+        } catch (IOException e) {
+            throw new ConfigException(String.format("The OAuth configuration option %s was not a valid URL (%s)", uriConfigName, urlConfigValue));
+        }
+
+        if (!url.getProtocol().equalsIgnoreCase("https")) {
+            log.warn("Not creating SSL socket factory as URL for {} ({}) is not SSL-/TLS-based", uriConfigName, url);
+            return null;
+        }
+
+        ConfigDef sslConfigDef = new ConfigDef();
+        sslConfigDef.withClientSslSupport();

Review comment:
       Hmm, these are ssl properties for connections between Kafka client and Kafka broker. Here, we need the ssl properties for connecting to the oauth provider. They may require a different set of configurations.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {

Review comment:
       Could we add a high level description of the class?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class);
+
+    public static final String CLIENT_ID_CONFIG = "clientId";
+    public static final String CLIENT_SECRET_CONFIG = "clientSecret";
+    public static final String SCOPE_CONFIG = "scope";
+
+    public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client ID to uniquely identify the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " +
+        "account and identifies the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " +
+        "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI + ") may need to specify an " +
+        "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " +
+        "include with the login request.";
+
+    private static final String EXTENSION_PREFIX = "extension_";
+
+    private Map<String, Object> moduleOptions;
+
+    private AccessTokenRetriever accessTokenRetriever;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> 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()));
+
+        moduleOptions = Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+        AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism);
+        configure(accessTokenRetriever, accessTokenValidator);
+    }
+
+    public void configure(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator accessTokenValidator) {
+        this.accessTokenRetriever = accessTokenRetriever;
+        this.accessTokenValidator = accessTokenValidator;
+
+        try {
+            this.accessTokenRetriever.init();
+        } catch (IOException e) {
+            throw new KafkaException("The OAuth login configuration encountered an error when initializing the AccessTokenRetriever", e);
+        }
+
+        isConfigured = true;
+    }
+
+    @Override
+    public void close() {
+        if (accessTokenRetriever != null) {
+            try {
+                this.accessTokenRetriever.close();
+            } catch (IOException e) {
+                log.warn("The OAuth login configuration encountered an error when closing the AccessTokenRetriever", e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkConfigured();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerTokenCallback) {
+                handle((OAuthBearerTokenCallback) callback);
+            } else if (callback instanceof SaslExtensionsCallback) {
+                handle((SaslExtensionsCallback) callback);
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    void handle(OAuthBearerTokenCallback callback) throws IOException {

Review comment:
       Could this method and the next one be private?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");

Review comment:
       Should we include responseBody in the exception?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {
+        this.configs = configs;
+
+        if (saslMechanism != null && !saslMechanism.trim().isEmpty())
+            this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
+        else
+            this.prefix = null;
+    }
+
+    public Map<String, ?> getSslClientConfig(String uriConfigName) {
+        String urlConfigValue = get(uriConfigName);
+
+        if (urlConfigValue == null || urlConfigValue.trim().isEmpty())
+            throw new ConfigException(String.format("The OAuth configuration option %s is required", uriConfigName));
+
+        URL url;
+
+        try {
+            url = new URL(urlConfigValue);
+        } catch (IOException e) {
+            throw new ConfigException(String.format("The OAuth configuration option %s was not a valid URL (%s)", uriConfigName, urlConfigValue));
+        }
+
+        if (!url.getProtocol().equalsIgnoreCase("https")) {
+            log.warn("Not creating SSL socket factory as URL for {} ({}) is not SSL-/TLS-based", uriConfigName, url);
+            return null;
+        }
+
+        ConfigDef sslConfigDef = new ConfigDef();
+        sslConfigDef.withClientSslSupport();
+        AbstractConfig sslClientConfig = new AbstractConfig(sslConfigDef, configs);
+        return sslClientConfig.values();
+    }
+
+    public SSLSocketFactory createSSLSocketFactory(String uriConfigName) {
+        Map<String, ?> sslClientConfig = getSslClientConfig(uriConfigName);
+
+        if (sslClientConfig == null) {
+            log.warn("Requesting SSL client socket factory but SSL configs were null");

Review comment:
       Do we need this warning since we already log a warn in getSslClientConfig()?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/AccessTokenRetrieverFactory.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG;
+
+import java.net.URI;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+
+public class AccessTokenRetrieverFactory  {
+
+    /**
+     * Create an {@link AccessTokenRetriever} from the given SASL and JAAS configuration.
+     *
+     * <b>Note</b>: the returned <code>AccessTokenRetriever</code> is <em>not</em> initialized
+     * here and must be done by the caller prior to use.
+     *
+     * @param configs    SASL configuration
+     * @param jaasConfig JAAS configuration
+     *
+     * @return Non-<code>null</code> {@link AccessTokenRetriever}
+     */
+
+    public static AccessTokenRetriever create(Map<String, ?> configs, Map<String, Object> jaasConfig) {
+        return create(configs, null, jaasConfig);
+    }
+
+    public static AccessTokenRetriever create(Map<String, ?> configs, String saslMechanism, Map<String, Object> jaasConfig) {
+        ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism);
+        URI tokenEndpointUri = cu.validateUri(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI);
+
+        if (tokenEndpointUri.getScheme().toLowerCase(Locale.ROOT).equals("file")) {
+            return new FileTokenRetriever(cu.validateFile(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI));
+        } else {
+            ConfigurationUtils jaasCu = new ConfigurationUtils(jaasConfig);
+            String clientId = jaasCu.validateString(CLIENT_ID_CONFIG);
+            String clientSecret = jaasCu.validateString(CLIENT_SECRET_CONFIG);
+            String scope = jaasCu.get(SCOPE_CONFIG);
+
+            SSLSocketFactory sslSocketFactory = cu.createSSLSocketFactory(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI);
+
+            return new HttpAccessTokenRetriever(clientId,
+                clientSecret,
+                scope,
+                sslSocketFactory,
+                tokenEndpointUri.toString(),
+                cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MS),
+                cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS),
+                cu.validateInteger(SASL_LOGIN_CONNECT_TIMEOUT_MS, false, null),

Review comment:
       This could just use the constructor without the min value. Ditto below.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());
+                requestParameters.append("&scope=").append(encodedScope);
+            }
+
+            return requestParameters.toString();
+        } catch (UnsupportedEncodingException e) {
+            // The world has gone crazy!
+            throw new IOException(String.format("Encoding %s not supported", StandardCharsets.UTF_8.name()));
+        }
+    }
+
+    private static String sanitizeString(String name, String value) throws IOException {
+        if (value == null)
+            throw new IOException(String.format("%s value must be non-null", name));

Review comment:
       Hmm, this is not really an IOException. Perhaps it should be sth like IllegalArgument. Ditto below.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());
+                requestParameters.append("&scope=").append(encodedScope);
+            }
+
+            return requestParameters.toString();
+        } catch (UnsupportedEncodingException e) {
+            // The world has gone crazy!
+            throw new IOException(String.format("Encoding %s not supported", StandardCharsets.UTF_8.name()));
+        }
+    }
+
+    private static String sanitizeString(String name, String value) throws IOException {
+        if (value == null)
+            throw new IOException(String.format("%s value must be non-null", name));
+
+        if (value.isEmpty())
+            throw new IOException(String.format("%s value must be non-empty", name));

Review comment:
       name -> value ?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory data structure</li>
+ *     <li>
+ *         Presence of scope, <code>exp</code>, subject, <code>iss</code>, and
+ *         <code>iat</code> claims
+ *     </li>
+ *     <li>
+ *         Signature matching validation against the <code>kid</code> and those provided b
+ *         the OAuth/OIDC provider's JWKS
+ *     </li>
+ * </ol>
+ */
+
+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 <code>null</code> 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
+     *                                <code>aud</code> claim and if this value is set, the
+     *                                broker will match the value from JWT's <code>aud</code>
+     *                                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 <code>null</code> to not perform an

Review comment:
       `May be <code>null</code> to not perform an` seems incomplete.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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

Review comment:
       It seems that ValidatorAccessTokenValidator is used for validation in the client too?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {

Review comment:
       Could we add a comment to this class?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class);
+
+    public static final String CLIENT_ID_CONFIG = "clientId";
+    public static final String CLIENT_SECRET_CONFIG = "clientSecret";
+    public static final String SCOPE_CONFIG = "scope";
+
+    public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client ID to uniquely identify the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " +
+        "account and identifies the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " +
+        "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI + ") may need to specify an " +
+        "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " +
+        "include with the login request.";
+
+    private static final String EXTENSION_PREFIX = "extension_";
+
+    private Map<String, Object> moduleOptions;
+
+    private AccessTokenRetriever accessTokenRetriever;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> 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()));
+
+        moduleOptions = Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+        AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions);

Review comment:
       saslMechanism is used for listener prefix. Since listener is a broker side thing it seems that we don't need to pass this in the client? Ditto below.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);

Review comment:
       saslMechanism is used as listener prefix. It seems that we need to pass in the listener name too?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/JwksFileVerificationKeyResolver.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.security.Key;
+import java.util.List;
+import org.apache.kafka.common.utils.Utils;
+import org.jose4j.jwk.JsonWebKeySet;
+import org.jose4j.jws.JsonWebSignature;
+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;
+
+/**
+ * <code>JwksFileVerificationKeyResolver</code> is a {@link VerificationKeyResolver} implementation
+ * that will load the JWKS from the given file system directory.

Review comment:
       Could we describe the file format in code and the KIP?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory data structure</li>
+ *     <li>
+ *         Presence of scope, <code>exp</code>, subject, <code>iss</code>, and
+ *         <code>iat</code> claims
+ *     </li>
+ *     <li>
+ *         Signature matching validation against the <code>kid</code> and those provided b

Review comment:
       provided b => provided by

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        init(verificationKeyResolver, accessTokenValidator);
+    }
+
+    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;
+    }
+
+    @Override
+    public void close() {
+        if (verificationKeyResolver != null) {
+            try {
+                verificationKeyResolver.close();
+            } catch (Exception e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkInitialized();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerValidatorCallback) {
+                handleValidatorCallback((OAuthBearerValidatorCallback) callback);
+            } else if (callback instanceof OAuthBearerExtensionsValidatorCallback) {
+                OAuthBearerExtensionsValidatorCallback extensionsCallback = (OAuthBearerExtensionsValidatorCallback) callback;
+                extensionsCallback.inputExtensions().map().forEach((extensionName, v) -> extensionsCallback.valid(extensionName));
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    private void handleValidatorCallback(OAuthBearerValidatorCallback callback) {
+        checkInitialized();
+
+        OAuthBearerToken token;
+
+        try {
+            token = accessTokenValidator.validate(callback.tokenValue());
+            log.debug("handle - token: {}", token);
+            callback.token(token);
+        } catch (ValidateException e) {
+            e.printStackTrace();

Review comment:
       In general, we shouldn't print to stdout and should use logging instead.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/PemDirectoryVerificationKeyResolver.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.security.Key;
+import java.security.PublicKey;
+import java.security.interfaces.ECPublicKey;
+import java.security.interfaces.RSAPublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.kafka.common.utils.Utils;
+import org.jose4j.jwk.EllipticCurveJsonWebKey;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.jwk.RsaJsonWebKey;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.keys.EcKeyUtil;
+import org.jose4j.keys.RsaKeyUtil;
+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;
+
+/**
+ * <code>PemVerificationKeyResolver</code> is a {@link VerificationKeyResolver} implementation
+ * that will load the PEM files from the given file system directory.
+ *
+ * The instance is configured with the directory name that contains one or more
+ * <a href=\"https://en.wikipedia.org/wiki/Privacy-Enhanced_Mail\">public key files</a>

Review comment:
       Is https://en.wikipedia.org/wiki/Privacy-Enhanced_Mail the right URL?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInited;
+
+    /**
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = localJWKs;

Review comment:
       Hmm, I am a bit confused. What is jsonWebKeys intended for? Why is it set during init() but not after refresh()?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/Retry.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Retry encapsulates the mechanism to perform a retry and then exponential
+ * backoff using provided wait times between attempts.
+ *
+ * @param <R> Result type
+ */
+
+public class Retry<R> {

Review comment:
       This and Retryable are not oauth specific. Perhaps they can be put in common.utils?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory data structure</li>
+ *     <li>
+ *         Presence of scope, <code>exp</code>, subject, <code>iss</code>, and
+ *         <code>iat</code> claims
+ *     </li>
+ *     <li>
+ *         Signature matching validation against the <code>kid</code> and those provided b
+ *         the OAuth/OIDC provider's JWKS
+ *     </li>
+ * </ol>
+ */
+
+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 <code>null</code> 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
+     *                                <code>aud</code> claim and if this value is set, the
+     *                                broker will match the value from JWT's <code>aud</code>
+     *                                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 <code>null</code> to not perform an
+     * @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 <code>iss</code> claim
+     *                                and if this value is set, the broker will match it exactly
+     *                                against what is in the JWT's <code>iss</code> claim.
+     *                                If there is no match, the broker will reject the JWT and
+     *                                authentication will fail.

Review comment:
       What happens when expectedIssuer is null?

##########
File path: build.gradle
##########
@@ -829,6 +829,7 @@ project(':core') {
     implementation libs.jacksonDataformatCsv
     implementation libs.jacksonJDK8Datatypes
     implementation libs.joptSimple
+    implementation libs.jose4j

Review comment:
       Is this needed since core depends on clients, which already has this dependency?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInited;
+
+    /**
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = localJWKs;
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            executorService.scheduleAtFixedRate(this::refreshInternal,
+                0,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInited = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    @Override
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInited)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refreshInternal() {
+        try {
+            log.info("JWKS validation key refresh of {} starting", getLocation());
+
+            // Call the *actual* refresh implementation.
+            refresh();
+
+            List<JsonWebKey> jwks = getJsonWebKeys();
+
+            try {
+                refreshLock.writeLock().lock();
+
+                for (JsonWebKey jwk : jwks)
+                    missingKeyIds.remove(jwk.getKeyId());
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            log.info("JWKS validation key refresh of {} complete", getLocation());
+        } catch (JoseException | IOException e) {
+            // Let's wait a random, but short amount of time before trying again.
+            long waitMs = ThreadLocalRandom.current().nextLong(1000, 10000);

Review comment:
       Should we backoff exponentially up to the next refresh interval?

##########
File path: clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
##########
@@ -75,30 +78,111 @@
     public static final String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = "Login refresh thread will sleep until the specified window factor relative to the"
             + " credential's lifetime has been reached, at which time it will try to refresh the credential."
             + " Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a default value of 0.8 (80%) is used"
-            + " if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR = 0.80;
 
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER = "sasl.login.refresh.window.jitter";
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = "The maximum amount of random jitter relative to the credential's lifetime"
             + " that is added to the login refresh thread's sleep time. Legal values are between 0 and 0.25 (25%) inclusive;"
-            + " a default value of 0.05 (5%) is used if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " a default value of 0.05 (5%) is used if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_JITTER = 0.05;
 
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS = "sasl.login.refresh.min.period.seconds";
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = "The desired minimum time for the login refresh thread to wait before refreshing a credential,"
             + " in seconds. Legal values are between 0 and 900 (15 minutes); a default value of 60 (1 minute) is used if no value is specified.  This value and "
             + " sasl.login.refresh.buffer.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS = 60;
 
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS = "sasl.login.refresh.buffer.seconds";
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = "The amount of buffer time before credential expiration to maintain when refreshing a credential,"
             + " in seconds. If a refresh would otherwise occur closer to expiration than the number of buffer seconds then the refresh will be moved up to maintain"
             + " as much of the buffer time as possible. Legal values are between 0 and 3600 (1 hour); a default value of  300 (5 minutes) is used if no value is specified."
             + " This value and sasl.login.refresh.min.period.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS = 300;
 
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS = "sasl.login.connect.timeout.ms";
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider connection timeout."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS = "sasl.login.read.timeout.ms";
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider read timeout."
+            + OAUTHBEARER_NOTE;
+
+    // These are only specified here outside their normal groupings so that they can be
+    // forward referencing.
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MS = "sasl.login.retry.backoff.ms";
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS = "sasl.login.retry.backoff.max.ms";
+
+    private static final String EXPONENTIAL_BACKOFF_NOTE = " Login uses an exponential backoff algorithm with an initial wait based on the"
+            + " " + SASL_LOGIN_RETRY_BACKOFF_MS
+            + " setting and will double in wait length between attempts up to a maximum wait length specified by the"
+            + " " + SASL_LOGIN_RETRY_BACKOFF_MAX_MS
+            + " setting.";
+
+    public static final long DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS = 10000;
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC = "The (optional) value in milliseconds for the maximum wait between login attempts to the"
+            + " external authentication provider."
+            + EXPONENTIAL_BACKOFF_NOTE;
+
+    public static final long DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS = 100;
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MS_DOC = "The (optional) value in milliseconds for the initial wait between login attempts to the external"
+            + " authentication provider."
+            + EXPONENTIAL_BACKOFF_NOTE;
+
+    public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME = "sasl.oauthbearer.scope.claim.name";
+    public static final String DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME = "scope";
+    public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC = "The OAuth claim for the scope is often named \"" + DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME + "\", but this (optional)"
+            + " setting can provide a different name to use for the scope included in the JWT payload's claims if the OAuth/OIDC provider uses a different"
+            + " name for that claim.";
+
+    public static final String SASL_OAUTHBEARER_SUB_CLAIM_NAME = "sasl.oauthbearer.sub.claim.name";
+    public static final String DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME = "sub";
+    public static final String SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC = "The OAuth claim for the subject is often named \"" + DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME + "\", but this (optional)"
+            + " setting can provide a different name to use for the subject included in the JWT payload's claims if the OAuth/OIDC provider uses a different"
+            + " name for that claim.";
+
+    public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI = "sasl.oauthbearer.token.endpoint.uri";
+    public static final String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI_DOC = "The URL for the OAuth/OIDC identity provider. If the URL is HTTP(S)-based, it is the issuer's token"
+            + " endpoint URI to which requests will be made to login based on the configuration in " + SASL_JAAS_CONFIG + ". If the URL is file-based, it"
+            + " specifies a file containing an access token (in JWT serialized form) issued by the OAuth/OIDC identity provider to use for authorization.";
+
+    // These are only specified here outside their normal groupings so that they can be
+    // forward referencing.
+    public static final String SASL_OAUTHBEARER_JWKS_ENDPOINT_URI = "sasl.oauthbearer.jwks.endpoint.uri";
+    public static final String SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS = "sasl.oauthbearer.jwks.endpoint.refresh.ms";

Review comment:
       Should we keep this close to its default and doc?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInited;
+
+    /**
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = localJWKs;
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            executorService.scheduleAtFixedRate(this::refreshInternal,
+                0,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInited = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    @Override
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInited)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refreshInternal() {
+        try {
+            log.info("JWKS validation key refresh of {} starting", getLocation());
+
+            // Call the *actual* refresh implementation.
+            refresh();
+
+            List<JsonWebKey> jwks = getJsonWebKeys();
+
+            try {
+                refreshLock.writeLock().lock();
+
+                for (JsonWebKey jwk : jwks)
+                    missingKeyIds.remove(jwk.getKeyId());
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            log.info("JWKS validation key refresh of {} complete", getLocation());
+        } catch (JoseException | IOException e) {
+            // Let's wait a random, but short amount of time before trying again.
+            long waitMs = ThreadLocalRandom.current().nextLong(1000, 10000);
+
+            String message = String.format("JWKS validation key refresh of %s encountered an error; waiting %s ms before trying again",
+                getLocation(),
+                waitMs);
+            log.warn(message, e);
+
+            executorService.schedule(this::refreshInternal, waitMs, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    public boolean maybeScheduleRefreshForMissingKeyId(String keyId) {
+        if (keyId.length() > MISSING_KEY_ID_MAX_KEY_LENGTH) {
+            log.warn("Key ID starting with {} with length {} was too long to cache", keyId.substring(0, 16), keyId.length());
+            return false;
+        } else {
+            try {
+                refreshLock.writeLock().lock();
+
+                // If there's no entry in the missing key ID cache for the incoming key ID,
+                // or it has expired, schedule a refresh.
+                Long lastCheckTime = missingKeyIds.get(keyId);

Review comment:
       It seems that this is the next check time?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);

Review comment:
       Should we log this before throwing the IOException above?

##########
File path: clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
##########
@@ -75,30 +78,111 @@
     public static final String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = "Login refresh thread will sleep until the specified window factor relative to the"
             + " credential's lifetime has been reached, at which time it will try to refresh the credential."
             + " Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a default value of 0.8 (80%) is used"
-            + " if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR = 0.80;
 
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER = "sasl.login.refresh.window.jitter";
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = "The maximum amount of random jitter relative to the credential's lifetime"
             + " that is added to the login refresh thread's sleep time. Legal values are between 0 and 0.25 (25%) inclusive;"
-            + " a default value of 0.05 (5%) is used if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " a default value of 0.05 (5%) is used if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_JITTER = 0.05;
 
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS = "sasl.login.refresh.min.period.seconds";
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = "The desired minimum time for the login refresh thread to wait before refreshing a credential,"
             + " in seconds. Legal values are between 0 and 900 (15 minutes); a default value of 60 (1 minute) is used if no value is specified.  This value and "
             + " sasl.login.refresh.buffer.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS = 60;
 
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS = "sasl.login.refresh.buffer.seconds";
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = "The amount of buffer time before credential expiration to maintain when refreshing a credential,"
             + " in seconds. If a refresh would otherwise occur closer to expiration than the number of buffer seconds then the refresh will be moved up to maintain"
             + " as much of the buffer time as possible. Legal values are between 0 and 3600 (1 hour); a default value of  300 (5 minutes) is used if no value is specified."
             + " This value and sasl.login.refresh.min.period.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS = 300;
 
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS = "sasl.login.connect.timeout.ms";
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider connection timeout."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS = "sasl.login.read.timeout.ms";
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider read timeout."
+            + OAUTHBEARER_NOTE;
+
+    // These are only specified here outside their normal groupings so that they can be
+    // forward referencing.
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MS = "sasl.login.retry.backoff.ms";
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS = "sasl.login.retry.backoff.max.ms";
+
+    private static final String EXPONENTIAL_BACKOFF_NOTE = " Login uses an exponential backoff algorithm with an initial wait based on the"
+            + " " + SASL_LOGIN_RETRY_BACKOFF_MS
+            + " setting and will double in wait length between attempts up to a maximum wait length specified by the"
+            + " " + SASL_LOGIN_RETRY_BACKOFF_MAX_MS
+            + " setting.";

Review comment:
       Should we add OAUTHBEARER_NOTE?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733897632



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory data structure</li>
+ *     <li>
+ *         Presence of scope, <code>exp</code>, subject, <code>iss</code>, and
+ *         <code>iat</code> claims
+ *     </li>
+ *     <li>
+ *         Signature matching validation against the <code>kid</code> and those provided b
+ *         the OAuth/OIDC provider's JWKS
+ *     </li>
+ * </ol>
+ */
+
+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 <code>null</code> 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
+     *                                <code>aud</code> claim and if this value is set, the
+     *                                broker will match the value from JWT's <code>aud</code>
+     *                                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 <code>null</code> to not perform an

Review comment:
       Completed the thought so that it now reads:
   
   ```
   May be <code>null</code> to not perform any check to verify the JWT's <code>aud</code> claim matches any fixed set of known/expected audiences.
   ```
   
   Thanks for catching that.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733901463



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        init(verificationKeyResolver, accessTokenValidator);
+    }
+
+    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;
+    }
+
+    @Override
+    public void close() {
+        if (verificationKeyResolver != null) {
+            try {
+                verificationKeyResolver.close();
+            } catch (Exception e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkInitialized();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerValidatorCallback) {
+                handleValidatorCallback((OAuthBearerValidatorCallback) callback);
+            } else if (callback instanceof OAuthBearerExtensionsValidatorCallback) {
+                OAuthBearerExtensionsValidatorCallback extensionsCallback = (OAuthBearerExtensionsValidatorCallback) callback;
+                extensionsCallback.inputExtensions().map().forEach((extensionName, v) -> extensionsCallback.valid(extensionName));
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    private void handleValidatorCallback(OAuthBearerValidatorCallback callback) {
+        checkInitialized();
+
+        OAuthBearerToken token;
+
+        try {
+            token = accessTokenValidator.validate(callback.tokenValue());
+            log.debug("handle - token: {}", token);
+            callback.token(token);
+        } catch (ValidateException e) {
+            e.printStackTrace();

Review comment:
       Whoopsie! Removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733887473



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());
+                requestParameters.append("&scope=").append(encodedScope);
+            }
+
+            return requestParameters.toString();
+        } catch (UnsupportedEncodingException e) {
+            // The world has gone crazy!
+            throw new IOException(String.format("Encoding %s not supported", StandardCharsets.UTF_8.name()));
+        }
+    }
+
+    private static String sanitizeString(String name, String value) throws IOException {
+        if (value == null)
+            throw new IOException(String.format("%s value must be non-null", name));
+
+        if (value.isEmpty())
+            throw new IOException(String.format("%s value must be non-empty", name));

Review comment:
       It is correct as written, though obviously confusing.
   
   I changed the format string to:
   
   ```
   The value for %s must be non-null
   ```
   
   e.g.:
   
   ```
   The value for the token endpoint request client ID parameter must be non-null
   The value for the token endpoint response's access_token JSON attribute must not contain only whitespace
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734100491



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInited;
+
+    /**
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = localJWKs;

Review comment:
       Whoops! Refactoring gone wrong. It was meant to retrieve the JWKS outside of the lock (`super.getJsonWebKeys()`) and then assign it inside the write lock.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r737754724



##########
File path: clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
##########
@@ -75,30 +78,116 @@
     public static final String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = "Login refresh thread will sleep until the specified window factor relative to the"
             + " credential's lifetime has been reached, at which time it will try to refresh the credential."
             + " Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a default value of 0.8 (80%) is used"
-            + " if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR = 0.80;
 
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER = "sasl.login.refresh.window.jitter";
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = "The maximum amount of random jitter relative to the credential's lifetime"
             + " that is added to the login refresh thread's sleep time. Legal values are between 0 and 0.25 (25%) inclusive;"
-            + " a default value of 0.05 (5%) is used if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " a default value of 0.05 (5%) is used if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_JITTER = 0.05;
 
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS = "sasl.login.refresh.min.period.seconds";
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = "The desired minimum time for the login refresh thread to wait before refreshing a credential,"
             + " in seconds. Legal values are between 0 and 900 (15 minutes); a default value of 60 (1 minute) is used if no value is specified.  This value and "
             + " sasl.login.refresh.buffer.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS = 60;
 
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS = "sasl.login.refresh.buffer.seconds";
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = "The amount of buffer time before credential expiration to maintain when refreshing a credential,"
             + " in seconds. If a refresh would otherwise occur closer to expiration than the number of buffer seconds then the refresh will be moved up to maintain"
             + " as much of the buffer time as possible. Legal values are between 0 and 3600 (1 hour); a default value of  300 (5 minutes) is used if no value is specified."
             + " This value and sasl.login.refresh.min.period.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS = 300;
 
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS = "sasl.login.connect.timeout.ms";
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider connection timeout."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS = "sasl.login.read.timeout.ms";
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider read timeout."
+            + OAUTHBEARER_NOTE;
+
+    private static final String EXPONENTIAL_BACKOFF_NOTE = " Login uses an exponential backoff algorithm with an initial wait based on the"
+            + " sasl.login.retry.backoff.ms setting and will double in wait length between attempts up to a maximum wait length specified by the"
+            + " sasl.login.retry.backoff.max.ms setting."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS = "sasl.login.retry.backoff.max.ms";
+    public static final long DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS = 10000;
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC = "The (optional) value in milliseconds for the maximum wait between login attempts to the"
+            + " external authentication provider."
+            + EXPONENTIAL_BACKOFF_NOTE;

Review comment:
       Renamed `EXPONENTIAL_BACKOFF_NOTE` to `LOGIN_EXPONENTIAL_BACKOFF_NOTE` with the existing message.
   
   Added `JWKS_EXPONENTIAL_BACKOFF_NOTE` which reads:
   
   ```
   JWKS retrieval uses an exponential backoff algorithm with an initial wait based on
   the sasl.oauthbearer.jwks.endpoint.retry.backoff.ms setting and will double in wait
   length between attempts up to a maximum wait length specified by the
   sasl.oauthbearer.jwks.endpoint.retry.backoff.max.ms setting.
   ```
   
   Updated `SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS` and `SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS` to reference `JWKS_EXPONENTIAL_BACKOFF_NOTE`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r731507827



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = null;
+
+        try (InputStream is = con.getInputStream()) {
+            ByteArrayOutputStream os = new ByteArrayOutputStream();
+            log.debug("handleOutput - preparing to read request body to {}", con.getURL());
+            copy(is, os);
+            responseBody = os.toString(StandardCharsets.UTF_8.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());

Review comment:
       dummy question: does only the authorization header need to be Base64 encoded, the request body doesn't?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732258747



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        configure(verificationKeyResolver, accessTokenValidator);
+    }
+
+    public void configure(CloseableVerificationKeyResolver verificationKeyResolver, AccessTokenValidator accessTokenValidator) {

Review comment:
       Sure! I changed it to `init` here too.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732259126



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        configure(verificationKeyResolver, accessTokenValidator);
+    }
+
+    public void configure(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);
+        }
+
+        isConfigured = true;

Review comment:
       Unit tests will call `init` directly, which is OK. Changed the flag to `isInitialized` so it fits in better now.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733900761



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory data structure</li>
+ *     <li>
+ *         Presence of scope, <code>exp</code>, subject, <code>iss</code>, and
+ *         <code>iat</code> claims
+ *     </li>
+ *     <li>
+ *         Signature matching validation against the <code>kid</code> and those provided b
+ *         the OAuth/OIDC provider's JWKS
+ *     </li>
+ * </ol>
+ */
+
+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 <code>null</code> 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
+     *                                <code>aud</code> claim and if this value is set, the
+     *                                broker will match the value from JWT's <code>aud</code>
+     *                                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 <code>null</code> to not perform an
+     * @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 <code>iss</code> claim
+     *                                and if this value is set, the broker will match it exactly
+     *                                against what is in the JWT's <code>iss</code> claim.
+     *                                If there is no match, the broker will reject the JWT and
+     *                                authentication will fail.

Review comment:
       I generically copied and pasted from above so that it now reads:
   
   ```
   May be <code>null</code> to not perform any check to verify the JWT's <code>iss</code> claim matches a specific issuer.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733888042



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());
+                requestParameters.append("&scope=").append(encodedScope);
+            }
+
+            return requestParameters.toString();
+        } catch (UnsupportedEncodingException e) {
+            // The world has gone crazy!
+            throw new IOException(String.format("Encoding %s not supported", StandardCharsets.UTF_8.name()));
+        }
+    }
+
+    private static String sanitizeString(String name, String value) throws IOException {
+        if (value == null)
+            throw new IOException(String.format("%s value must be non-null", name));
+
+        if (value.isEmpty())
+            throw new IOException(String.format("%s value must be non-empty", name));
+
+        value = value.trim();
+
+        if (value.isEmpty())
+            throw new IOException(String.format("%s value must not contain only whitespace", name));

Review comment:
       I adjusted the format string likewise, as per the above comment.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734081500



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {

Review comment:
       Yes, I have added appropriate JavaDoc describing this callback handler and contrasting it with the other.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734086548



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {
+        this.configs = configs;
+
+        if (saslMechanism != null && !saslMechanism.trim().isEmpty())
+            this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
+        else
+            this.prefix = null;
+    }
+
+    public Map<String, ?> getSslClientConfig(String uriConfigName) {
+        String urlConfigValue = get(uriConfigName);
+
+        if (urlConfigValue == null || urlConfigValue.trim().isEmpty())
+            throw new ConfigException(String.format("The OAuth configuration option %s is required", uriConfigName));
+
+        URL url;
+
+        try {
+            url = new URL(urlConfigValue);
+        } catch (IOException e) {
+            throw new ConfigException(String.format("The OAuth configuration option %s was not a valid URL (%s)", uriConfigName, urlConfigValue));
+        }
+
+        if (!url.getProtocol().equalsIgnoreCase("https")) {
+            log.warn("Not creating SSL socket factory as URL for {} ({}) is not SSL-/TLS-based", uriConfigName, url);
+            return null;
+        }
+
+        ConfigDef sslConfigDef = new ConfigDef();
+        sslConfigDef.withClientSslSupport();

Review comment:
       The client SSL configuration for OAuth comes from the JAAS options, not the configuration. So the user can specify SSL options like this:
   
   ```
   sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required \
       clientId="foo" \
       clientSecret="bar" \
       scope="baz" \
       ssl.protocol="SSL"  \
       ssl.keystore.type="...." ;
   ```
   
   This allows the HTTP client connection to use the same set of SSL configuration when connecting to the OAuth provider.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#issuecomment-949090564


   @junrao - I believe I've addressed all of the points listed except:
   
   1. The listener/SASL mechanism prefix issues are really confusing to me because the code is currently behaving the way that I'm assuming it's supposed to. For example, if I have a listener named `foo` and an OAuth configuration named `bar`, `sasl.oauthbearer.bar=1` is picked up by my code unless there's a listener-specific override (e.g. `listener.name.foo.oauthbearer.sasl.oauthbearer.bar=2`). And if I specify a value for some other listener (e.g. `listener.name.someotherlistener.oauthbearer.sasl.oauthbearer.bar=3`, I don't see that value in the callback handler that is initialized for the `foo` listener.
   2. I need to implement exponential backoff for the case of on-demand refresh of the JWKS.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733864413



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);

Review comment:
       Moved the logging to before the exception is thrown.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733854104



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = 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.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());
+                requestParameters.append("&scope=").append(encodedScope);
+            }
+
+            return requestParameters.toString();
+        } catch (UnsupportedEncodingException e) {
+            // The world has gone crazy!
+            throw new IOException(String.format("Encoding %s not supported", StandardCharsets.UTF_8.name()));
+        }
+    }
+
+    private static String sanitizeString(String name, String value) throws IOException {
+        if (value == null)
+            throw new IOException(String.format("%s value must be non-null", name));

Review comment:
       Changed to `IllegalArgumentException` and updated relevant unit tests.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735902697



##########
File path: build.gradle
##########
@@ -1604,6 +1608,7 @@ project(':tools') {
     implementation libs.slf4jApi
     implementation libs.log4j
 
+    implementation libs.jose4j                    // for SASL/OAUTHBEARER JWT validation

Review comment:
       Added the dependency to `shell` and `connect.runtime` too.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733893905



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ValidatorAccessTokenValidator.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.secured;
+
+import static org.jose4j.jwa.AlgorithmConstraints.DISALLOW_NONE;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+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;
+
+/**
+ * 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

Review comment:
       Yes and no.
   
   The `AccessTokenValidator` interface is used by both client and broker, but the `ValidatorAccessTokenValidator` implementation is specifically designed to only be used on the broker while the `LoginAccessTokenValidator` is used as the client implementation. 
   
   The reason for this is that on the broker we have the ability to bundle utility libraries (like jose4j) that allow `ValidatorAccessTokenValidator` to perform more thorough validation of the JWT. We intentionally want to keep the client dependencies lightweight, so `LoginAccessTokenValidator` only performs basic parsing and sanity checking on the JWT.
   
   I made an attempt to document this rationale in the `AccessTokenValidator` interface documentation. Let me know if there's something more specific I should put in there.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734089482



##########
File path: build.gradle
##########
@@ -1604,6 +1608,7 @@ project(':tools') {
     implementation libs.slf4jApi
     implementation libs.log4j
 
+    implementation libs.jose4j

Review comment:
       It is needed here as per the explanation for the need for the dependency in `:core`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734100842



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInited;
+
+    /**
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = localJWKs;
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            executorService.scheduleAtFixedRate(this::refreshInternal,
+                0,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInited = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    @Override
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInited)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refreshInternal() {
+        try {
+            log.info("JWKS validation key refresh of {} starting", getLocation());
+
+            // Call the *actual* refresh implementation.
+            refresh();
+
+            List<JsonWebKey> jwks = getJsonWebKeys();
+
+            try {
+                refreshLock.writeLock().lock();
+
+                for (JsonWebKey jwk : jwks)
+                    missingKeyIds.remove(jwk.getKeyId());
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            log.info("JWKS validation key refresh of {} complete", getLocation());
+        } catch (JoseException | IOException e) {
+            // Let's wait a random, but short amount of time before trying again.
+            long waitMs = ThreadLocalRandom.current().nextLong(1000, 10000);
+
+            String message = String.format("JWKS validation key refresh of %s encountered an error; waiting %s ms before trying again",
+                getLocation(),
+                waitMs);
+            log.warn(message, e);
+
+            executorService.schedule(this::refreshInternal, waitMs, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    public boolean maybeScheduleRefreshForMissingKeyId(String keyId) {
+        if (keyId.length() > MISSING_KEY_ID_MAX_KEY_LENGTH) {
+            log.warn("Key ID starting with {} with length {} was too long to cache", keyId.substring(0, 16), keyId.length());
+            return false;
+        } else {
+            try {
+                refreshLock.writeLock().lock();
+
+                // If there's no entry in the missing key ID cache for the incoming key ID,
+                // or it has expired, schedule a refresh.
+                Long lastCheckTime = missingKeyIds.get(keyId);

Review comment:
       Yes. I renamed the variable.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733259046



##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/AccessTokenValidatorFactoryTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.kafka.common.KafkaException;
+import org.junit.jupiter.api.Test;
+
+public class AccessTokenValidatorFactoryTest extends OAuthBearerTest {
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorInit() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void init() throws IOException {
+                throw new IOException("My init had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+
+        assertThrowsWithMessage(
+            KafkaException.class, () -> handler.configure(accessTokenRetriever, accessTokenValidator), "encountered an error when initializing");
+    }
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void close() throws IOException {
+                throw new IOException("My close had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+        handler.configure(accessTokenRetriever, accessTokenValidator);
+
+        // Basically asserting this doesn't throw an exception :(

Review comment:
       The `AuthenticateCallbackHandler` interface's `close` method doesn't allow for exceptions. So in our implementation, we just catch any errors and log them.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733259234



##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/AccessTokenValidatorFactoryTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.kafka.common.KafkaException;
+import org.junit.jupiter.api.Test;
+
+public class AccessTokenValidatorFactoryTest extends OAuthBearerTest {
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorInit() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void init() throws IOException {
+                throw new IOException("My init had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+
+        assertThrowsWithMessage(
+            KafkaException.class, () -> handler.configure(accessTokenRetriever, accessTokenValidator), "encountered an error when initializing");
+    }
+
+    @Test
+    public void testConfigureThrowsExceptionOnAccessTokenValidatorClose() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = new AccessTokenRetriever() {
+            @Override
+            public void close() throws IOException {
+                throw new IOException("My close had an error!");
+            }
+            @Override
+            public String retrieve() {
+                return "dummy";
+            }
+        };
+
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+        handler.configure(accessTokenRetriever, accessTokenValidator);
+
+        // Basically asserting this doesn't throw an exception :(
+        handler.close();
+    }
+
+    private OAuthBearerLoginCallbackHandler createHandler(AccessTokenRetriever accessTokenRetriever, Map<String, ?> configs) {

Review comment:
       Correct. I removed it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733851806



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {
+        this.configs = configs;
+
+        if (saslMechanism != null && !saslMechanism.trim().isEmpty())
+            this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
+        else
+            this.prefix = null;
+    }
+
+    public Map<String, ?> getSslClientConfig(String uriConfigName) {
+        String urlConfigValue = get(uriConfigName);
+
+        if (urlConfigValue == null || urlConfigValue.trim().isEmpty())
+            throw new ConfigException(String.format("The OAuth configuration option %s is required", uriConfigName));
+
+        URL url;
+
+        try {
+            url = new URL(urlConfigValue);
+        } catch (IOException e) {
+            throw new ConfigException(String.format("The OAuth configuration option %s was not a valid URL (%s)", uriConfigName, urlConfigValue));
+        }
+
+        if (!url.getProtocol().equalsIgnoreCase("https")) {
+            log.warn("Not creating SSL socket factory as URL for {} ({}) is not SSL-/TLS-based", uriConfigName, url);
+            return null;
+        }
+
+        ConfigDef sslConfigDef = new ConfigDef();
+        sslConfigDef.withClientSslSupport();
+        AbstractConfig sslClientConfig = new AbstractConfig(sslConfigDef, configs);
+        return sslClientConfig.values();
+    }
+
+    public SSLSocketFactory createSSLSocketFactory(String uriConfigName) {
+        Map<String, ?> sslClientConfig = getSslClientConfig(uriConfigName);
+
+        if (sslClientConfig == null) {
+            log.warn("Requesting SSL client socket factory but SSL configs were null");
+            return null;
+        }
+
+        SslFactory sslFactory = new SslFactory(Mode.CLIENT);
+        sslFactory.configure(sslClientConfig);
+        SSLSocketFactory socketFactory = ((DefaultSslEngineFactory) sslFactory.sslEngineFactory()).sslContext().getSocketFactory();
+        log.debug("Created SSLSocketFactory: {}", sslClientConfig);
+        return socketFactory;
+    }
+
+    /**
+     * Validates that, if a value is supplied, is a file that:
+     *
+     * <li>
+     *     <ul>exists</ul>
+     *     <ul>has read permission</ul>
+     *     <ul>points to a file</ul>
+     * </li>
+     *
+     * 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) {
+        URI uri = validateUri(name);
+        File file = new File(uri.getRawPath()).getAbsoluteFile();
+
+        if (!file.exists())
+            throw new ConfigException(name, file, String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", name, file));
+
+        if (!file.canRead())
+            throw new ConfigException(name, file, 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(name, file, 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:
+     *
+     * <li>
+     *     <ul>is an Integer</ul>
+     *     <ul>has a value that is not less than the provided minimum value</ul>
+     * </li>
+     *
+     * 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) {
+        return validateInteger(name, true);
+    }
+
+    public Integer validateInteger(String name, boolean isRequired) {
+        return validateInteger(name, isRequired, null);
+    }
+
+    public Integer validateInteger(String name, boolean isRequired, Integer min) {
+        Integer value = get(name);
+
+        if (value == null) {
+            if (isRequired)
+                throw new ConfigException(name, null, String.format("The OAuth configuration option %s must be non-null", name));
+            else
+                return null;
+        }
+
+        if (min != null && value < min)
+            throw new ConfigException(name, value, String.format("The OAuth configuration option %s value must be at least %s", name, min));
+
+        return value;
+    }
+
+    /**
+     * Validates that, if a value is supplied, is a value that:
+     *
+     * <li>
+     *     <ul>is an Integer</ul>
+     *     <ul>has a value that is not less than the provided minimum value</ul>
+     * </li>
+     *
+     * 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(name, null, String.format("The OAuth configuration option %s must be non-null", name));
+            else
+                return null;
+        }
+
+        if (min != null && value < min)
+            throw new ConfigException(name, value, String.format("The OAuth configuration option %s value must be at least %s", name, min));
+
+        return value;
+    }
+
+    /**
+     * Validates that the configured URI that:
+     *
+     * <li>
+     *     <ul>is well-formed</ul>
+     *     <ul>contains a scheme</ul>
+     *     <ul>uses either HTTP, HTTPS, or file protocols</ul>
+     * </li>
+     *
+     * No effort is made to contact the URL in the validation step.
+     */
+
+    public URI validateUri(String name) {
+        String value = validateString(name);
+        URI uri;
+
+        try {
+            uri = new URI(value.trim());

Review comment:
       Extraneous `trim` removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735929417



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current

Review comment:
       th => the

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.
+                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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, this refresh method
+        // will still be called again within RETRY_BACKOFF_MS :)
+        //
+        // Note: timeBeforeNextRefresh is negative when we're in the midst of executing refresh
+        // in a scheduled fashion. ScheduledFuture.getDelay will reset *after* the method has
+        // completed.
+        if (timeBeforeNextRefresh > 0 && timeBeforeNextRefresh < RETRY_BACKOFF_MS) {
+            log.info("OAuth JWKS refresh does not have enough time before next scheduled refresh");
+            return;
+        }
+
+        try {
+            log.info("OAuth JWKS refresh of {} starting", httpsJwks.getLocation());
+            Retry<List<JsonWebKey>> retry = new Retry<>(RETRY_BACKOFF_MS, timeBeforeNextRefresh);
+            List<JsonWebKey> localJWKs = retry.execute(() -> {
+                try {
+                    log.debug("JWKS validation key calling refresh of {} starting", httpsJwks.getLocation());
+                    // Call the *actual* refresh implementation that will more than likely issue
+                    // HTTP(S) calls over the network.
+                    httpsJwks.refresh();
+                    List<JsonWebKey> jwks = httpsJwks.getJsonWebKeys();
+                    log.debug("JWKS validation key refresh of {} complete", httpsJwks.getLocation());
+                    return jwks;
+                } catch (Exception e) {
+                    throw new ExecutionException(e);
+                }
+            });
+
+            try {
+                refreshLock.writeLock().lock();
+
+                for (JsonWebKey jwk : localJWKs)
+                    missingKeyIds.remove(jwk.getKeyId());
+
+                jsonWebKeys = Collections.unmodifiableList(localJWKs);
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            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);
+        }
+    }
+
+    public boolean maybeScheduleRefreshForMissingKeyId(String keyId) {
+        if (keyId.length() > MISSING_KEY_ID_MAX_KEY_LENGTH) {
+            // Only grab the first N characters so that if the key ID is huge, we don't blow up

Review comment:
       don't blow up => will blow up ?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.
+                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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, this refresh method
+        // will still be called again within RETRY_BACKOFF_MS :)
+        //
+        // Note: timeBeforeNextRefresh is negative when we're in the midst of executing refresh
+        // in a scheduled fashion. ScheduledFuture.getDelay will reset *after* the method has

Review comment:
       Hmm, if we're in the midst of executing refresh, it seems that should return immediately instead of refreshing?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.

Review comment:
       run => running

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {

Review comment:
       A meta question, currently, it seems that each processor creates its own sasl callbacks, each of which then creates its own RefreshingHttpsJwks. This means that each processor will make its own http requests and maintain its own key cache. This could add overhead. It seems that a better approach is to have all processors for the same listener share the http requests and the key cache. What do you think?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735981362



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.

Review comment:
       "run" sounds right to me, but I'll change it to "running" because I tend to forgot rules of grammar 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736009184



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.
+                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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, this refresh method
+        // will still be called again within RETRY_BACKOFF_MS :)
+        //
+        // Note: timeBeforeNextRefresh is negative when we're in the midst of executing refresh
+        // in a scheduled fashion. ScheduledFuture.getDelay will reset *after* the method has
+        // completed.
+        if (timeBeforeNextRefresh > 0 && timeBeforeNextRefresh < RETRY_BACKOFF_MS) {
+            log.info("OAuth JWKS refresh does not have enough time before next scheduled refresh");
+            return;
+        }
+
+        try {
+            log.info("OAuth JWKS refresh of {} starting", httpsJwks.getLocation());
+            Retry<List<JsonWebKey>> retry = new Retry<>(RETRY_BACKOFF_MS, timeBeforeNextRefresh);
+            List<JsonWebKey> localJWKs = retry.execute(() -> {
+                try {
+                    log.debug("JWKS validation key calling refresh of {} starting", httpsJwks.getLocation());
+                    // Call the *actual* refresh implementation that will more than likely issue
+                    // HTTP(S) calls over the network.
+                    httpsJwks.refresh();
+                    List<JsonWebKey> jwks = httpsJwks.getJsonWebKeys();
+                    log.debug("JWKS validation key refresh of {} complete", httpsJwks.getLocation());
+                    return jwks;
+                } catch (Exception e) {
+                    throw new ExecutionException(e);
+                }
+            });
+
+            try {
+                refreshLock.writeLock().lock();
+
+                for (JsonWebKey jwk : localJWKs)
+                    missingKeyIds.remove(jwk.getKeyId());
+
+                jsonWebKeys = Collections.unmodifiableList(localJWKs);
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            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);
+        }
+    }
+
+    public boolean maybeScheduleRefreshForMissingKeyId(String keyId) {
+        if (keyId.length() > MISSING_KEY_ID_MAX_KEY_LENGTH) {
+            // Only grab the first N characters so that if the key ID is huge, we don't blow up

Review comment:
       Changed to:
   
   ```
   // Although there's no limit on the length of the key ID, they're generally
   // "reasonably" short. If we have a very long key ID length, we're going to assume
   // the JWT is malformed and we will not actually try to resolve the key.
   // 
   // In this case, let's prevent blowing out our memory in two ways:
   // 
   //     1. Don't try to resolve the key as the large ID will sit in our cache
   //     2. Report the issue in the logs but include only the first N characters
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735772911



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> that will be used by the
+     * {@link RefreshingHttpsJwksVerificationKeyResolver} to resolve new key IDs in JWTs.
+     *
+     * @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
+     */
+
+    public RefreshingHttpsJwks(HttpsJwks httpsJwks, long refreshMs) {
+        this(Time.SYSTEM, httpsJwks, refreshMs);
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> 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();
+            }
+
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                0,

Review comment:
       Since we just retrieved from httpsJwks, should we set an initial delay?

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> that will be used by the
+     * {@link RefreshingHttpsJwksVerificationKeyResolver} to resolve new key IDs in JWTs.
+     *
+     * @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
+     */
+
+    public RefreshingHttpsJwks(HttpsJwks httpsJwks, long refreshMs) {
+        this(Time.SYSTEM, httpsJwks, refreshMs);
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> 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();
+            }
+
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                0,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, refreshInternal will

Review comment:
       refreshInternal no longer exists.

##########
File path: build.gradle
##########
@@ -1604,6 +1608,7 @@ project(':tools') {
     implementation libs.slf4jApi
     implementation libs.log4j
 
+    implementation libs.jose4j                    // for SASL/OAUTHBEARER JWT validation

Review comment:
       Should we add the same dependency to shell and connect:runtime?

##########
File path: build.gradle
##########
@@ -1208,6 +1210,8 @@ project(':clients') {
     testRuntimeOnly libs.slf4jlog4j
     testRuntimeOnly libs.jacksonDatabind
     testRuntimeOnly libs.jacksonJDK8Datatypes
+    testRuntimeOnly libs.jose4j
+    testImplementation libs.jose4j

Review comment:
       Hmm, I thought that testImplementation covers testRuntimeOnly?

##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwksTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.secured;
+
+import static org.apache.kafka.common.security.oauthbearer.secured.RefreshingHttpsJwks.MISSING_KEY_ID_CACHE_IN_FLIGHT_MS;
+import static org.apache.kafka.common.security.oauthbearer.secured.RefreshingHttpsJwks.MISSING_KEY_ID_MAX_KEY_LENGTH;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.kafka.common.utils.MockTime;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.junit.jupiter.api.Test;
+
+public class RefreshingHttpsJwksTest extends OAuthBearerTest {
+
+    public static final int REFRESH_MS = 10000;
+
+    /**
+     * Test that a key not previously scheduled for refresh will be scheduled without a refresh.
+     */
+
+    @Test
+    public void test() throws Exception {
+        String keyId = "abc123";
+        MockTime time = new MockTime();
+        HttpsJwks httpsJwks = mockHttpsJwks();
+
+        try (RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS)) {
+            refreshingHttpsJwks.init();
+            assertTrue(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+            verify(httpsJwks, times(0)).refresh();
+        }
+    }
+
+    /**
+     * Test that a key previously scheduled for refresh will <b>not</b> be scheduled a second time
+     * if it's requested right away.
+     */
+
+    @Test
+    public void testScheduleRefreshForMissingKeyNoDelay() throws Exception {
+        String keyId = "abc123";
+        MockTime time = new MockTime();
+        HttpsJwks httpsJwks = mockHttpsJwks();
+
+        try (RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS)) {
+            refreshingHttpsJwks.init();
+            assertTrue(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+            assertFalse(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+        }
+    }
+
+    /**
+     * Test that a key previously scheduled for refresh <b>will</b> be scheduled a second time
+     * if it's requested after the delay.
+     */
+
+    @Test
+    public void testScheduleRefreshForMissingKeyDelays() throws Exception {
+        assertScheduleRefreshForMissingKeyWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS - 1, false);
+        assertScheduleRefreshForMissingKeyWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS, true);
+        assertScheduleRefreshForMissingKeyWithDelay(MISSING_KEY_ID_CACHE_IN_FLIGHT_MS + 1, true);
+    }
+
+    @Test
+    public void testLongKey() throws Exception {
+        char[] keyIdChars = new char[MISSING_KEY_ID_MAX_KEY_LENGTH + 1];
+        Arrays.fill(keyIdChars, '0');
+        String keyId = new String(keyIdChars);
+
+        MockTime time = new MockTime();
+        HttpsJwks httpsJwks = mockHttpsJwks();
+
+        try (RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS)) {
+            refreshingHttpsJwks.init();
+            assertFalse(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+            verify(httpsJwks, times(0)).refresh();
+        }
+    }
+
+    private void assertScheduleRefreshForMissingKeyWithDelay(long sleepDelay, boolean shouldBeScheduled) throws Exception {
+        String keyId = "abc123";
+        MockTime time = new MockTime();
+        HttpsJwks httpsJwks = mockHttpsJwks();
+
+        try (RefreshingHttpsJwks refreshingHttpsJwks = new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS)) {
+            refreshingHttpsJwks.init();
+            assertTrue(refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+            time.sleep(sleepDelay);
+            assertEquals(shouldBeScheduled, refreshingHttpsJwks.maybeScheduleRefreshForMissingKeyId(keyId));
+        }
+    }
+
+    private HttpsJwks mockHttpsJwks() throws Exception {
+        return mockHttpsJwks(Collections.emptyList());
+    }
+
+    private HttpsJwks mockHttpsJwks(List<JsonWebKey> objects) throws Exception {
+        HttpsJwks httpsJwks = mock(HttpsJwks.class);
+        when(httpsJwks.getJsonWebKeys()).thenReturn(objects);
+        when(httpsJwks.getLocation()).thenReturn("https://www.example.com");
+        return httpsJwks;
+    }
+
+    private JsonWebKey mockJsonWebKey(String keyId) {

Review comment:
       This seems unused.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734831120



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {

Review comment:
       It seems that the configs passed to the following method already has the listener prefix removed.
   
   ` void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries);
   `
   
   The logic does that is in ChannelBuilders.create(), where we have the following code.
   
   `        Map<String, Object> configs = channelBuilderConfigs(config, listenerName);
   `
   
   So, the code in the PR is correct.

##########
File path: build.gradle
##########
@@ -829,6 +829,7 @@ project(':core') {
     implementation libs.jacksonDataformatCsv
     implementation libs.jacksonJDK8Datatypes
     implementation libs.joptSimple
+    implementation libs.jose4j

Review comment:
       This is fine. It will be useful to check all other projects that depend on clients and see if the jose4j dependency needs to be added explicitly. For example, it's convenient to add it to shell and potentially connect:runtime.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}. If <b>not</b> using
+ * OAuth for inter-broker communication, but using it for validation, a dummy JAAS option named
+ * <code>unsecuredLoginStringClaim_sub</code> is needed:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required \

Review comment:
       Hmm, are OAuthBearerLoginModule and unsecuredLoginStringClaim_sub needed on the server side in the secure mode? The latter wasn't mentioned in the KIP.

##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandlerTest.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.apache.kafka.common.utils.Utils;
+import org.junit.jupiter.api.Test;
+
+public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest {
+
+    @Test
+    public void testHandleTokenCallback() throws Exception {
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenBuilder builder = new AccessTokenBuilder();
+        String accessToken = builder.build();
+        AccessTokenRetriever accessTokenRetriever = () -> accessToken;
+
+        OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs);
+
+        try {
+            OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+            handler.handle(new Callback[] {callback});
+
+            assertNotNull(callback.token());
+            OAuthBearerToken token = callback.token();
+            assertEquals(accessToken, 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 testHandleSaslExtensionsCallback() throws Exception {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        Map<String, ?> configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
+        Map<String, Object> jaasConfig = new HashMap<>();
+        jaasConfig.put(CLIENT_ID_CONFIG, "an ID");
+        jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret");
+        jaasConfig.put("extension_foo", "1");
+        jaasConfig.put("extension_bar", 2);
+        jaasConfig.put("EXTENSION_baz", "3");
+        configureHandler(handler, configs, jaasConfig);
+
+        try {
+            SaslExtensionsCallback callback = new SaslExtensionsCallback();
+            handler.handle(new Callback[]{callback});
+
+            assertNotNull(callback.extensions());
+            Map<String, String> extensions = callback.extensions().map();
+            assertEquals("1", extensions.get("foo"));
+            assertEquals("2", extensions.get("bar"));
+            assertNull(extensions.get("baz"));
+            assertEquals(2, extensions.size());
+        } finally {
+            handler.close();
+        }
+    }
+
+    @Test
+    public void testHandleSaslExtensionsCallbackWithInvalidExtension() {
+        String illegalKey = "extension_" + OAuthBearerClientInitialResponse.AUTH_KEY;
+
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        Map<String, ?> configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
+        Map<String, Object> jaasConfig = new HashMap<>();
+        jaasConfig.put(CLIENT_ID_CONFIG, "an ID");
+        jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret");
+        jaasConfig.put(illegalKey, "this key isn't allowed per OAuthBearerClientInitialResponse.validateExtensions");
+        configureHandler(handler, configs, jaasConfig);
+
+        try {
+            SaslExtensionsCallback callback = new SaslExtensionsCallback();
+            assertThrowsWithMessage(ConfigException.class,
+                () -> handler.handle(new Callback[]{callback}),
+                "Extension name " + OAuthBearerClientInitialResponse.AUTH_KEY + " is invalid");
+        } finally {
+            handler.close();
+        }
+    }
+
+    @Test
+    public void testInvalidCallbackGeneratesUnsupportedCallbackException() {
+        Map<String, ?> configs = getSaslConfigs();
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = () -> "foo";
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+        handler.init(accessTokenRetriever, accessTokenValidator);
+
+        try {
+            Callback unsupportedCallback = new Callback() { };
+            assertThrows(UnsupportedCallbackException.class, () -> handler.handle(new Callback[]{unsupportedCallback}));
+        } finally {
+            handler.close();
+        }
+    }
+
+    @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");
+    }
+
+    @Test
+    public void testMissingAccessToken() {
+        AccessTokenRetriever accessTokenRetriever = () -> {
+            throw new IOException("The token endpoint response access_token value must be non-null");
+        };
+        Map<String, ?> configs = getSaslConfigs();
+        OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs);
+
+        try {
+            OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+            assertThrowsWithMessage(IOException.class,
+                () -> handler.handle(new Callback[]{callback}),
+                "token endpoint response access_token value must be non-null");
+        } finally {
+            handler.close();
+        }
+    }
+
+    @Test
+    public void testNotConfigured() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure or init method");
+    }
+
+    @Test
+    public void testConfigureWithAccessTokenFile() throws Exception {
+        String expected = "{}";
+
+        File tmpDir = createTempDir("access-token");
+        File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", expected);
+
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        Map<String, ?> configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString());
+        Map<String, Object> jaasConfigs = Collections.emptyMap();
+        configureHandler(handler, configs, jaasConfigs);

Review comment:
       This test doesn't seem to verify anything. Ditto for the next test.

##########
File path: clients/src/test/resources/log4j.properties
##########
@@ -12,10 +12,10 @@
 # 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.
-log4j.rootLogger=OFF, stdout
+log4j.rootLogger=INFO, stdout

Review comment:
       Are the changes in this file needed?

##########
File path: tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTest.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.tools;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_ID_DOC;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_DOC;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.SCOPE_DOC;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import net.sourceforge.argparse4j.ArgumentParsers;
+import net.sourceforge.argparse4j.inf.ArgumentParser;
+import net.sourceforge.argparse4j.inf.ArgumentParserException;
+import net.sourceforge.argparse4j.inf.Namespace;
+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.secured.AccessTokenRetriever;
+import org.apache.kafka.common.security.oauthbearer.secured.AccessTokenRetrieverFactory;
+import org.apache.kafka.common.security.oauthbearer.secured.AccessTokenValidator;
+import org.apache.kafka.common.security.oauthbearer.secured.AccessTokenValidatorFactory;
+import org.apache.kafka.common.security.oauthbearer.secured.CloseableVerificationKeyResolver;
+import org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.secured.VerificationKeyResolverFactory;
+import org.apache.kafka.common.utils.Exit;
+
+public class OAuthCompatibilityTest {
+
+    public static void main(String[] args) {
+        String description = String.format(
+            "This tool is used to verify OAuth/OIDC provider compatibility.%n%n" +
+            "To use, first export KAFKA_OPTS with Java system properties that match%n" +
+            "your OAuth/OIDC configuration. Next, run the following script to%n" +
+            "execute the test:%n%n" +
+            "    ./bin/kafka-run-class.sh %s" +
+            "%n%n" +
+            "Please refer to the following source files for OAuth/OIDC client and%n" +
+            "broker configuration options:" +
+            "%n%n" +
+            "    %s%n" +
+            "    %s",
+            OAuthCompatibilityTest.class.getName(),
+            SaslConfigs.class.getName(),
+            OAuthBearerLoginCallbackHandler.class.getName());
+
+        ArgumentParser parser = ArgumentParsers
+            .newArgumentParser("oauth-compatibility-test")
+            .defaultHelp(true)
+            .description(description);
+
+        parser.addArgument("--connect-timeout-ms")
+            .type(Long.class)
+            .dest("connectTimeoutMs")
+            .help(SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC);
+        parser.addArgument("--read-timeout-ms")
+            .type(Long.class)
+            .dest("readTimeoutMs")
+            .help(SASL_LOGIN_READ_TIMEOUT_MS_DOC);
+        parser.addArgument("--retry-backoff-ms")
+            .type(Long.class)
+            .dest("retryBackoffMs")
+            .help(SASL_LOGIN_RETRY_BACKOFF_MS_DOC);
+        parser.addArgument("--retry-backoff-max-ms")
+            .type(Long.class)
+            .dest("retryBackoffMax")
+            .help(SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC);
+        parser.addArgument("--scope-claim-name")
+            .dest("scopeClaimName")
+            .help(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC);
+        parser.addArgument("--sub-claim-name")
+            .dest("subClaimName")
+            .help(SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC);
+        parser.addArgument("--token-endpoint-url")
+            .dest("tokenEndpointUrl")
+            .help(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC);
+        parser.addArgument("--jwks-endpoint-url")
+            .dest("jwksEndpointUrl")
+            .help(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC);
+        parser.addArgument("--jwks-endpoint-refresh-ms")
+            .type(Long.class)
+            .dest("jwksEndpointRefreshMs")
+            .help(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC);
+        parser.addArgument("--clock-skew-seconds")
+            .type(Integer.class)
+            .dest("clockSkewSeconds")
+            .help(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC);
+        parser.addArgument("--expected-audience")
+            .dest("expectedAudience")
+            .help(SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC);
+        parser.addArgument("--expected-issuer")
+            .dest("expectedIssuer")
+            .help(SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC);
+
+        parser.addArgument("--client-id")
+            .dest("clientId")
+            .help(CLIENT_ID_DOC);
+        parser.addArgument("--client-secret")
+            .dest("clientSecret")
+            .help(CLIENT_SECRET_DOC);
+        parser.addArgument("--scope")
+            .dest("scope")
+            .help(SCOPE_DOC);
+
+        Namespace namespace;
+
+        try {
+            namespace = parser.parseArgs(args);
+        } catch (ArgumentParserException e) {
+            parser.handleError(e);
+            Exit.exit(1);
+            return;
+        }
+
+        Map<String, ?> configs = getConfigs(namespace);
+        Map<String, Object> jaasConfigs = getJaasConfigs(namespace);
+
+        try {
+            String accessToken;
+
+            {
+                // Client side...
+                try (AccessTokenRetriever atr = AccessTokenRetrieverFactory.create(configs, jaasConfigs)) {
+                    atr.init();
+                    AccessTokenValidator atv = AccessTokenValidatorFactory.create(configs);
+                    System.out.println("PASSED 1/5: client configuration");
+
+                    accessToken = atr.retrieve();
+                    System.out.println("PASSED 2/5: client JWT retrieval");
+
+                    atv.validate(accessToken);
+                    System.out.println("PASSED 3/5: client JWT validation");
+                }
+            }
+
+            {
+                // Broker side...
+                try (CloseableVerificationKeyResolver vkr = VerificationKeyResolverFactory.create(configs, jaasConfigs)) {
+                    vkr.init();
+                    AccessTokenValidator atv = AccessTokenValidatorFactory.create(configs, vkr);
+                    System.out.println("PASSED 4/5: broker configuration");
+
+                    atv.validate(accessToken);
+                    System.out.println("PASSED 5/5: broker JWT validation");
+                }
+            }
+
+            System.out.println("SUCCESS");
+            Exit.exit(0);
+        } catch (Throwable t) {
+            System.out.println("FAILED:");
+            t.printStackTrace();
+
+            if (t instanceof ConfigException) {
+                System.out.printf("%n");
+                parser.printHelp();
+            }
+
+            Exit.exit(1);
+        }
+    }
+
+    private static Map<String, ?> getConfigs(Namespace namespace) {
+        Map<String, Object> c = new HashMap<>();
+        maybeAddLong(namespace, "connectTimeoutMs", c, SASL_LOGIN_CONNECT_TIMEOUT_MS);

Review comment:
       This and the next one seem to be of type int.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> that will be used by the
+     * {@link RefreshingHttpsJwksVerificationKeyResolver} to resolve new key IDs in JWTs.
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = Collections.unmodifiableList(localJWKs);

Review comment:
       `this.jsonWebKeys` could just be `jsonWebKeys`?

##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerTest.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.secured;
+
+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;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+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.util.Base64;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Random;
+import java.util.function.Consumer;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.authenticator.TestJaasConfig;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.utils.Utils;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.TestInstance.Lifecycle;
+import org.junit.jupiter.api.function.Executable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@TestInstance(Lifecycle.PER_CLASS)
+public abstract class OAuthBearerTest {
+
+    protected final Logger log = LoggerFactory.getLogger(getClass());
+
+    protected ObjectMapper mapper = new ObjectMapper();
+
+    protected void assertThrowsWithMessage(Class<? extends Exception> clazz,
+        Executable executable,
+        String substring) {
+        boolean failed = false;
+
+        try {
+            executable.execute();
+        } catch (Throwable t) {
+            failed = true;
+            assertTrue(clazz.isInstance(t), String.format("Test failed by exception %s, but expected %s", t.getClass(), clazz));
+
+            assertErrorMessageContains(t.getMessage(), substring);
+        }
+
+        if (!failed)
+            fail("Expected test to fail with " + clazz + " that contains the string " + substring);
+    }
+
+    protected void assertErrorMessageContains(String actual, String expectedSubstring) {
+        assertTrue(actual.contains(expectedSubstring),
+            String.format("Expected exception message (\"%s\") to contain substring (\"%s\")",
+                actual,
+                expectedSubstring));
+    }
+
+    protected void configureHandler(AuthenticateCallbackHandler handler,
+        Map<String, ?> configs,
+        Map<String, Object> jaasConfig) {
+        TestJaasConfig config = new TestJaasConfig();
+        config.createOrUpdateEntry("KafkaClient", OAuthBearerLoginModule.class.getName(), jaasConfig);
+        AppConfigurationEntry kafkaClient = config.getAppConfigurationEntry("KafkaClient")[0];
+
+        handler.configure(configs,
+            OAuthBearerLoginModule.OAUTHBEARER_MECHANISM,
+            Collections.singletonList(kafkaClient));
+    }
+
+    protected String createBase64JsonJwtSection(Consumer<ObjectNode> c) {
+        String json = createJsonJwtSection(c);
+
+        try {
+            return Utils.utf8(Base64.getEncoder().encode(Utils.utf8(json)));
+        } catch (Throwable t) {
+            fail(t);
+
+            // Shouldn't get to here...
+            return null;
+        }
+    }
+
+    protected String createJsonJwtSection(Consumer<ObjectNode> c) {
+        ObjectNode node = mapper.createObjectNode();
+        c.accept(node);
+
+        try {
+            return mapper.writeValueAsString(node);
+        } catch (Throwable t) {
+            fail(t);
+
+            // Shouldn't get to here...
+            return null;
+        }
+    }
+
+    protected Retryable<String> createRetryable(Exception[] attempts) {
+        Iterator<Exception> i = Arrays.asList(attempts).iterator();
+
+        return () -> {
+            Exception e = i.hasNext() ? i.next() : null;
+
+            if (e == null) {
+                return "success!";
+            } else {
+                if (e instanceof IOException)
+                    throw (IOException) 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 createTempPemDir() throws IOException {

Review comment:
       This seems unused.

##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/LoginAccessTokenValidator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
+import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerIllegalTokenException;
+import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredJws;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * LoginAccessTokenValidator is an implementation of {@link AccessTokenValidator} that is used
+ * by the client to perform some rudimentary validation of the JWT access token that is received
+ * as part of the response from posting the client credentials to the OAuth/OIDC provider's
+ * token endpoint.
+ *
+ * The validation steps performed are:
+ *
+ * <ol>
+ *     <li>
+ *         Basic structural validation of the <code>b64token</code> value as defined in
+ *         <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
+ *     </li>
+ *     <li>Basic conversion of the token into an in-memory map</li>
+ *     <li>Presence of scope, <code>exp</code>, subject, and <code>iat</code> claims</li>
+ * </ol>
+ */
+
+public class LoginAccessTokenValidator implements AccessTokenValidator {
+
+    private static final Logger log = LoggerFactory.getLogger(LoginAccessTokenValidator.class);
+
+    public static final String EXPIRATION_CLAIM_NAME = "exp";
+
+    public static final String ISSUED_AT_CLAIM_NAME = "iat";
+
+    private final String scopeClaimName;
+
+    private final String subClaimName;
+
+    /**
+     * Creates a new LoginAccessTokenValidator that will be used by the client for lightweight
+     * validation of the JWT.
+     */
+
+    public LoginAccessTokenValidator() {

Review comment:
       This seems never used?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734908220



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}. If <b>not</b> using
+ * OAuth for inter-broker communication, but using it for validation, a dummy JAAS option named
+ * <code>unsecuredLoginStringClaim_sub</code> is needed:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required \

Review comment:
       Ugh. Need to address. Thanks for reminding me 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734908281



##########
File path: clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandlerTest.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.apache.kafka.common.utils.Utils;
+import org.junit.jupiter.api.Test;
+
+public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest {
+
+    @Test
+    public void testHandleTokenCallback() throws Exception {
+        Map<String, ?> configs = getSaslConfigs();
+        AccessTokenBuilder builder = new AccessTokenBuilder();
+        String accessToken = builder.build();
+        AccessTokenRetriever accessTokenRetriever = () -> accessToken;
+
+        OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs);
+
+        try {
+            OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+            handler.handle(new Callback[] {callback});
+
+            assertNotNull(callback.token());
+            OAuthBearerToken token = callback.token();
+            assertEquals(accessToken, 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 testHandleSaslExtensionsCallback() throws Exception {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        Map<String, ?> configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
+        Map<String, Object> jaasConfig = new HashMap<>();
+        jaasConfig.put(CLIENT_ID_CONFIG, "an ID");
+        jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret");
+        jaasConfig.put("extension_foo", "1");
+        jaasConfig.put("extension_bar", 2);
+        jaasConfig.put("EXTENSION_baz", "3");
+        configureHandler(handler, configs, jaasConfig);
+
+        try {
+            SaslExtensionsCallback callback = new SaslExtensionsCallback();
+            handler.handle(new Callback[]{callback});
+
+            assertNotNull(callback.extensions());
+            Map<String, String> extensions = callback.extensions().map();
+            assertEquals("1", extensions.get("foo"));
+            assertEquals("2", extensions.get("bar"));
+            assertNull(extensions.get("baz"));
+            assertEquals(2, extensions.size());
+        } finally {
+            handler.close();
+        }
+    }
+
+    @Test
+    public void testHandleSaslExtensionsCallbackWithInvalidExtension() {
+        String illegalKey = "extension_" + OAuthBearerClientInitialResponse.AUTH_KEY;
+
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        Map<String, ?> configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, "http://www.example.com");
+        Map<String, Object> jaasConfig = new HashMap<>();
+        jaasConfig.put(CLIENT_ID_CONFIG, "an ID");
+        jaasConfig.put(CLIENT_SECRET_CONFIG, "a secret");
+        jaasConfig.put(illegalKey, "this key isn't allowed per OAuthBearerClientInitialResponse.validateExtensions");
+        configureHandler(handler, configs, jaasConfig);
+
+        try {
+            SaslExtensionsCallback callback = new SaslExtensionsCallback();
+            assertThrowsWithMessage(ConfigException.class,
+                () -> handler.handle(new Callback[]{callback}),
+                "Extension name " + OAuthBearerClientInitialResponse.AUTH_KEY + " is invalid");
+        } finally {
+            handler.close();
+        }
+    }
+
+    @Test
+    public void testInvalidCallbackGeneratesUnsupportedCallbackException() {
+        Map<String, ?> configs = getSaslConfigs();
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        AccessTokenRetriever accessTokenRetriever = () -> "foo";
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs);
+        handler.init(accessTokenRetriever, accessTokenValidator);
+
+        try {
+            Callback unsupportedCallback = new Callback() { };
+            assertThrows(UnsupportedCallbackException.class, () -> handler.handle(new Callback[]{unsupportedCallback}));
+        } finally {
+            handler.close();
+        }
+    }
+
+    @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");
+    }
+
+    @Test
+    public void testMissingAccessToken() {
+        AccessTokenRetriever accessTokenRetriever = () -> {
+            throw new IOException("The token endpoint response access_token value must be non-null");
+        };
+        Map<String, ?> configs = getSaslConfigs();
+        OAuthBearerLoginCallbackHandler handler = createHandler(accessTokenRetriever, configs);
+
+        try {
+            OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+            assertThrowsWithMessage(IOException.class,
+                () -> handler.handle(new Callback[]{callback}),
+                "token endpoint response access_token value must be non-null");
+        } finally {
+            handler.close();
+        }
+    }
+
+    @Test
+    public void testNotConfigured() {
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        assertThrowsWithMessage(IllegalStateException.class, () -> handler.handle(new Callback[] {}), "first call the configure or init method");
+    }
+
+    @Test
+    public void testConfigureWithAccessTokenFile() throws Exception {
+        String expected = "{}";
+
+        File tmpDir = createTempDir("access-token");
+        File accessTokenFile = createTempFile(tmpDir, "access-token-", ".json", expected);
+
+        OAuthBearerLoginCallbackHandler handler = new OAuthBearerLoginCallbackHandler();
+        Map<String, ?> configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString());
+        Map<String, Object> jaasConfigs = Collections.emptyMap();
+        configureHandler(handler, configs, jaasConfigs);

Review comment:
       Added proper assertions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734905620



##########
File path: tools/src/main/java/org/apache/kafka/tools/OAuthCompatibilityTest.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.tools;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL;
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_ID_DOC;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_DOC;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.SCOPE_CONFIG;
+import static org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler.SCOPE_DOC;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import net.sourceforge.argparse4j.ArgumentParsers;
+import net.sourceforge.argparse4j.inf.ArgumentParser;
+import net.sourceforge.argparse4j.inf.ArgumentParserException;
+import net.sourceforge.argparse4j.inf.Namespace;
+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.secured.AccessTokenRetriever;
+import org.apache.kafka.common.security.oauthbearer.secured.AccessTokenRetrieverFactory;
+import org.apache.kafka.common.security.oauthbearer.secured.AccessTokenValidator;
+import org.apache.kafka.common.security.oauthbearer.secured.AccessTokenValidatorFactory;
+import org.apache.kafka.common.security.oauthbearer.secured.CloseableVerificationKeyResolver;
+import org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.secured.VerificationKeyResolverFactory;
+import org.apache.kafka.common.utils.Exit;
+
+public class OAuthCompatibilityTest {
+
+    public static void main(String[] args) {
+        String description = String.format(
+            "This tool is used to verify OAuth/OIDC provider compatibility.%n%n" +
+            "To use, first export KAFKA_OPTS with Java system properties that match%n" +
+            "your OAuth/OIDC configuration. Next, run the following script to%n" +
+            "execute the test:%n%n" +
+            "    ./bin/kafka-run-class.sh %s" +
+            "%n%n" +
+            "Please refer to the following source files for OAuth/OIDC client and%n" +
+            "broker configuration options:" +
+            "%n%n" +
+            "    %s%n" +
+            "    %s",
+            OAuthCompatibilityTest.class.getName(),
+            SaslConfigs.class.getName(),
+            OAuthBearerLoginCallbackHandler.class.getName());
+
+        ArgumentParser parser = ArgumentParsers
+            .newArgumentParser("oauth-compatibility-test")
+            .defaultHelp(true)
+            .description(description);
+
+        parser.addArgument("--connect-timeout-ms")
+            .type(Long.class)
+            .dest("connectTimeoutMs")
+            .help(SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC);
+        parser.addArgument("--read-timeout-ms")
+            .type(Long.class)
+            .dest("readTimeoutMs")
+            .help(SASL_LOGIN_READ_TIMEOUT_MS_DOC);
+        parser.addArgument("--retry-backoff-ms")
+            .type(Long.class)
+            .dest("retryBackoffMs")
+            .help(SASL_LOGIN_RETRY_BACKOFF_MS_DOC);
+        parser.addArgument("--retry-backoff-max-ms")
+            .type(Long.class)
+            .dest("retryBackoffMax")
+            .help(SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC);
+        parser.addArgument("--scope-claim-name")
+            .dest("scopeClaimName")
+            .help(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC);
+        parser.addArgument("--sub-claim-name")
+            .dest("subClaimName")
+            .help(SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC);
+        parser.addArgument("--token-endpoint-url")
+            .dest("tokenEndpointUrl")
+            .help(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC);
+        parser.addArgument("--jwks-endpoint-url")
+            .dest("jwksEndpointUrl")
+            .help(SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC);
+        parser.addArgument("--jwks-endpoint-refresh-ms")
+            .type(Long.class)
+            .dest("jwksEndpointRefreshMs")
+            .help(SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC);
+        parser.addArgument("--clock-skew-seconds")
+            .type(Integer.class)
+            .dest("clockSkewSeconds")
+            .help(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC);
+        parser.addArgument("--expected-audience")
+            .dest("expectedAudience")
+            .help(SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC);
+        parser.addArgument("--expected-issuer")
+            .dest("expectedIssuer")
+            .help(SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC);
+
+        parser.addArgument("--client-id")
+            .dest("clientId")
+            .help(CLIENT_ID_DOC);
+        parser.addArgument("--client-secret")
+            .dest("clientSecret")
+            .help(CLIENT_SECRET_DOC);
+        parser.addArgument("--scope")
+            .dest("scope")
+            .help(SCOPE_DOC);
+
+        Namespace namespace;
+
+        try {
+            namespace = parser.parseArgs(args);
+        } catch (ArgumentParserException e) {
+            parser.handleError(e);
+            Exit.exit(1);
+            return;
+        }
+
+        Map<String, ?> configs = getConfigs(namespace);
+        Map<String, Object> jaasConfigs = getJaasConfigs(namespace);
+
+        try {
+            String accessToken;
+
+            {
+                // Client side...
+                try (AccessTokenRetriever atr = AccessTokenRetrieverFactory.create(configs, jaasConfigs)) {
+                    atr.init();
+                    AccessTokenValidator atv = AccessTokenValidatorFactory.create(configs);
+                    System.out.println("PASSED 1/5: client configuration");
+
+                    accessToken = atr.retrieve();
+                    System.out.println("PASSED 2/5: client JWT retrieval");
+
+                    atv.validate(accessToken);
+                    System.out.println("PASSED 3/5: client JWT validation");
+                }
+            }
+
+            {
+                // Broker side...
+                try (CloseableVerificationKeyResolver vkr = VerificationKeyResolverFactory.create(configs, jaasConfigs)) {
+                    vkr.init();
+                    AccessTokenValidator atv = AccessTokenValidatorFactory.create(configs, vkr);
+                    System.out.println("PASSED 4/5: broker configuration");
+
+                    atv.validate(accessToken);
+                    System.out.println("PASSED 5/5: broker JWT validation");
+                }
+            }
+
+            System.out.println("SUCCESS");
+            Exit.exit(0);
+        } catch (Throwable t) {
+            System.out.println("FAILED:");
+            t.printStackTrace();
+
+            if (t instanceof ConfigException) {
+                System.out.printf("%n");
+                parser.printHelp();
+            }
+
+            Exit.exit(1);
+        }
+    }
+
+    private static Map<String, ?> getConfigs(Namespace namespace) {
+        Map<String, Object> c = new HashMap<>();
+        maybeAddLong(namespace, "connectTimeoutMs", c, SASL_LOGIN_CONNECT_TIMEOUT_MS);

Review comment:
       Wow! Nice catch. Changed to use `maybeAddInt` instead.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734909451



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}. If <b>not</b> using
+ * OAuth for inter-broker communication, but using it for validation, a dummy JAAS option named
+ * <code>unsecuredLoginStringClaim_sub</code> is needed:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required \

Review comment:
       Outdated documentation removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735980753



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current

Review comment:
       Revised comments so this typo should be gone now.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732092804



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        configure(verificationKeyResolver, accessTokenValidator);
+    }
+
+    public void configure(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);
+        }
+
+        isConfigured = true;
+    }
+
+    @Override
+    public void close() {
+        if (verificationKeyResolver != null) {
+            try {
+                verificationKeyResolver.close();
+            } catch (Exception e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkConfigured();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerValidatorCallback) {
+                handle((OAuthBearerValidatorCallback) callback);
+            } else if (callback instanceof OAuthBearerExtensionsValidatorCallback) {
+                OAuthBearerExtensionsValidatorCallback extensionsCallback = (OAuthBearerExtensionsValidatorCallback) callback;
+                extensionsCallback.inputExtensions().map().forEach((extensionName, v) -> extensionsCallback.valid(extensionName));

Review comment:
       For now, the server side extension validation is a no-op, we just make it as validated? The real function will be our ce-kafka server side authentication?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732088424



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerValidatorCallbackHandler.class);
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        CloseableVerificationKeyResolver verificationKeyResolver = VerificationKeyResolverFactory.create(configs, saslMechanism);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        configure(verificationKeyResolver, accessTokenValidator);
+    }
+
+    public void configure(CloseableVerificationKeyResolver verificationKeyResolver, AccessTokenValidator accessTokenValidator) {

Review comment:
       ditto: should we use init() function name?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r733262176



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerLoginCallbackHandler.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.oauthbearer.secured;
+
+import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.auth.SaslExtensions;
+import org.apache.kafka.common.security.auth.SaslExtensionsCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerClientInitialResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OAuthBearerLoginCallbackHandler implements AuthenticateCallbackHandler {
+
+    private static final Logger log = LoggerFactory.getLogger(OAuthBearerLoginCallbackHandler.class);
+
+    public static final String CLIENT_ID_CONFIG = "clientId";
+    public static final String CLIENT_SECRET_CONFIG = "clientSecret";
+    public static final String SCOPE_CONFIG = "scope";
+
+    public static final String CLIENT_ID_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client ID to uniquely identify the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_SECRET_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String CLIENT_SECRET_DOC = "The OAuth/OIDC identity provider-issued " +
+        "client secret serves a similar function as a password to the " + CLIENT_ID_CONFIG + " " +
+        "account and identifies the service account to use for authentication for " +
+        "this client. The value must be paired with a corresponding " + CLIENT_ID_CONFIG + " " +
+        "value and is provided to the OAuth provider using the OAuth " +
+        "clientcredentials grant type.";
+
+    public static final String SCOPE_DOC = "The (optional) HTTP/HTTPS login request to the " +
+        "token endpoint (" + SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI + ") may need to specify an " +
+        "OAuth \"scope\". If so, the " + SCOPE_CONFIG + " is used to provide the value to " +
+        "include with the login request.";
+
+    private static final String EXTENSION_PREFIX = "extension_";
+
+    private Map<String, Object> moduleOptions;
+
+    private AccessTokenRetriever accessTokenRetriever;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isConfigured = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> 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()));
+
+        moduleOptions = Collections.unmodifiableMap(jaasConfigEntries.get(0).getOptions());
+        AccessTokenRetriever accessTokenRetriever = AccessTokenRetrieverFactory.create(configs, saslMechanism, moduleOptions);
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism);
+        configure(accessTokenRetriever, accessTokenValidator);
+    }
+
+    public void configure(AccessTokenRetriever accessTokenRetriever, AccessTokenValidator accessTokenValidator) {
+        this.accessTokenRetriever = accessTokenRetriever;
+        this.accessTokenValidator = accessTokenValidator;
+
+        try {
+            this.accessTokenRetriever.init();
+        } catch (IOException e) {
+            throw new KafkaException("The OAuth login configuration encountered an error when initializing the AccessTokenRetriever", e);
+        }
+
+        isConfigured = true;
+    }
+
+    @Override
+    public void close() {
+        if (accessTokenRetriever != null) {
+            try {
+                this.accessTokenRetriever.close();
+            } catch (IOException e) {
+                log.warn("The OAuth login configuration encountered an error when closing the AccessTokenRetriever", e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkConfigured();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerTokenCallback) {
+                handle((OAuthBearerTokenCallback) callback);
+            } else if (callback instanceof SaslExtensionsCallback) {
+                handle((SaslExtensionsCallback) callback);
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    void handle(OAuthBearerTokenCallback callback) throws IOException {

Review comment:
       I believe these two methods were package visible because there were some tests that called them directly. Apparently those tests have been refactored. I'll change them to `private`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736970042



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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 of set of configuration
+     * may have multiple instances, we want to reuse the single instance.
+     */
+
+    private static final Map<VkrKey, CloseableVerificationKeyResolver> VKR_CACHE = new HashMap<>();
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        Map<String, Object> moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries);
+        CloseableVerificationKeyResolver verificationKeyResolver;
+
+        // Here's the logic which keeps our VKRs down to a single instance.
+        synchronized (VKR_CACHE) {
+            VkrKey key = new VkrKey(configs, saslMechanism, moduleOptions);

Review comment:
       We do assert that the SASL mechanism is `oauthbearer` in a previous step, so I'll just remove this as part of the key.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736970042



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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 of set of configuration
+     * may have multiple instances, we want to reuse the single instance.
+     */
+
+    private static final Map<VkrKey, CloseableVerificationKeyResolver> VKR_CACHE = new HashMap<>();
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        Map<String, Object> moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries);
+        CloseableVerificationKeyResolver verificationKeyResolver;
+
+        // Here's the logic which keeps our VKRs down to a single instance.
+        synchronized (VKR_CACHE) {
+            VkrKey key = new VkrKey(configs, saslMechanism, moduleOptions);

Review comment:
       We do assert that the SASL mechanism is `oauthbearer` in a previous step, so I'll just remove `saslMechanism` as part of the key.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736970385



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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 of set of configuration

Review comment:
       Changed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r737754724



##########
File path: clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
##########
@@ -75,30 +78,116 @@
     public static final String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = "Login refresh thread will sleep until the specified window factor relative to the"
             + " credential's lifetime has been reached, at which time it will try to refresh the credential."
             + " Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a default value of 0.8 (80%) is used"
-            + " if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR = 0.80;
 
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER = "sasl.login.refresh.window.jitter";
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = "The maximum amount of random jitter relative to the credential's lifetime"
             + " that is added to the login refresh thread's sleep time. Legal values are between 0 and 0.25 (25%) inclusive;"
-            + " a default value of 0.05 (5%) is used if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " a default value of 0.05 (5%) is used if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_JITTER = 0.05;
 
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS = "sasl.login.refresh.min.period.seconds";
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = "The desired minimum time for the login refresh thread to wait before refreshing a credential,"
             + " in seconds. Legal values are between 0 and 900 (15 minutes); a default value of 60 (1 minute) is used if no value is specified.  This value and "
             + " sasl.login.refresh.buffer.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS = 60;
 
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS = "sasl.login.refresh.buffer.seconds";
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = "The amount of buffer time before credential expiration to maintain when refreshing a credential,"
             + " in seconds. If a refresh would otherwise occur closer to expiration than the number of buffer seconds then the refresh will be moved up to maintain"
             + " as much of the buffer time as possible. Legal values are between 0 and 3600 (1 hour); a default value of  300 (5 minutes) is used if no value is specified."
             + " This value and sasl.login.refresh.min.period.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS = 300;
 
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS = "sasl.login.connect.timeout.ms";
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider connection timeout."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS = "sasl.login.read.timeout.ms";
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider read timeout."
+            + OAUTHBEARER_NOTE;
+
+    private static final String EXPONENTIAL_BACKOFF_NOTE = " Login uses an exponential backoff algorithm with an initial wait based on the"
+            + " sasl.login.retry.backoff.ms setting and will double in wait length between attempts up to a maximum wait length specified by the"
+            + " sasl.login.retry.backoff.max.ms setting."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS = "sasl.login.retry.backoff.max.ms";
+    public static final long DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS = 10000;
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC = "The (optional) value in milliseconds for the maximum wait between login attempts to the"
+            + " external authentication provider."
+            + EXPONENTIAL_BACKOFF_NOTE;

Review comment:
       Renamed `EXPONENTIAL_BACKOFF_NOTE` to `LOGIN_EXPONENTIAL_BACKOFF_NOTE` with the existing message.
   
   Added `JWKS_EXPONENTIAL_BACKOFF_NOTE` which reads:
   
   ```
   JWKS retrieval uses an exponential backoff algorithm with an initial wait based on the sasl.oauthbearer.jwks.endpoint.retry.backoff.ms setting and will double in wait length between attempts up to a maximum wait length specified by the sasl.oauthbearer.jwks.endpoint.retry.backoff.max.ms setting.
   ```
   
   Updated `SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS` and `SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS` to reference `JWKS_EXPONENTIAL_BACKOFF_NOTE`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r737755077



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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<VerificationKeyResolverKey, CloseableVerificationKeyResolver> VERIFICATION_KEY_RESOLVER_CACHE = new HashMap<>();
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        Map<String, Object> 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 RefCountingVerificationKeyResolve(VerificationKeyResolverFactory.create(configs, saslMechanism, moduleOptions)));
+        }
+
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        init(verificationKeyResolver, accessTokenValidator);
+    }
+
+    /*
+     * Package-visible for testing.
+     */
+
+    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;
+    }
+
+    @Override
+    public void close() {
+        if (verificationKeyResolver != null) {
+            try {
+                verificationKeyResolver.close();
+            } catch (Exception e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkInitialized();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerValidatorCallback) {
+                handleValidatorCallback((OAuthBearerValidatorCallback) callback);
+            } else if (callback instanceof OAuthBearerExtensionsValidatorCallback) {
+                handleExtensionsValidatorCallback((OAuthBearerExtensionsValidatorCallback) callback);
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    private void handleValidatorCallback(OAuthBearerValidatorCallback callback) {
+        checkInitialized();
+
+        OAuthBearerToken token;
+
+        try {
+            token = accessTokenValidator.validate(callback.tokenValue());
+            log.debug("handle - token: {}", token);
+            callback.token(token);
+        } catch (ValidateException e) {
+            log.warn(e.getMessage(), e);
+            callback.error("invalid_token", null, null);
+        }
+    }
+
+    private void handleExtensionsValidatorCallback(OAuthBearerExtensionsValidatorCallback extensionsValidatorCallback) {
+        checkInitialized();
+
+        extensionsValidatorCallback.inputExtensions().map().forEach((extensionName, v) -> extensionsValidatorCallback.valid(extensionName));
+    }
+
+    private void checkInitialized() {
+        if (!isInitialized)
+            throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName()));
+    }
+
+    /**
+     * <code>VkrKey</code> 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<String, ?> configs;
+
+        private final Map<String, Object> moduleOptions;
+
+        public VerificationKeyResolverKey(Map<String, ?> configs, Map<String, Object> 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);
+        }
+
+    }
+
+    /**
+     * <code>RefCountingVerificationKeyResolve</code> 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 RefCountingVerificationKeyResolve implements CloseableVerificationKeyResolver {

Review comment:
       Will make your suggested change.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r737751231



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler

Review comment:
       Updated the KIP. Thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732257932



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = null;
+
+        try (InputStream is = con.getInputStream()) {
+            ByteArrayOutputStream os = new ByteArrayOutputStream();
+            log.debug("handleOutput - preparing to read request body to {}", con.getURL());

Review comment:
       Yes 😱  Good catch!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#issuecomment-947122444


   Thanks for the feedback, @YiDing-Duke!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734886554



##########
File path: build.gradle
##########
@@ -1200,6 +1201,7 @@ project(':clients') {
 
     compileOnly libs.jacksonDatabind // for SASL/OAUTHBEARER bearer token parsing
     compileOnly libs.jacksonJDK8Datatypes
+    compileOnly libs.jose4j

Review comment:
       Do we want to add a comment like:
   
   ```text
   // for SASL/OAUTHBEARER bearer token parsing
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735894744



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInited;
+
+    /**
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = localJWKs;
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            executorService.scheduleAtFixedRate(this::refreshInternal,
+                0,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInited = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    @Override
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInited)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refreshInternal() {
+        try {
+            log.info("JWKS validation key refresh of {} starting", getLocation());
+
+            // Call the *actual* refresh implementation.
+            refresh();
+
+            List<JsonWebKey> jwks = getJsonWebKeys();
+
+            try {
+                refreshLock.writeLock().lock();
+
+                for (JsonWebKey jwk : jwks)
+                    missingKeyIds.remove(jwk.getKeyId());
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            log.info("JWKS validation key refresh of {} complete", getLocation());
+        } catch (JoseException | IOException e) {
+            // Let's wait a random, but short amount of time before trying again.
+            long waitMs = ThreadLocalRandom.current().nextLong(1000, 10000);

Review comment:
       Implementation and tests for the backoff/retry now included.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r735902362



##########
File path: build.gradle
##########
@@ -1208,6 +1210,8 @@ project(':clients') {
     testRuntimeOnly libs.slf4jlog4j
     testRuntimeOnly libs.jacksonDatabind
     testRuntimeOnly libs.jacksonJDK8Datatypes
+    testRuntimeOnly libs.jose4j
+    testImplementation libs.jose4j

Review comment:
       It appears it does. Thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736969618



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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 of set of configuration
+     * may have multiple instances, we want to reuse the single instance.
+     */
+
+    private static final Map<VkrKey, CloseableVerificationKeyResolver> VKR_CACHE = new HashMap<>();
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        Map<String, Object> moduleOptions = JaasOptionsUtils.getOptions(saslMechanism, jaasConfigEntries);
+        CloseableVerificationKeyResolver verificationKeyResolver;
+
+        // Here's the logic which keeps our VKRs down to a single instance.
+        synchronized (VKR_CACHE) {
+            VkrKey key = new VkrKey(configs, saslMechanism, moduleOptions);
+            verificationKeyResolver = VKR_CACHE.computeIfAbsent(key, k -> {
+                CloseableVerificationKeyResolver vkr = VerificationKeyResolverFactory.create(configs, saslMechanism, moduleOptions);
+                return new RefCountingVkr(vkr);
+            });
+        }
+
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        init(verificationKeyResolver, accessTokenValidator);
+    }
+
+    /*
+     * Package-visible for testing.
+     */
+
+    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;
+    }
+
+    @Override
+    public void close() {
+        if (verificationKeyResolver != null) {
+            try {
+                verificationKeyResolver.close();
+            } catch (Exception e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkInitialized();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerValidatorCallback) {
+                handleValidatorCallback((OAuthBearerValidatorCallback) callback);
+            } else if (callback instanceof OAuthBearerExtensionsValidatorCallback) {
+                handleExtensionsValidatorCallback((OAuthBearerExtensionsValidatorCallback) callback);
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    private void handleValidatorCallback(OAuthBearerValidatorCallback callback) {
+        checkInitialized();
+
+        OAuthBearerToken token;
+
+        try {
+            token = accessTokenValidator.validate(callback.tokenValue());
+            log.debug("handle - token: {}", token);
+            callback.token(token);
+        } catch (ValidateException e) {
+            log.warn(e.getMessage(), e);
+            callback.error("invalid_token", null, null);
+        }
+    }
+
+    private void handleExtensionsValidatorCallback(OAuthBearerExtensionsValidatorCallback extensionsValidatorCallback) {
+        checkInitialized();
+
+        extensionsValidatorCallback.inputExtensions().map().forEach((extensionName, v) -> extensionsValidatorCallback.valid(extensionName));
+    }
+
+    private void checkInitialized() {
+        if (!isInitialized)
+            throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName()));
+    }
+
+    /**
+     * <code>VkrKey</code> 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 VkrKey {

Review comment:
       Stands for `VerificationKeyResolver`, but I will spell it out as it's not obvious to everyone.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r737625437



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/OAuthBearerValidatorCallbackHandler.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.secured;
+
+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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerToken;
+import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwx.JsonWebStructure;
+import org.jose4j.lang.UnresolvableKeyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * <code>OAuthBearerValidatorCallbackHandler</code> is an {@link AuthenticateCallbackHandler} that
+ * accepts {@link OAuthBearerValidatorCallback} and {@link OAuthBearerExtensionsValidatorCallback}
+ * callbacks to implement OAuth/OIDC validation. This callback handler is intended only to be used
+ * on the Kafka broker side as it will receive a {@link OAuthBearerValidatorCallback} that includes
+ * the JWT provided by the Kafka client. That JWT is validated in terms of format, expiration,
+ * signature, and audience and issuer (if desired). This callback handler is the broker side of the
+ * OAuth functionality, whereas {@link OAuthBearerLoginCallbackHandler} is used by clients.
+ * </p>
+ *
+ * <p>
+ * 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}
+ * like so:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
+ * </code>
+ * </p>
+ *
+ * <p>
+ * The JAAS configuration for OAuth is also needed. If using OAuth for inter-broker communication,
+ * the options are those specified in {@link OAuthBearerLoginCallbackHandler}.
+ * </p>
+ *
+ * <p>
+ * The configuration option
+ * {@link org.apache.kafka.common.config.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:
+ *
+ * <code>
+ * listener.name.<listener name>.oauthbearer.sasl.oauthbearer.jwks.endpoint.url=https://example.com/oauth2/v1/keys
+ * </code>
+ *
+ * Please see the OAuth/OIDC providers documentation for the JWKS endpoint URL.
+ * </p>
+ *
+ * <p>
+ * The following is a list of all the configuration options that are available for the broker
+ * validation callback handler:
+ *
+ * <ul>
+ *   <li>{@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_ISSUER}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_JWKS_ENDPOINT_URL}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SCOPE_CLAIM_NAME}</li>
+ *   <li>{@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_SUB_CLAIM_NAME}</li>
+ * </ul>
+ * </p>
+ */
+
+public class OAuthBearerValidatorCallbackHandler implements AuthenticateCallbackHandler {
+
+    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<VerificationKeyResolverKey, CloseableVerificationKeyResolver> VERIFICATION_KEY_RESOLVER_CACHE = new HashMap<>();
+
+    private CloseableVerificationKeyResolver verificationKeyResolver;
+
+    private AccessTokenValidator accessTokenValidator;
+
+    private boolean isInitialized = false;
+
+    @Override
+    public void configure(Map<String, ?> configs, String saslMechanism, List<AppConfigurationEntry> jaasConfigEntries) {
+        Map<String, Object> 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 RefCountingVerificationKeyResolve(VerificationKeyResolverFactory.create(configs, saslMechanism, moduleOptions)));
+        }
+
+        AccessTokenValidator accessTokenValidator = AccessTokenValidatorFactory.create(configs, saslMechanism, verificationKeyResolver);
+        init(verificationKeyResolver, accessTokenValidator);
+    }
+
+    /*
+     * Package-visible for testing.
+     */
+
+    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;
+    }
+
+    @Override
+    public void close() {
+        if (verificationKeyResolver != null) {
+            try {
+                verificationKeyResolver.close();
+            } catch (Exception e) {
+                log.error(e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        checkInitialized();
+
+        for (Callback callback : callbacks) {
+            if (callback instanceof OAuthBearerValidatorCallback) {
+                handleValidatorCallback((OAuthBearerValidatorCallback) callback);
+            } else if (callback instanceof OAuthBearerExtensionsValidatorCallback) {
+                handleExtensionsValidatorCallback((OAuthBearerExtensionsValidatorCallback) callback);
+            } else {
+                throw new UnsupportedCallbackException(callback);
+            }
+        }
+    }
+
+    private void handleValidatorCallback(OAuthBearerValidatorCallback callback) {
+        checkInitialized();
+
+        OAuthBearerToken token;
+
+        try {
+            token = accessTokenValidator.validate(callback.tokenValue());
+            log.debug("handle - token: {}", token);
+            callback.token(token);
+        } catch (ValidateException e) {
+            log.warn(e.getMessage(), e);
+            callback.error("invalid_token", null, null);
+        }
+    }
+
+    private void handleExtensionsValidatorCallback(OAuthBearerExtensionsValidatorCallback extensionsValidatorCallback) {
+        checkInitialized();
+
+        extensionsValidatorCallback.inputExtensions().map().forEach((extensionName, v) -> extensionsValidatorCallback.valid(extensionName));
+    }
+
+    private void checkInitialized() {
+        if (!isInitialized)
+            throw new IllegalStateException(String.format("To use %s, first call the configure or init method", getClass().getSimpleName()));
+    }
+
+    /**
+     * <code>VkrKey</code> 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<String, ?> configs;
+
+        private final Map<String, Object> moduleOptions;
+
+        public VerificationKeyResolverKey(Map<String, ?> configs, Map<String, Object> 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);
+        }
+
+    }
+
+    /**
+     * <code>RefCountingVerificationKeyResolve</code> 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 RefCountingVerificationKeyResolve implements CloseableVerificationKeyResolver {

Review comment:
       RefCountingVerificationKeyResolve => RefCountingVerificationKeyResolver?

##########
File path: clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
##########
@@ -75,30 +78,116 @@
     public static final String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = "Login refresh thread will sleep until the specified window factor relative to the"
             + " credential's lifetime has been reached, at which time it will try to refresh the credential."
             + " Legal values are between 0.5 (50%) and 1.0 (100%) inclusive; a default value of 0.8 (80%) is used"
-            + " if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR = 0.80;
 
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER = "sasl.login.refresh.window.jitter";
     public static final String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = "The maximum amount of random jitter relative to the credential's lifetime"
             + " that is added to the login refresh thread's sleep time. Legal values are between 0 and 0.25 (25%) inclusive;"
-            + " a default value of 0.05 (5%) is used if no value is specified. Currently applies only to OAUTHBEARER.";
+            + " a default value of 0.05 (5%) is used if no value is specified."
+            + OAUTHBEARER_NOTE;
     public static final double DEFAULT_LOGIN_REFRESH_WINDOW_JITTER = 0.05;
 
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS = "sasl.login.refresh.min.period.seconds";
     public static final String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = "The desired minimum time for the login refresh thread to wait before refreshing a credential,"
             + " in seconds. Legal values are between 0 and 900 (15 minutes); a default value of 60 (1 minute) is used if no value is specified.  This value and "
             + " sasl.login.refresh.buffer.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS = 60;
 
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS = "sasl.login.refresh.buffer.seconds";
     public static final String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = "The amount of buffer time before credential expiration to maintain when refreshing a credential,"
             + " in seconds. If a refresh would otherwise occur closer to expiration than the number of buffer seconds then the refresh will be moved up to maintain"
             + " as much of the buffer time as possible. Legal values are between 0 and 3600 (1 hour); a default value of  300 (5 minutes) is used if no value is specified."
             + " This value and sasl.login.refresh.min.period.seconds are both ignored if their sum exceeds the remaining lifetime of a credential."
-            + " Currently applies only to OAUTHBEARER.";
+            + OAUTHBEARER_NOTE;
     public static final short DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS = 300;
 
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS = "sasl.login.connect.timeout.ms";
+    public static final String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider connection timeout."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS = "sasl.login.read.timeout.ms";
+    public static final String SASL_LOGIN_READ_TIMEOUT_MS_DOC = "The (optional) value in milliseconds for the external authentication provider read timeout."
+            + OAUTHBEARER_NOTE;
+
+    private static final String EXPONENTIAL_BACKOFF_NOTE = " Login uses an exponential backoff algorithm with an initial wait based on the"
+            + " sasl.login.retry.backoff.ms setting and will double in wait length between attempts up to a maximum wait length specified by the"
+            + " sasl.login.retry.backoff.max.ms setting."
+            + OAUTHBEARER_NOTE;
+
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS = "sasl.login.retry.backoff.max.ms";
+    public static final long DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS = 10000;
+    public static final String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC = "The (optional) value in milliseconds for the maximum wait between login attempts to the"
+            + " external authentication provider."
+            + EXPONENTIAL_BACKOFF_NOTE;

Review comment:
       EXPONENTIAL_BACKOFF_NOTE refers to sasl.login.retry.backoff.max.ms, which will be confusing. Ditto for  SASL_LOGIN_RETRY_BACKOFF_MS_DOC.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r737758788



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {
+
+    private static final long RETRY_BACKOFF_MS = 2000;
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final HttpsJwks httpsJwks;
+
+    private final ScheduledExecutorService executorService;
+
+    private ScheduledFuture<?> refreshFuture;
+
+    private final Time time;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInitialized;
+
+    /**
+     * Creates a <code>RefreshingHttpsJwks</code> 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
+     */
+
+    public RefreshingHttpsJwks(Time time, HttpsJwks httpsJwks, long refreshMs) {
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        this.httpsJwks = httpsJwks;
+        this.time = time;
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                // This will trigger a call th HttpsJwks.refresh() and that will block the current
+                // thread. It's OK to do this in init() but we avoid blocking elsewhere as we're
+                // run in a network thread.
+                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.
+            refreshFuture = executorService.scheduleAtFixedRate(this::refresh,
+                refreshMs,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInitialized = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    /**
+     * Overrides the base implementation because the base implementation has a case that performs
+     * a blocking call to refresh(), which we want to avoid in the authentication validation path.
+     *
+     * The list may be stale up to refreshMs.
+     *
+     * @return {@link List} of {@link JsonWebKey} instances
+     *
+     * @throws JoseException Thrown if a problem is encountered parsing the JSON content into JWKs
+     * @throws IOException Thrown f a problem is encountered making the HTTP request
+     */
+
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInitialized)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    public String getLocation() {
+        return httpsJwks.getLocation();
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refresh() {
+        // How much time (in milliseconds) do we have before the next refresh is scheduled to
+        // occur? This value will [0..refreshMs]. Every time a scheduled refresh occurs, the
+        // value of refreshFuture is reset to refreshMs and works down to 0.
+        long timeBeforeNextRefresh = refreshFuture.getDelay(TimeUnit.MILLISECONDS);
+        log.debug("timeBeforeNextRefresh: {}, RETRY_BACKOFF_MS: {}", timeBeforeNextRefresh, RETRY_BACKOFF_MS);
+
+        // If the time left before the next scheduled refresh is less than the amount of time we
+        // have set aside for retries, log the fact and return. Don't worry, this refresh method
+        // will still be called again within RETRY_BACKOFF_MS :)
+        //
+        // Note: timeBeforeNextRefresh is negative when we're in the midst of executing refresh
+        // in a scheduled fashion. ScheduledFuture.getDelay will reset *after* the method has

Review comment:
       Changed the logic to be simpler: if we're in the middle of a `refresh` and another thread comes in, we simply return.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r736016076



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.kafka.common.utils.Time;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks implements Initable, Closeable {

Review comment:
       Yes, that seems reasonable.
   
   The trick is that the `AuthenticateCallbackHandler` doesn't get the listener name, so it's hard to know when to share and when to create a new instance.
   
   I'll look into the implementation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734086876



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/ConfigurationUtils.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.secured;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Path;
+import java.util.Locale;
+import java.util.Map;
+import javax.net.ssl.SSLSocketFactory;
+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.ListenerName;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory;
+import org.apache.kafka.common.security.ssl.SslFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>ConfigurationUtils</code> is a utility class to perform basic configuration-related
+ * logic and is separated out here for easier, more direct testing.
+ */
+
+public class ConfigurationUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ConfigurationUtils.class);
+
+    private final Map<String, ?> configs;
+
+    private final String prefix;
+
+    public ConfigurationUtils(Map<String, ?> configs) {
+        this(configs, null);
+    }
+
+    public ConfigurationUtils(Map<String, ?> configs, String saslMechanism) {
+        this.configs = configs;
+
+        if (saslMechanism != null && !saslMechanism.trim().isEmpty())
+            this.prefix = ListenerName.saslMechanismPrefix(saslMechanism.trim());
+        else
+            this.prefix = null;
+    }
+
+    public Map<String, ?> getSslClientConfig(String uriConfigName) {
+        String urlConfigValue = get(uriConfigName);
+
+        if (urlConfigValue == null || urlConfigValue.trim().isEmpty())
+            throw new ConfigException(String.format("The OAuth configuration option %s is required", uriConfigName));
+
+        URL url;
+
+        try {
+            url = new URL(urlConfigValue);
+        } catch (IOException e) {
+            throw new ConfigException(String.format("The OAuth configuration option %s was not a valid URL (%s)", uriConfigName, urlConfigValue));
+        }
+
+        if (!url.getProtocol().equalsIgnoreCase("https")) {
+            log.warn("Not creating SSL socket factory as URL for {} ({}) is not SSL-/TLS-based", uriConfigName, url);
+            return null;
+        }
+
+        ConfigDef sslConfigDef = new ConfigDef();
+        sslConfigDef.withClientSslSupport();

Review comment:
       That said, you're right, the code was accidentally referring to the top-level configuration `Map`, not the JAAS options `Map`. That has been fixed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734905324



##########
File path: clients/src/test/resources/log4j.properties
##########
@@ -12,10 +12,10 @@
 # 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.
-log4j.rootLogger=OFF, stdout
+log4j.rootLogger=INFO, stdout

Review comment:
       No. This was me being careless 😱 
   
   Sorry.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732251460



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = null;
+
+        try (InputStream is = con.getInputStream()) {
+            ByteArrayOutputStream os = new ByteArrayOutputStream();
+            log.debug("handleOutput - preparing to read request body to {}", con.getURL());
+            copy(is, os);
+            responseBody = os.toString(StandardCharsets.UTF_8.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());

Review comment:
       The request body (parameters) are _URL_-encoded, but the `Authorization` header value after `Basic` needs to be _base 64_-encoded.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r732251460



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.secured;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+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 javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <code>HttpAccessTokenRetriever</code> 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_URI}).
+ *
+ * @see AccessTokenRetriever
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_ID_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#CLIENT_SECRET_CONFIG
+ * @see OAuthBearerLoginCallbackHandler#SCOPE_CONFIG
+ * @see SaslConfigs#SASL_OAUTHBEARER_TOKEN_ENDPOINT_URI
+ */
+
+public class HttpAccessTokenRetriever implements AccessTokenRetriever {
+
+    private static final Logger log = LoggerFactory.getLogger(HttpAccessTokenRetriever.class);
+
+    private static final Set<Integer> UNRETRYABLE_HTTP_CODES;
+
+    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 tokenEndpointUri;
+
+    private final long loginRetryBackoffMs;
+
+    private final long loginRetryBackoffMaxMs;
+
+    private final Integer loginConnectTimeoutMs;
+
+    private final Integer loginReadTimeoutMs;
+
+    public HttpAccessTokenRetriever(String clientId,
+        String clientSecret,
+        String scope,
+        SSLSocketFactory sslSocketFactory,
+        String tokenEndpointUri,
+        long loginRetryBackoffMs,
+        long loginRetryBackoffMaxMs,
+        Integer loginConnectTimeoutMs,
+        Integer loginReadTimeoutMs) {
+        this.clientId = Objects.requireNonNull(clientId);
+        this.clientSecret = Objects.requireNonNull(clientSecret);
+        this.scope = scope;
+        this.sslSocketFactory = sslSocketFactory;
+        this.tokenEndpointUri = Objects.requireNonNull(tokenEndpointUri);
+        this.loginRetryBackoffMs = loginRetryBackoffMs;
+        this.loginRetryBackoffMaxMs = loginRetryBackoffMaxMs;
+        this.loginConnectTimeoutMs = loginConnectTimeoutMs;
+        this.loginReadTimeoutMs = loginReadTimeoutMs;
+    }
+
+    /**
+     * 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.
+     *
+     * <b>Note</b>: 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-<code>null</code> 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);
+        String requestBody = formatRequestBody(scope);
+
+        Retry<String> retry = new Retry<>(Time.SYSTEM,
+            loginRetryBackoffMs,
+            loginRetryBackoffMaxMs);
+
+        Map<String, String> headers = Collections.singletonMap(AUTHORIZATION_HEADER, authorizationHeader);
+
+        String responseBody = retry.execute(() -> {
+            HttpURLConnection con = (HttpURLConnection) new URL(tokenEndpointUri).openConnection();
+
+            if (sslSocketFactory != null && con instanceof HttpsURLConnection)
+                ((HttpsURLConnection) con).setSSLSocketFactory(sslSocketFactory);
+
+            try {
+                return post(con, headers, requestBody, loginConnectTimeoutMs, loginReadTimeoutMs);
+            } finally {
+                con.disconnect();
+            }
+        });
+        log.debug("retrieve - responseBody: {}", responseBody);
+
+        return parseAccessToken(responseBody);
+    }
+
+    public static String post(HttpURLConnection con,
+        Map<String, String> 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<String, String> 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<String, String> 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);
+
+        String responseBody = null;
+
+        try (InputStream is = con.getInputStream()) {
+            ByteArrayOutputStream os = new ByteArrayOutputStream();
+            log.debug("handleOutput - preparing to read request body to {}", con.getURL());
+            copy(is, os);
+            responseBody = os.toString(StandardCharsets.UTF_8.name());
+        } catch (Exception e) {
+            log.warn("handleOutput - error retrieving data", e);
+        }
+
+        if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) {
+            if (responseBody == null || responseBody.isEmpty())
+                throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL()));
+
+            log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody);
+
+            return responseBody;
+        } else {
+            log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody);
+
+            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 was encountered reading the token endpoint response; will not attempt further retries", responseCode)));
+            } 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 was encountered reading the token endpoint response", responseCode));
+            }
+        }
+    }
+
+    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 parseAccessToken(String responseBody) throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        JsonNode rootNode = mapper.readTree(responseBody);
+        JsonNode accessTokenNode = rootNode.at("/access_token");
+
+        if (accessTokenNode == null)
+            throw new IOException("The token endpoint response did not contain an access_token value");
+
+        return sanitizeString("The token endpoint response access_token", accessTokenNode.textValue());
+    }
+
+    static String formatAuthorizationHeader(String clientId, String clientSecret) throws IOException {
+        clientId = sanitizeString("The token endpoint request clientId", clientId);
+        clientSecret = sanitizeString("The token endpoint request clientId", clientSecret);
+
+        String s = String.format("%s:%s", clientId, clientSecret);
+        String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s));
+        return String.format("Basic %s", encoded);
+    }
+
+    static String formatRequestBody(String scope) throws IOException {
+        try {
+            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.name());

Review comment:
       The request body (parameters) are _URL_-encoded, but the `Authorization` header value after `Basic` needs to be _base 64_-encoded. That's per HTTP, I believe, not OAuth.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kirktrue commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r734121443



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/RefreshingHttpsJwks.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.secured;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.lang.JoseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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
+ * possible to receive a JWT that contains a <code>kid</code> that points to yet-unknown JWK,
+ * thus requiring a connection to the OAuth/OIDC provider to be made. Hopefully, in practice,
+ * keys are made available for some amount of time before they're used within JWTs.
+ *
+ * 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
+ * a JWT.
+ *
+ * @see org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver
+ * @see org.jose4j.keys.resolvers.VerificationKeyResolver
+ * @see ValidatorAccessTokenValidator
+ */
+
+public final class RefreshingHttpsJwks extends HttpsJwks implements Initable, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RefreshingHttpsJwks.class);
+
+    private static final int MISSING_KEY_ID_CACHE_MAX_ENTRIES = 16;
+
+    private static final long MISSING_KEY_ID_CACHE_IN_FLIGHT_MS = 60000;
+
+    private static final int MISSING_KEY_ID_MAX_KEY_LENGTH = 1000;
+
+    private static final int SHUTDOWN_TIMEOUT = 10;
+
+    private static final TimeUnit SHUTDOWN_TIME_UNIT = TimeUnit.SECONDS;
+
+    private final ScheduledExecutorService executorService;
+
+    private final long refreshMs;
+
+    private final ReadWriteLock refreshLock = new ReentrantReadWriteLock();
+
+    private final Map<String, Long> missingKeyIds;
+
+    private List<JsonWebKey> jsonWebKeys;
+
+    private boolean isInited;
+
+    /**
+     *
+     * @param location  HTTP/HTTPS endpoint 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
+     */
+
+    public RefreshingHttpsJwks(String location, long refreshMs) {
+        super(location);
+
+        if (refreshMs <= 0)
+            throw new IllegalArgumentException("JWKS validation key refresh configuration value retryWaitMs value must be positive");
+
+        setDefaultCacheDuration(refreshMs);
+
+        this.refreshMs = refreshMs;
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+        this.missingKeyIds = new LinkedHashMap<String, Long>(MISSING_KEY_ID_CACHE_MAX_ENTRIES, .75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<String, Long> eldest) {
+                return this.size() > MISSING_KEY_ID_CACHE_MAX_ENTRIES;
+            }
+        };
+    }
+
+    @Override
+    public void init() throws IOException {
+        try {
+            log.debug("init started");
+
+            List<JsonWebKey> localJWKs;
+
+            try {
+                localJWKs = super.getJsonWebKeys();
+            } catch (JoseException e) {
+                throw new IOException("Could not refresh JWKS", e);
+            }
+
+            try {
+                refreshLock.writeLock().lock();
+                this.jsonWebKeys = localJWKs;
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            executorService.scheduleAtFixedRate(this::refreshInternal,
+                0,
+                refreshMs,
+                TimeUnit.MILLISECONDS);
+
+            log.info("JWKS validation key refresh thread started with a refresh interval of {} ms", refreshMs);
+        } finally {
+            isInited = true;
+
+            log.debug("init completed");
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            log.debug("close started");
+
+            try {
+                log.debug("JWKS validation key refresh thread shutting down");
+                executorService.shutdown();
+
+                if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT)) {
+                    log.warn("JWKS validation key refresh thread termination did not end after {} {}",
+                        SHUTDOWN_TIMEOUT, SHUTDOWN_TIME_UNIT);
+                }
+            } catch (InterruptedException e) {
+                log.warn("JWKS validation key refresh thread error during close", e);
+            }
+        } finally {
+            log.debug("close completed");
+        }
+    }
+
+    @Override
+    public List<JsonWebKey> getJsonWebKeys() throws JoseException, IOException {
+        if (!isInited)
+            throw new IllegalStateException("Please call init() first");
+
+        try {
+            refreshLock.readLock().lock();
+            return jsonWebKeys;
+        } finally {
+            refreshLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Internal method that will refresh the cache and if errors are encountered, re-queues
+     * the refresh attempt in a background thread.
+     */
+
+    private void refreshInternal() {
+        try {
+            log.info("JWKS validation key refresh of {} starting", getLocation());
+
+            // Call the *actual* refresh implementation.
+            refresh();
+
+            List<JsonWebKey> jwks = getJsonWebKeys();
+
+            try {
+                refreshLock.writeLock().lock();
+
+                for (JsonWebKey jwk : jwks)
+                    missingKeyIds.remove(jwk.getKeyId());
+            } finally {
+                refreshLock.writeLock().unlock();
+            }
+
+            log.info("JWKS validation key refresh of {} complete", getLocation());
+        } catch (JoseException | IOException e) {
+            // Let's wait a random, but short amount of time before trying again.
+            long waitMs = ThreadLocalRandom.current().nextLong(1000, 10000);

Review comment:
       Yes, that seems prudent. I have to think about this a bit more to make sure that the on-demand thread and the normally-scheduled background thread don't step on each other's toes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao merged pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
junrao merged pull request #11284:
URL: https://github.com/apache/kafka/pull/11284


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] YiDing-Duke commented on a change in pull request #11284: KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

Posted by GitBox <gi...@apache.org>.
YiDing-Duke commented on a change in pull request #11284:
URL: https://github.com/apache/kafka/pull/11284#discussion_r731524967



##########
File path: clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/Retry.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.secured;
+
+import java.io.IOException;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Retry encapsulates the mechanism to perform a retry and then exponential
+ * backoff using provided wait times between attempts.
+ *
+ * @param <R> Result type
+ */
+
+public class Retry<R> {
+
+    private static final Logger log = LoggerFactory.getLogger(Retry.class);
+
+    private final Time time;
+
+    private final long retryBackoffMs;
+
+    private final long 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 %s must be non-negative", retryBackoffMs));
+
+        if (this.retryBackoffMaxMs < 0)
+            throw new IllegalArgumentException(String.format("retryBackoffMaxMs %s value must be non-negative", retryBackoffMaxMs));
+
+        if (this.retryBackoffMaxMs < this.retryBackoffMs)
+            throw new IllegalArgumentException(String.format("retryBackoffMaxMs %s is less than retryBackoffMs %s", retryBackoffMaxMs, retryBackoffMs));
+    }
+
+    public R execute(Retryable<R> retryable) throws IOException {
+        int currAttempt = 0;
+        long end = time.milliseconds() + retryBackoffMaxMs;
+        IOException error = null;
+
+        while (time.milliseconds() <= end) {
+            currAttempt++;
+
+            try {
+                return retryable.call();
+            } catch (IOException e) {
+                if (error == null)
+                    error = e;
+
+                long waitMs = retryBackoffMs * (long) Math.pow(2, currAttempt - 1);
+                long diff = end - time.milliseconds();
+                waitMs = Math.min(waitMs, diff);
+
+                if (waitMs <= 0)

Review comment:
       When it comes to retry timeout, should we log this error so that we can figure out a non retry-able error pattern to add to the list in the future?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org