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-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org