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-unsubscr...@kafka.apache.org

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


Reply via email to