From 91871b67c8b3cbf9877ec57f57949437a25a15c1 Mon Sep 17 00:00:00 2001 From: ggivo Date: Mon, 2 Dec 2024 18:45:01 +0200 Subject: [PATCH 01/28] Support for StreamingCredentials This enables use cases like credential rotation and token based auth without client disconnect. Especially with Pub/Sub clients will reduce the chnance of missing events. --- .../core/BaseRedisAuthenticationHandler.java | 117 +++++++++++++++++ .../core/RedisAuthenticationHandler.java | 44 +++++++ .../core/StatefulRedisConnectionImpl.java | 16 +++ .../core/StreamingCredentialsProvider.java | 15 +++ .../RedisClusterAuthenticationHandler.java | 45 +++++++ .../StatefulRedisClusterConnectionImpl.java | 10 ++ .../core/AuthenticationIntegrationTests.java | 124 ++++++++++++++++++ .../BaseRedisAuthenticationHandlerTest.java | 111 ++++++++++++++++ .../core/RedisAuthenticationHandlerTest.java | 52 ++++++++ 9 files changed, 534 insertions(+) create mode 100644 src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java create mode 100644 src/main/java/io/lettuce/core/RedisAuthenticationHandler.java create mode 100644 src/main/java/io/lettuce/core/StreamingCredentialsProvider.java create mode 100644 src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java create mode 100644 src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java create mode 100644 src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java diff --git a/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java new file mode 100644 index 000000000..fa9f9eb84 --- /dev/null +++ b/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java @@ -0,0 +1,117 @@ +package io.lettuce.core; + +import io.lettuce.core.codec.StringCodec; +import io.lettuce.core.protocol.AsyncCommand; +import io.lettuce.core.protocol.RedisCommand; +import io.netty.util.internal.logging.InternalLogger; +import io.netty.util.internal.logging.InternalLoggerFactory; +import reactor.core.Disposable; +import reactor.core.publisher.Flux; + +import java.nio.CharBuffer; +import java.util.concurrent.atomic.AtomicReference; + +public abstract class BaseRedisAuthenticationHandler> { + + private static final InternalLogger log = InternalLoggerFactory.getInstance(BaseRedisAuthenticationHandler.class); + + protected final T connection; + + private final RedisCommandBuilder commandBuilder = new RedisCommandBuilder<>(StringCodec.UTF8); + + private final AtomicReference credentialsSubscription = new AtomicReference<>(); + + public BaseRedisAuthenticationHandler(T connection) { + this.connection = connection; + } + + /** + * Subscribes to the provided `Flux` of credentials if the given `RedisCredentialsProvider` supports streaming credentials. + *

+ * This method subscribes to a stream of credentials provided by the `StreamingCredentialsProvider`. Each time new + * credentials are received, the client is reauthenticated. If the connection is not supported, the method returns without + * subscribing. + *

+ * The previous subscription, if any, is disposed of before setting the new subscription. + * + * @param credentialsProvider the credentials provider to subscribe to + */ + public void subscribe(RedisCredentialsProvider credentialsProvider) { + if (credentialsProvider == null) { + return; + } + + if (credentialsProvider instanceof StreamingCredentialsProvider) { + if (!isSupportedConnection()) { + return; + } + + Flux credentialsFlux = ((StreamingCredentialsProvider) credentialsProvider).credentials(); + + Disposable subscription = credentialsFlux.subscribe(this::onNext, this::onError, this::complete); + + Disposable oldSubscription = credentialsSubscription.getAndSet(subscription); + if (oldSubscription != null && !oldSubscription.isDisposed()) { + oldSubscription.dispose(); + } + } + } + + /** + * Unsubscribes from the current credentials stream. + */ + public void unsubscribe() { + Disposable subscription = credentialsSubscription.getAndSet(null); + if (subscription != null && !subscription.isDisposed()) { + subscription.dispose(); + } + } + + protected void complete() { + log.debug("Credentials stream completed"); + } + + protected void onNext(RedisCredentials credentials) { + reauthenticate(credentials); + } + + protected void onError(Throwable e) { + log.error("Credentials renew failed.", e); + } + + /** + * Performs re-authentication with the provided credentials. + * + * @param credentials the new credentials + */ + private void reauthenticate(RedisCredentials credentials) { + CharSequence password = CharBuffer.wrap(credentials.getPassword()); + + AsyncCommand authCmd; + if (credentials.hasUsername()) { + authCmd = new AsyncCommand<>(commandBuilder.auth(credentials.getUsername(), password)); + } else { + authCmd = new AsyncCommand<>(commandBuilder.auth(password)); + } + + dispatchAuth(authCmd).exceptionally(throwable -> { + log.error("Re-authentication {} failed.", credentials.hasUsername() ? "with username" : "without username", + throwable); + return null; + }); + } + + protected boolean isSupportedConnection() { + return true; + } + + private AsyncCommand dispatchAuth(RedisCommand authCommand) { + AsyncCommand asyncCommand = new AsyncCommand<>(authCommand); + RedisCommand dispatched = connection.getChannelWriter().write(asyncCommand); + if (dispatched instanceof AsyncCommand) { + return (AsyncCommand) dispatched; + } + return asyncCommand; + } + +} diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java new file mode 100644 index 000000000..cdf57e987 --- /dev/null +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -0,0 +1,44 @@ +/* + * Copyright 2019-Present, Redis Ltd. and Contributors + * All rights reserved. + * + * Licensed under the MIT License. + * + * This file contains contributions from third-party contributors + * licensed 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 + * + * https://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 io.lettuce.core; + +import io.lettuce.core.protocol.ProtocolVersion; +import io.lettuce.core.pubsub.StatefulRedisPubSubConnection; +import io.netty.util.internal.logging.InternalLogger; +import io.netty.util.internal.logging.InternalLoggerFactory; + +class RedisAuthenticationHandler extends BaseRedisAuthenticationHandler> { + + private static final InternalLogger logger = InternalLoggerFactory.getInstance(RedisAuthenticationHandler.class); + + public RedisAuthenticationHandler(StatefulRedisConnectionImpl connection) { + super(connection); + } + + protected boolean isSupportedConnection() { + if (connection instanceof StatefulRedisPubSubConnection + && ProtocolVersion.RESP2 == connection.getConnectionState().getNegotiatedProtocolVersion()) { + logger.warn("Renewable credentials are not supported with RESP2 protocol on a pub/sub connection."); + return false; + } + return true; + } + +} diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index 14ba7b570..b385c6c36 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -67,6 +67,8 @@ public class StatefulRedisConnectionImpl extends RedisChannelHandler private final PushHandler pushHandler; + private final RedisAuthenticationHandler authHandler; + private final Mono parser; protected MultiOutput multi; @@ -104,6 +106,8 @@ public StatefulRedisConnectionImpl(RedisChannelWriter writer, PushHandler pushHa this.async = newRedisAsyncCommandsImpl(); this.sync = newRedisSyncCommandsImpl(); this.reactive = newRedisReactiveCommandsImpl(); + + this.authHandler = new RedisAuthenticationHandler(this); } public RedisCodec getCodec() { @@ -315,4 +319,16 @@ public ConnectionState getConnectionState() { return state; } + @Override + public void activated() { + super.activated(); + authHandler.subscribe(state.getCredentialsProvider()); + } + + @Override + public void deactivated() { + authHandler.unsubscribe(); + super.deactivated(); + } + } diff --git a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java new file mode 100644 index 000000000..08ab89850 --- /dev/null +++ b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java @@ -0,0 +1,15 @@ +package io.lettuce.core; + +import reactor.core.publisher.Flux; + +public interface StreamingCredentialsProvider extends RedisCredentialsProvider { + + /** + * Returns a {@link Flux} emitting {@link RedisCredentials} that can be used to authorize a Redis connection. This + * credential provider supports streaming credentials, meaning that it can emit multiple credentials over time. + * + * @return + */ + Flux credentials(); + +} diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java b/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java new file mode 100644 index 000000000..1e7f52405 --- /dev/null +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019-Present, Redis Ltd. and Contributors + * All rights reserved. + * + * Licensed under the MIT License. + * + * This file contains contributions from third-party contributors + * licensed 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 + * + * https://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 io.lettuce.core.cluster; + +import io.lettuce.core.BaseRedisAuthenticationHandler; +import io.lettuce.core.cluster.pubsub.StatefulRedisClusterPubSubConnection; +import io.lettuce.core.protocol.ProtocolVersion; +import io.netty.util.internal.logging.InternalLogger; +import io.netty.util.internal.logging.InternalLoggerFactory; + +class RedisClusterAuthenticationHandler extends BaseRedisAuthenticationHandler> { + + private static final InternalLogger logger = InternalLoggerFactory.getInstance(RedisClusterAuthenticationHandler.class); + + public RedisClusterAuthenticationHandler(StatefulRedisClusterConnectionImpl connection) { + super(connection); + } + + protected boolean isSupportedConnection() { + if (connection instanceof StatefulRedisClusterPubSubConnection + && ProtocolVersion.RESP2 == connection.getConnectionState().getNegotiatedProtocolVersion()) { + logger.warn("Renewable credentials are not supported with RESP2 protocol on a pub/sub connection."); + return false; + } + return true; + } + +} diff --git a/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java b/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java index c84193491..109c0da39 100644 --- a/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java +++ b/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java @@ -89,6 +89,8 @@ public class StatefulRedisClusterConnectionImpl extends RedisChannelHandle private volatile Partitions partitions; + private final RedisClusterAuthenticationHandler authHandler; + /** * Initialize a new connection. * @@ -123,6 +125,8 @@ public StatefulRedisClusterConnectionImpl(RedisChannelWriter writer, ClusterPush this.async = newRedisAdvancedClusterAsyncCommandsImpl(); this.sync = newRedisAdvancedClusterCommandsImpl(); this.reactive = newRedisAdvancedClusterReactiveCommandsImpl(); + + this.authHandler = new RedisClusterAuthenticationHandler(this); } protected RedisAdvancedClusterReactiveCommandsImpl newRedisAdvancedClusterReactiveCommandsImpl() { @@ -230,6 +234,12 @@ public void activated() { super.activated(); async.clusterMyId().thenAccept(connectionState::setNodeId); + authHandler.subscribe(connectionState.getCredentialsProvider()); + } + + @Override + public void deactivated() { + authHandler.unsubscribe(); } ClusterDistributionChannelWriter getClusterDistributionChannelWriter() { diff --git a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java index 864a2103b..93d502747 100644 --- a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java +++ b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java @@ -5,6 +5,10 @@ import javax.inject.Inject; +import io.lettuce.core.event.command.CommandListener; +import io.lettuce.core.event.command.CommandSucceededEvent; +import io.lettuce.core.protocol.RedisCommand; +import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -19,12 +23,20 @@ import io.lettuce.test.WithPassword; import io.lettuce.test.condition.EnabledOnCommand; import io.lettuce.test.settings.TestSettings; +import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; +import reactor.core.publisher.Sinks; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; /** * Integration test for authentication. * * @author Mark Paluch + * @author Ivo Gaydajiev */ @Tag(INTEGRATION_TEST) @ExtendWith(LettuceExtension.class) @@ -71,4 +83,116 @@ void ownCredentialProvider(RedisClient client) { }); } + // Simulate test user credential rotation, and verify that re-authentication is successful + @Test + @Inject + void renewableCredentialProvider(RedisClient client) { + + // Thread-safe list to capture intercepted commands + List> interceptedCommands = Collections.synchronizedList(new ArrayList<>()); + + // CommandListener to track successful commands + CommandListener commandListener = new CommandListener() { + + @Override + public void commandSucceeded(CommandSucceededEvent event) { + interceptedCommands.add(event.getCommand()); + } + + }; + + // Add CommandListener to the client + client.addListener(commandListener); + + // Configure client options + client.setOptions( + ClientOptions.builder().disconnectedBehavior(ClientOptions.DisconnectedBehavior.REJECT_COMMANDS).build()); + + // Connection for managing test user credential rotation + StatefulRedisConnection adminConnection = client.connect(); + + String testUser = "streaming_cred_test_user"; + char[] initialPassword = "token_1".toCharArray(); + char[] updatedPassword = "token_2".toCharArray(); + + // Streaming credentials provider to simulate token emission + RenewableRedisCredentialsProvider credentialsProvider = new RenewableRedisCredentialsProvider(); + + // Build RedisURI with streaming credentials provider + RedisURI uri = RedisURI.builder().withHost(TestSettings.host()).withPort(TestSettings.port()) + .withClientName("streaming_cred_test").withAuthentication(credentialsProvider) + .withTimeout(Duration.ofSeconds(1)).build(); + + // Create test user and set initial credentials + createTestUser(adminConnection, testUser, initialPassword); + credentialsProvider.emitToken(new StaticRedisCredentials(testUser, initialPassword)); + + // Establish connection using the streaming credentials provider + StatefulRedisConnection userConnection = client.connect(StringCodec.UTF8, uri); + + // Verify initial authentication + assertThat(userConnection.sync().aclWhoami()).isEqualTo(testUser); + + // Update test user credentials and emit updated credentials + updateTestUser(adminConnection, testUser, updatedPassword); + credentialsProvider.emitToken(new StaticRedisCredentials(testUser, updatedPassword)); + + // Wait for the `AUTH` command with updated credentials + Awaitility.await().atMost(Duration.ofSeconds(1)).until(() -> interceptedCommands.stream() + .anyMatch(command -> isAuthCommandWithCredentials(command, testUser, updatedPassword))); + + // Verify re-authentication and connection functionality + assertThat(userConnection.sync().ping()).isEqualTo("PONG"); + assertThat(userConnection.sync().aclWhoami()).isEqualTo(testUser); + + // Clean up + adminConnection.close(); + userConnection.close(); + } + + private void createTestUser(StatefulRedisConnection connection, String username, char[] password) { + AclSetuserArgs args = AclSetuserArgs.Builder.on().allCommands().allChannels().allKeys().nopass() + .addPassword(String.valueOf(password)); + connection.sync().aclSetuser(username, args); + } + + private void updateTestUser(StatefulRedisConnection connection, String username, char[] newPassword) { + AclSetuserArgs args = AclSetuserArgs.Builder.on().allCommands().allChannels().allKeys().nopass() + .addPassword(String.valueOf(newPassword)); + connection.sync().aclSetuser(username, args); + } + + private boolean isAuthCommandWithCredentials(RedisCommand command, String username, char[] password) { + if (command.getType() == CommandType.AUTH) { + CommandArgs args = command.getArgs(); + return args.toCommandString().contains(username) && args.toCommandString().contains(String.valueOf(password)); + } + return false; + } + + static class RenewableRedisCredentialsProvider implements StreamingCredentialsProvider { + + private final Sinks.Many credentialsSink = Sinks.many().replay().latest(); + + @Override + public Mono resolveCredentials() { + + return credentialsSink.asFlux().next(); + } + + public Flux credentials() { + + return credentialsSink.asFlux().onBackpressureLatest(); // Provide a continuous stream of credentials + } + + public void shutdown() { + credentialsSink.tryEmitComplete(); + } + + public void emitToken(RedisCredentials credentials) { + credentialsSink.tryEmitNext(credentials); + } + + } + } diff --git a/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java b/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java new file mode 100644 index 000000000..ba5b087b1 --- /dev/null +++ b/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java @@ -0,0 +1,111 @@ +package io.lettuce.core; + +import io.lettuce.core.protocol.CommandType; +import io.lettuce.core.protocol.RedisCommand; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Sinks; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +class BaseRedisAuthenticationHandlerTest { + + private BaseRedisAuthenticationHandler> handler; + + private RedisChannelHandler connection; + + private RedisChannelWriter channelWriter; + + private StreamingCredentialsProvider streamingCredentialsProvider; + + private Sinks.Many sink; + + @BeforeEach + void setUp() { + + connection = mock(RedisChannelHandler.class); + channelWriter = mock(RedisChannelWriter.class); + when(connection.getChannelWriter()).thenReturn(channelWriter); + streamingCredentialsProvider = mock(StreamingCredentialsProvider.class); + sink = Sinks.many().replay().latest(); + Flux credentialsFlux = sink.asFlux(); + when(streamingCredentialsProvider.credentials()).thenReturn(credentialsFlux); + handler = new BaseRedisAuthenticationHandler>(connection) { + + @Override + protected boolean isSupportedConnection() { + return true; + } + + }; + } + + @SuppressWarnings("unchecked") + @Test + void subscribeWithStreamingCredentialsProviderInvokesReauth() { + + // Subscribe to the provider + handler.subscribe(streamingCredentialsProvider); + sink.tryEmitNext(RedisCredentials.just("newuser", "newpassword")); + + // Ensure credentials() method was invoked + verify(streamingCredentialsProvider).credentials(); + + // Verify that write() is invoked once + verify(channelWriter, times(1)).write(any(RedisCommand.class)); + + ArgumentCaptor> captor = ArgumentCaptor.forClass(RedisCommand.class); + verify(channelWriter).write(captor.capture()); + + RedisCommand capturedCommand = captor.getValue(); + assertThat(capturedCommand.getType()).isEqualTo(CommandType.AUTH); + assertThat(capturedCommand.getArgs().toCommandString()).contains("newuser"); + assertThat(capturedCommand.getArgs().toCommandString()).contains("newpassword"); + } + + @Test + void shouldHandleErrorInCredentialsStream() { + Sinks.Many sink = Sinks.many().replay().latest(); + Flux credentialsFlux = sink.asFlux(); + StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); + when(credentialsProvider.credentials()).thenReturn(credentialsFlux); + + // Subscribe to the provider and simulate an error + handler.subscribe(credentialsProvider); + sink.tryEmitError(new RuntimeException("Test error")); + + verify(connection.getChannelWriter(), times(0)).write(any(RedisCommand.class)); // No command should be sent + } + + @Test + void shouldNotSubscribeIfConnectionIsNotSupported() { + Sinks.Many sink = Sinks.many().replay().latest(); + Flux credentialsFlux = sink.asFlux(); + StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); + when(credentialsProvider.credentials()).thenReturn(credentialsFlux); + + BaseRedisAuthenticationHandler handler = new BaseRedisAuthenticationHandler>(connection) { + + @Override + protected boolean isSupportedConnection() { + // Simulate : Pub/Sub connections are not supported with RESP2 + return false; + } + + }; + + // Subscribe to the provider (it should not subscribe due to unsupported connection) + handler.subscribe(credentialsProvider); + + // Ensure credentials() was not called + verify(credentialsProvider, times(0)).credentials(); + } + +} diff --git a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java new file mode 100644 index 000000000..04ca9405f --- /dev/null +++ b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java @@ -0,0 +1,52 @@ +package io.lettuce.core; + +import io.lettuce.core.protocol.ProtocolVersion; +import io.lettuce.core.pubsub.StatefulRedisPubSubConnection; +import io.lettuce.core.pubsub.StatefulRedisPubSubConnectionImpl; +import org.junit.jupiter.api.Test; + +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.when; +import static org.mockito.Mockito.withSettings; + +public class RedisAuthenticationHandlerTest { + + @Test + void testIsSupportedConnectionWithRESP2ProtocolOnPubSubConnection() { + StatefulRedisPubSubConnectionImpl connection = mock(StatefulRedisPubSubConnectionImpl.class, + withSettings().extraInterfaces(StatefulRedisPubSubConnection.class)); + + ConnectionState connectionState = mock(ConnectionState.class); + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); + when(connection.getConnectionState()).thenReturn(connectionState); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection); + + assertFalse(handler.isSupportedConnection()); + } + + @Test + void testIsSupportedConnectionWithNonPubSubConnection() { + StatefulRedisConnectionImpl connection = mock(StatefulRedisConnectionImpl.class); + ConnectionState connectionState = mock(ConnectionState.class); + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); + when(connection.getConnectionState()).thenReturn(connectionState); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection); + + assertTrue(handler.isSupportedConnection()); + } + + @Test + void testIsSupportedConnectionWithRESP3ProtocolOnPubSubConnection() { + + StatefulRedisPubSubConnectionImpl connection = mock(StatefulRedisPubSubConnectionImpl.class); + ConnectionState connectionState = mock(ConnectionState.class); + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP3); + when(connection.getConnectionState()).thenReturn(connectionState); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection); + + assertTrue(handler.isSupportedConnection()); + } + +} From e9d4d63bd0c3ad6e26025cd161a62c4645cd270b Mon Sep 17 00:00:00 2001 From: ggivo Date: Tue, 3 Dec 2024 21:23:22 +0200 Subject: [PATCH 02/28] Tests & publish ReauthEvent --- .../core/BaseRedisAuthenticationHandler.java | 34 +++- .../core/RedisAuthenticationHandler.java | 6 +- .../core/StatefulRedisConnectionImpl.java | 2 +- .../RedisClusterAuthenticationHandler.java | 5 +- .../StatefulRedisClusterConnectionImpl.java | 2 +- .../core/event/connection/ReauthEvent.java | 22 +++ .../event/connection/ReauthFailedEvent.java | 36 ++++ .../core/AuthenticationIntegrationTests.java | 111 +++--------- .../BaseRedisAuthenticationHandlerTest.java | 49 ++--- .../MyStreamingRedisCredentialsProvider.java | 38 ++++ .../core/RedisAuthenticationHandlerTest.java | 7 +- ...gCredentialsProviderlIntegrationTests.java | 168 ++++++++++++++++++ 12 files changed, 351 insertions(+), 129 deletions(-) create mode 100644 src/main/java/io/lettuce/core/event/connection/ReauthEvent.java create mode 100644 src/main/java/io/lettuce/core/event/connection/ReauthFailedEvent.java create mode 100644 src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java create mode 100644 src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java diff --git a/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java index fa9f9eb84..ce5b7f850 100644 --- a/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java @@ -1,7 +1,11 @@ package io.lettuce.core; import io.lettuce.core.codec.StringCodec; +import io.lettuce.core.event.EventBus; +import io.lettuce.core.event.connection.ReauthEvent; +import io.lettuce.core.event.connection.ReauthFailedEvent; import io.lettuce.core.protocol.AsyncCommand; +import io.lettuce.core.protocol.Endpoint; import io.lettuce.core.protocol.RedisCommand; import io.netty.util.internal.logging.InternalLogger; import io.netty.util.internal.logging.InternalLoggerFactory; @@ -21,8 +25,11 @@ public abstract class BaseRedisAuthenticationHandler credentialsSubscription = new AtomicReference<>(); - public BaseRedisAuthenticationHandler(T connection) { + protected final EventBus eventBus; + + public BaseRedisAuthenticationHandler(T connection, EventBus eventBus) { this.connection = connection; + this.eventBus = eventBus; } /** @@ -94,11 +101,23 @@ private void reauthenticate(RedisCredentials credentials) { authCmd = new AsyncCommand<>(commandBuilder.auth(password)); } - dispatchAuth(authCmd).exceptionally(throwable -> { - log.error("Re-authentication {} failed.", credentials.hasUsername() ? "with username" : "without username", - throwable); + dispatchAuth(authCmd).thenRun(() -> { + publishReauthEvent(); + log.info("Re-authentication succeeded for endpoint {}.", getEpid()); + }).exceptionally(throwable -> { + publishReauthFailedEvent(throwable); + log.error("Re-authentication failed for endpoint {}.", getEpid(), throwable); return null; }); + ; + } + + private void publishReauthEvent() { + eventBus.publish(new ReauthEvent(getEpid())); + } + + private void publishReauthFailedEvent(Throwable throwable) { + eventBus.publish(new ReauthFailedEvent(getEpid(), throwable)); } protected boolean isSupportedConnection() { @@ -114,4 +133,11 @@ private AsyncCommand dispatchAuth(RedisCommand connection) { - super(connection); + public RedisAuthenticationHandler(StatefulRedisConnectionImpl connection, EventBus eventBus) { + super(connection, eventBus); } protected boolean isSupportedConnection() { diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index b385c6c36..ef084f398 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -107,7 +107,7 @@ public StatefulRedisConnectionImpl(RedisChannelWriter writer, PushHandler pushHa this.sync = newRedisSyncCommandsImpl(); this.reactive = newRedisReactiveCommandsImpl(); - this.authHandler = new RedisAuthenticationHandler(this); + this.authHandler = new RedisAuthenticationHandler(this, getResources().eventBus()); } public RedisCodec getCodec() { diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java b/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java index 1e7f52405..f4eb6eddf 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java @@ -21,6 +21,7 @@ import io.lettuce.core.BaseRedisAuthenticationHandler; import io.lettuce.core.cluster.pubsub.StatefulRedisClusterPubSubConnection; +import io.lettuce.core.event.EventBus; import io.lettuce.core.protocol.ProtocolVersion; import io.netty.util.internal.logging.InternalLogger; import io.netty.util.internal.logging.InternalLoggerFactory; @@ -29,8 +30,8 @@ class RedisClusterAuthenticationHandler extends BaseRedisAuthenticationHandler connection) { - super(connection); + public RedisClusterAuthenticationHandler(StatefulRedisClusterConnectionImpl connection, EventBus eventBus) { + super(connection, eventBus); } protected boolean isSupportedConnection() { diff --git a/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java b/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java index 109c0da39..7735a23b2 100644 --- a/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java +++ b/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java @@ -126,7 +126,7 @@ public StatefulRedisClusterConnectionImpl(RedisChannelWriter writer, ClusterPush this.sync = newRedisAdvancedClusterCommandsImpl(); this.reactive = newRedisAdvancedClusterReactiveCommandsImpl(); - this.authHandler = new RedisClusterAuthenticationHandler(this); + this.authHandler = new RedisClusterAuthenticationHandler(this, getResources().eventBus()); } protected RedisAdvancedClusterReactiveCommandsImpl newRedisAdvancedClusterReactiveCommandsImpl() { diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthEvent.java new file mode 100644 index 000000000..6dd2e0b4b --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/ReauthEvent.java @@ -0,0 +1,22 @@ +package io.lettuce.core.event.connection; + +import io.lettuce.core.event.Event; + +/** + * Event fired on failed authentication caused either by I/O issues or during connection reauthentication. + * + * @author Ivo Gaydajiev + */ +public class ReauthEvent implements Event { + + private final String epId; + + public ReauthEvent(String epId) { + this.epId = epId; + } + + public String getEpId() { + return epId; + } + +} diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthFailedEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthFailedEvent.java new file mode 100644 index 000000000..e7d282d2f --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/ReauthFailedEvent.java @@ -0,0 +1,36 @@ +package io.lettuce.core.event.connection; + +import io.lettuce.core.event.Event; + +import java.net.SocketAddress; + +/** + * Event fired on failed authentication caused either by I/O issues or during connection reauthentication. + * + * @author Ivo Gaydajiev + */ +public class ReauthFailedEvent implements Event { + + private final String epId; + + private final Throwable cause; + + public ReauthFailedEvent(String epId, Throwable cause) { + this.epId = epId; + this.cause = cause; + } + + public String getEpId() { + return epId; + } + + /** + * Returns the {@link Throwable} that describes the reauth failure cause. + * + * @return the {@link Throwable} that describes the reauth failure cause. + */ + public Throwable getCause() { + return cause; + } + +} diff --git a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java index 93d502747..01ef7b853 100644 --- a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java +++ b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java @@ -23,13 +23,10 @@ import io.lettuce.test.WithPassword; import io.lettuce.test.condition.EnabledOnCommand; import io.lettuce.test.settings.TestSettings; -import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.publisher.Sinks; import java.time.Duration; import java.util.ArrayList; -import java.util.Collections; import java.util.List; /** @@ -49,6 +46,9 @@ void setUp(StatefulRedisConnection connection) { connection.sync().dispatch(CommandType.ACL, new StatusOutput<>(StringCodec.UTF8), new CommandArgs<>(StringCodec.UTF8).add("SETUSER").add("john").add("on").add(">foobared").add("-@all")); + + connection.sync().dispatch(CommandType.ACL, new StatusOutput<>(StringCodec.UTF8), + new CommandArgs<>(StringCodec.UTF8).add("SETUSER").add("steave").add("on").add(">foobared").add("+@all")); } @Test @@ -86,80 +86,35 @@ void ownCredentialProvider(RedisClient client) { // Simulate test user credential rotation, and verify that re-authentication is successful @Test @Inject - void renewableCredentialProvider(RedisClient client) { - - // Thread-safe list to capture intercepted commands - List> interceptedCommands = Collections.synchronizedList(new ArrayList<>()); - - // CommandListener to track successful commands - CommandListener commandListener = new CommandListener() { - - @Override - public void commandSucceeded(CommandSucceededEvent event) { - interceptedCommands.add(event.getCommand()); - } - - }; + void streamingCredentialProvider(RedisClient client) { - // Add CommandListener to the client - client.addListener(commandListener); - - // Configure client options - client.setOptions( - ClientOptions.builder().disconnectedBehavior(ClientOptions.DisconnectedBehavior.REJECT_COMMANDS).build()); - - // Connection for managing test user credential rotation - StatefulRedisConnection adminConnection = client.connect(); - - String testUser = "streaming_cred_test_user"; - char[] initialPassword = "token_1".toCharArray(); - char[] updatedPassword = "token_2".toCharArray(); - - // Streaming credentials provider to simulate token emission - RenewableRedisCredentialsProvider credentialsProvider = new RenewableRedisCredentialsProvider(); + TestCommandListener listener = new TestCommandListener(); + client.addListener(listener); // Build RedisURI with streaming credentials provider + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); RedisURI uri = RedisURI.builder().withHost(TestSettings.host()).withPort(TestSettings.port()) .withClientName("streaming_cred_test").withAuthentication(credentialsProvider) - .withTimeout(Duration.ofSeconds(1)).build(); - - // Create test user and set initial credentials - createTestUser(adminConnection, testUser, initialPassword); - credentialsProvider.emitToken(new StaticRedisCredentials(testUser, initialPassword)); - - // Establish connection using the streaming credentials provider - StatefulRedisConnection userConnection = client.connect(StringCodec.UTF8, uri); - - // Verify initial authentication - assertThat(userConnection.sync().aclWhoami()).isEqualTo(testUser); + .withTimeout(Duration.ofSeconds(5)).build(); - // Update test user credentials and emit updated credentials - updateTestUser(adminConnection, testUser, updatedPassword); - credentialsProvider.emitToken(new StaticRedisCredentials(testUser, updatedPassword)); + credentialsProvider.emitCredentials(TestSettings.username(), TestSettings.password().toString().toCharArray()); - // Wait for the `AUTH` command with updated credentials - Awaitility.await().atMost(Duration.ofSeconds(1)).until(() -> interceptedCommands.stream() - .anyMatch(command -> isAuthCommandWithCredentials(command, testUser, updatedPassword))); + // verify that the initial connection is successful with default user credentials + StatefulRedisConnection connection = client.connect(uri); + assertThat(connection.sync().aclWhoami()).isEqualTo(TestSettings.username()); - // Verify re-authentication and connection functionality - assertThat(userConnection.sync().ping()).isEqualTo("PONG"); - assertThat(userConnection.sync().aclWhoami()).isEqualTo(testUser); + // rotate the credentials + credentialsProvider.emitCredentials("steave", "foobared".toCharArray()); - // Clean up - adminConnection.close(); - userConnection.close(); - } + Awaitility.await().atMost(Duration.ofSeconds(1)).until(() -> listener.succeeded.stream() + .anyMatch(command -> isAuthCommandWithCredentials(command, "steave", "foobared".toCharArray()))); - private void createTestUser(StatefulRedisConnection connection, String username, char[] password) { - AclSetuserArgs args = AclSetuserArgs.Builder.on().allCommands().allChannels().allKeys().nopass() - .addPassword(String.valueOf(password)); - connection.sync().aclSetuser(username, args); - } + // verify that the connection is re-authenticated with the new user credentials + assertThat(connection.sync().aclWhoami()).isEqualTo("steave"); - private void updateTestUser(StatefulRedisConnection connection, String username, char[] newPassword) { - AclSetuserArgs args = AclSetuserArgs.Builder.on().allCommands().allChannels().allKeys().nopass() - .addPassword(String.valueOf(newPassword)); - connection.sync().aclSetuser(username, args); + credentialsProvider.shutdown(); + connection.close(); + client.removeListener(listener); } private boolean isAuthCommandWithCredentials(RedisCommand command, String username, char[] password) { @@ -170,27 +125,15 @@ private boolean isAuthCommandWithCredentials(RedisCommand command, Stri return false; } - static class RenewableRedisCredentialsProvider implements StreamingCredentialsProvider { + static class TestCommandListener implements CommandListener { - private final Sinks.Many credentialsSink = Sinks.many().replay().latest(); + final List> succeeded = new ArrayList<>(); @Override - public Mono resolveCredentials() { - - return credentialsSink.asFlux().next(); - } - - public Flux credentials() { - - return credentialsSink.asFlux().onBackpressureLatest(); // Provide a continuous stream of credentials - } - - public void shutdown() { - credentialsSink.tryEmitComplete(); - } - - public void emitToken(RedisCredentials credentials) { - credentialsSink.tryEmitNext(credentials); + public void commandSucceeded(CommandSucceededEvent event) { + synchronized (succeeded) { + succeeded.add(event.getCommand()); + } } } diff --git a/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java b/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java index ba5b087b1..0b1a0e2f5 100644 --- a/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java +++ b/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java @@ -1,5 +1,7 @@ package io.lettuce.core; +import io.lettuce.core.event.DefaultEventBus; +import io.lettuce.core.event.EventBus; import io.lettuce.core.protocol.CommandType; import io.lettuce.core.protocol.RedisCommand; import org.junit.jupiter.api.BeforeEach; @@ -7,6 +9,7 @@ import org.mockito.ArgumentCaptor; import reactor.core.publisher.Flux; import reactor.core.publisher.Sinks; +import reactor.core.scheduler.Schedulers; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; @@ -23,26 +26,13 @@ class BaseRedisAuthenticationHandlerTest { private RedisChannelWriter channelWriter; - private StreamingCredentialsProvider streamingCredentialsProvider; - - private Sinks.Many sink; - @BeforeEach void setUp() { - + EventBus eventBus = new DefaultEventBus(Schedulers.immediate()); connection = mock(RedisChannelHandler.class); channelWriter = mock(RedisChannelWriter.class); when(connection.getChannelWriter()).thenReturn(channelWriter); - streamingCredentialsProvider = mock(StreamingCredentialsProvider.class); - sink = Sinks.many().replay().latest(); - Flux credentialsFlux = sink.asFlux(); - when(streamingCredentialsProvider.credentials()).thenReturn(credentialsFlux); - handler = new BaseRedisAuthenticationHandler>(connection) { - - @Override - protected boolean isSupportedConnection() { - return true; - } + handler = new BaseRedisAuthenticationHandler>(connection, eventBus) { }; } @@ -50,16 +40,11 @@ protected boolean isSupportedConnection() { @SuppressWarnings("unchecked") @Test void subscribeWithStreamingCredentialsProviderInvokesReauth() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); // Subscribe to the provider - handler.subscribe(streamingCredentialsProvider); - sink.tryEmitNext(RedisCredentials.just("newuser", "newpassword")); - - // Ensure credentials() method was invoked - verify(streamingCredentialsProvider).credentials(); - - // Verify that write() is invoked once - verify(channelWriter, times(1)).write(any(RedisCommand.class)); + handler.subscribe(credentialsProvider); + credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); ArgumentCaptor> captor = ArgumentCaptor.forClass(RedisCommand.class); verify(channelWriter).write(captor.capture()); @@ -68,30 +53,30 @@ void subscribeWithStreamingCredentialsProviderInvokesReauth() { assertThat(capturedCommand.getType()).isEqualTo(CommandType.AUTH); assertThat(capturedCommand.getArgs().toCommandString()).contains("newuser"); assertThat(capturedCommand.getArgs().toCommandString()).contains("newpassword"); + + credentialsProvider.shutdown(); } @Test void shouldHandleErrorInCredentialsStream() { - Sinks.Many sink = Sinks.many().replay().latest(); - Flux credentialsFlux = sink.asFlux(); - StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); - when(credentialsProvider.credentials()).thenReturn(credentialsFlux); + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); // Subscribe to the provider and simulate an error handler.subscribe(credentialsProvider); - sink.tryEmitError(new RuntimeException("Test error")); + credentialsProvider.tryEmitError(new RuntimeException("Test error")); verify(connection.getChannelWriter(), times(0)).write(any(RedisCommand.class)); // No command should be sent + + credentialsProvider.shutdown(); } @Test void shouldNotSubscribeIfConnectionIsNotSupported() { - Sinks.Many sink = Sinks.many().replay().latest(); - Flux credentialsFlux = sink.asFlux(); + EventBus eventBus = new DefaultEventBus(Schedulers.immediate()); StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); - when(credentialsProvider.credentials()).thenReturn(credentialsFlux); - BaseRedisAuthenticationHandler handler = new BaseRedisAuthenticationHandler>(connection) { + BaseRedisAuthenticationHandler handler = new BaseRedisAuthenticationHandler>(connection, + eventBus) { @Override protected boolean isSupportedConnection() { diff --git a/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java b/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java new file mode 100644 index 000000000..34f60035e --- /dev/null +++ b/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java @@ -0,0 +1,38 @@ +package io.lettuce.core; + +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.core.publisher.Sinks; + +public class MyStreamingRedisCredentialsProvider implements StreamingCredentialsProvider { + + private final Sinks.Many credentialsSink = Sinks.many().replay().latest(); + + @Override + public Mono resolveCredentials() { + + return credentialsSink.asFlux().next(); + } + + public Flux credentials() { + + return credentialsSink.asFlux().onBackpressureLatest(); // Provide a continuous stream of credentials + } + + public void shutdown() { + credentialsSink.tryEmitComplete(); + } + + public void emitCredentials(RedisCredentials credentials) { + credentialsSink.tryEmitNext(credentials); + } + + public void emitCredentials(String username, char[] password) { + credentialsSink.tryEmitNext(new StaticRedisCredentials(username, password)); + } + + public void tryEmitError(RuntimeException testError) { + credentialsSink.tryEmitError(testError); + } + +} diff --git a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java index 04ca9405f..63a0d769b 100644 --- a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java +++ b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java @@ -1,5 +1,6 @@ package io.lettuce.core; +import io.lettuce.core.event.EventBus; import io.lettuce.core.protocol.ProtocolVersion; import io.lettuce.core.pubsub.StatefulRedisPubSubConnection; import io.lettuce.core.pubsub.StatefulRedisPubSubConnectionImpl; @@ -21,7 +22,7 @@ void testIsSupportedConnectionWithRESP2ProtocolOnPubSubConnection() { ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); when(connection.getConnectionState()).thenReturn(connectionState); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(EventBus.class)); assertFalse(handler.isSupportedConnection()); } @@ -32,7 +33,7 @@ void testIsSupportedConnectionWithNonPubSubConnection() { ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); when(connection.getConnectionState()).thenReturn(connectionState); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(EventBus.class)); assertTrue(handler.isSupportedConnection()); } @@ -44,7 +45,7 @@ void testIsSupportedConnectionWithRESP3ProtocolOnPubSubConnection() { ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP3); when(connection.getConnectionState()).thenReturn(connectionState); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(EventBus.class)); assertTrue(handler.isSupportedConnection()); } diff --git a/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java b/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java new file mode 100644 index 000000000..d69882567 --- /dev/null +++ b/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java @@ -0,0 +1,168 @@ +package io.lettuce.core.cluster; + +import io.lettuce.core.AclSetuserArgs; +import io.lettuce.core.MyStreamingRedisCredentialsProvider; +import io.lettuce.core.RedisCommandExecutionException; +import io.lettuce.core.RedisURI; +import io.lettuce.core.TestSupport; +import io.lettuce.core.api.StatefulRedisConnection; +import io.lettuce.core.api.sync.RedisCommands; +import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; +import io.lettuce.core.cluster.api.sync.Executions; +import io.lettuce.core.cluster.api.sync.RedisAdvancedClusterCommands; +import io.lettuce.test.CanConnect; +import io.lettuce.test.resource.FastShutdown; +import io.lettuce.test.resource.TestClientResources; +import io.lettuce.test.settings.TestSettings; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.stream.Collectors; + +import static io.lettuce.TestTags.INTEGRATION_TEST; +import static io.lettuce.test.settings.TestSettings.host; +import static io.lettuce.test.settings.TestSettings.hostAddr; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +/** + * @author Ivo Gaydajiev + */ +@Tag(INTEGRATION_TEST) +class RedisClusterStreamingCredentialsProviderIntegrationTests extends TestSupport { + + private static final int CLUSTER_PORT_SSL_1 = 7442; + + private static final int CLUSTER_PORT_SSL_2 = 7444; // replica cannot replicate properly with upstream + + private static final int CLUSTER_PORT_SSL_3 = 7445; + + private static final String SLOT_1_KEY = "8HMdi"; + + private static final String SLOT_16352_KEY = "UyAa4KqoWgPGKa"; + + private static MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); + + private static RedisURI redisURI = RedisURI.Builder.redis(host(), CLUSTER_PORT_SSL_1).withSsl(true) + .withAuthentication(credentialsProvider).withVerifyPeer(false).build(); + + private static RedisClusterClient redisClient = RedisClusterClient.create(TestClientResources.get(), redisURI); + + @BeforeEach + void before() { + assumeTrue(CanConnect.to(host(), CLUSTER_PORT_SSL_1), "Assume that stunnel runs on port 7442"); + assumeTrue(CanConnect.to(host(), CLUSTER_PORT_SSL_2), "Assume that stunnel runs on port 7444"); + assumeTrue(CanConnect.to(host(), CLUSTER_PORT_SSL_3), "Assume that stunnel runs on port 7445"); + assumeTrue(CanConnect.to(host(), 7479), "Assume that Redis runs on port 7479"); + assumeTrue(CanConnect.to(host(), 7480), "Assume that Redis runs on port 7480"); + assumeTrue(CanConnect.to(host(), 7481), "Assume that Redis runs on port 7481"); + } + + @BeforeAll + static void beforeClass() { + credentialsProvider.emitCredentials(TestSettings.username(), TestSettings.password().toString().toCharArray()); + } + + @AfterAll + static void afterClass() { + credentialsProvider.shutdown(); + FastShutdown.shutdown(redisClient); + } + + @Test + void defaultClusterConnectionShouldWork() { + + StatefulRedisClusterConnection connection = redisClient.connect(); + assertThat(connection.sync().ping()).isEqualTo("PONG"); + + connection.close(); + } + + @Test + void partitionViewShouldContainClusterPorts() { + + StatefulRedisClusterConnection connection = redisClient.connect(); + List ports = connection.getPartitions().stream().map(redisClusterNode -> redisClusterNode.getUri().getPort()) + .collect(Collectors.toList()); + connection.close(); + + assertThat(ports).contains(CLUSTER_PORT_SSL_1, CLUSTER_PORT_SSL_3); + } + + @Test + void routedOperationsAreWorking() { + + StatefulRedisClusterConnection connection = redisClient.connect(); + RedisAdvancedClusterCommands sync = connection.sync(); + + sync.set(SLOT_1_KEY, "value1"); + sync.set(SLOT_16352_KEY, "value2"); + + assertThat(sync.get(SLOT_1_KEY)).isEqualTo("value1"); + assertThat(sync.get(SLOT_16352_KEY)).isEqualTo("value2"); + + connection.close(); + } + + @Test + void nodeConnectionsShouldWork() { + + StatefulRedisClusterConnection connection = redisClient.connect(); + + // master 2 + StatefulRedisConnection node2Connection = connection.getConnection(hostAddr(), 7445); + + try { + node2Connection.sync().get(SLOT_1_KEY); + } catch (RedisCommandExecutionException e) { + assertThat(e).hasMessage("MOVED 1 127.0.0.1:7442"); + } + + connection.close(); + } + + @Test + void nodeSelectionApiShouldWork() { + + StatefulRedisClusterConnection connection = redisClient.connect(); + + Executions ping = connection.sync().all().commands().ping(); + assertThat(ping).hasSize(3).contains("PONG"); + + connection.close(); + } + + @Test + void shouldPerformNodeConnectionReauth() { + + StatefulRedisClusterConnection connection = redisClient.connect(); + connection.getPartitions().forEach( + partition -> createTestUser(connection.getConnection(partition.getNodeId()).sync(), "steave", "foobared")); + + credentialsProvider.emitCredentials("steave", "foobared".toCharArray()); + + // Verify each node's authenticated username matches the updated credentials + connection.getPartitions().forEach(partition -> { + StatefulRedisConnection userConn = connection.getConnection(partition.getNodeId()); + assertThat(userConn.sync().aclWhoami()).isEqualTo("steave"); + }); + + // re-auth with the default credentials + credentialsProvider.emitCredentials(TestSettings.username(), TestSettings.password().toString().toCharArray()); + + connection.getPartitions().forEach(partition -> { + connection.getConnection(partition.getNodeId()).sync().aclDeluser("steave"); + }); + + connection.close(); + } + + public static void createTestUser(RedisCommands commands, String username, String password) { + commands.aclSetuser(username, AclSetuserArgs.Builder.on().allCommands().addPassword(password)); + } + +} From 820fffffbe9cce4cf687a2688ae65f92dacd35f6 Mon Sep 17 00:00:00 2001 From: ggivo Date: Wed, 4 Dec 2024 10:59:54 +0200 Subject: [PATCH 03/28] Clean up & Format & Add ReauthenticateEvent test --- .../core/BaseRedisAuthenticationHandler.java | 17 ++++++++--- .../core/RedisAuthenticationHandler.java | 7 ++++- .../core/StreamingCredentialsProvider.java | 7 +++++ .../RedisClusterAuthenticationHandler.java | 8 +++++ .../event/connection/AuthenticateEvent.java | 15 ++++++++++ .../core/event/connection/JfrReauthEvent.java | 29 ++++++++++++++++++ .../connection/JfrReauthFailedEvent.java | 29 ++++++++++++++++++ .../core/event/connection/ReauthEvent.java | 22 -------------- .../event/connection/ReauthenticateEvent.java | 21 +++++++++++++ ...nt.java => ReauthenticateFailedEvent.java} | 14 ++++----- .../MyStreamingRedisCredentialsProvider.java | 6 ++++ ...ectionEventsTriggeredIntegrationTests.java | 30 +++++++++++++++++++ 12 files changed, 170 insertions(+), 35 deletions(-) create mode 100644 src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java create mode 100644 src/main/java/io/lettuce/core/event/connection/JfrReauthEvent.java create mode 100644 src/main/java/io/lettuce/core/event/connection/JfrReauthFailedEvent.java delete mode 100644 src/main/java/io/lettuce/core/event/connection/ReauthEvent.java create mode 100644 src/main/java/io/lettuce/core/event/connection/ReauthenticateEvent.java rename src/main/java/io/lettuce/core/event/connection/{ReauthFailedEvent.java => ReauthenticateFailedEvent.java} (68%) diff --git a/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java index ce5b7f850..74e99045a 100644 --- a/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java @@ -2,8 +2,8 @@ import io.lettuce.core.codec.StringCodec; import io.lettuce.core.event.EventBus; -import io.lettuce.core.event.connection.ReauthEvent; -import io.lettuce.core.event.connection.ReauthFailedEvent; +import io.lettuce.core.event.connection.ReauthenticateFailedEvent; +import io.lettuce.core.event.connection.ReauthenticateEvent; import io.lettuce.core.protocol.AsyncCommand; import io.lettuce.core.protocol.Endpoint; import io.lettuce.core.protocol.RedisCommand; @@ -15,6 +15,15 @@ import java.nio.CharBuffer; import java.util.concurrent.atomic.AtomicReference; +/** + * Base class for Redis authentication handlers. + *

+ * This class provides a mechanism to subscribe to a stream of credentials and re-authenticate the client when new credentials + * are received. Internal API. + * + * @author Ivor Gaydajiev + * @Since 6.5.2 + */ public abstract class BaseRedisAuthenticationHandler> { private static final InternalLogger log = InternalLoggerFactory.getInstance(BaseRedisAuthenticationHandler.class); @@ -113,11 +122,11 @@ private void reauthenticate(RedisCredentials credentials) { } private void publishReauthEvent() { - eventBus.publish(new ReauthEvent(getEpid())); + eventBus.publish(new ReauthenticateEvent(getEpid())); } private void publishReauthFailedEvent(Throwable throwable) { - eventBus.publish(new ReauthFailedEvent(getEpid(), throwable)); + eventBus.publish(new ReauthenticateFailedEvent(getEpid(), throwable)); } protected boolean isSupportedConnection() { diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index abd484b94..30bf55122 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -20,12 +20,17 @@ package io.lettuce.core; import io.lettuce.core.event.EventBus; -import io.lettuce.core.protocol.Endpoint; import io.lettuce.core.protocol.ProtocolVersion; import io.lettuce.core.pubsub.StatefulRedisPubSubConnection; import io.netty.util.internal.logging.InternalLogger; import io.netty.util.internal.logging.InternalLoggerFactory; +/** + * Redis authentication handler. Internally used to authenticate a Redis connection. Internal API. + * + * @author Ivo Gaydazhiev + * @since 6.5.2 + */ class RedisAuthenticationHandler extends BaseRedisAuthenticationHandler> { private static final InternalLogger logger = InternalLoggerFactory.getInstance(RedisAuthenticationHandler.class); diff --git a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java index 08ab89850..dba9302c7 100644 --- a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java +++ b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java @@ -2,6 +2,13 @@ import reactor.core.publisher.Flux; +/** + * A provider for streaming credentials that can be used to authorize a Redis connection and re-authenticate the connection when + * new credentials are received. + * + * @author Ivo Gaydajiev + * @since 6.5.2 + */ public interface StreamingCredentialsProvider extends RedisCredentialsProvider { /** diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java b/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java index f4eb6eddf..aae271985 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java @@ -26,6 +26,14 @@ import io.netty.util.internal.logging.InternalLogger; import io.netty.util.internal.logging.InternalLoggerFactory; +/** + * Redis Cluster authentication handler. Internally used to authenticate a Redis Cluster connection. Internal API. + * + *

+ * + * @author Ivo Gaydajiev + * @since 6.5.2 + */ class RedisClusterAuthenticationHandler extends BaseRedisAuthenticationHandler> { private static final InternalLogger logger = InternalLoggerFactory.getInstance(RedisClusterAuthenticationHandler.class); diff --git a/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java b/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java new file mode 100644 index 000000000..2d4c635e6 --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java @@ -0,0 +1,15 @@ +package io.lettuce.core.event.connection; + +import io.lettuce.core.event.Event; + +/** + * Interface for Connection authentication events + * + * @author Ivo Gaydajiev + * @since 3.4 + */ +public interface AuthenticateEvent extends Event { + + String getEpId(); + +} diff --git a/src/main/java/io/lettuce/core/event/connection/JfrReauthEvent.java b/src/main/java/io/lettuce/core/event/connection/JfrReauthEvent.java new file mode 100644 index 000000000..6f86e82f4 --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/JfrReauthEvent.java @@ -0,0 +1,29 @@ +package io.lettuce.core.event.connection; + +import jdk.jfr.Category; +import jdk.jfr.Event; +import jdk.jfr.Label; +import jdk.jfr.StackTrace; + +/** + * Flight recorder event variant of {@link ReauthenticateEvent}. + * + * @author Ivo Gaydajiev + * @since 6.5.2 + */ +@Category({ "Lettuce", "Connection Events" }) +@Label("Reauthenticate to a Redis server") +@StackTrace(value = false) +class JfrReauthEvent extends Event { + + private final String epId; + + public JfrReauthEvent(ReauthenticateEvent event) { + this.epId = event.getEpId(); + } + + public String getEpId() { + return epId; + } + +} diff --git a/src/main/java/io/lettuce/core/event/connection/JfrReauthFailedEvent.java b/src/main/java/io/lettuce/core/event/connection/JfrReauthFailedEvent.java new file mode 100644 index 000000000..d0e3e6996 --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/JfrReauthFailedEvent.java @@ -0,0 +1,29 @@ +package io.lettuce.core.event.connection; + +import jdk.jfr.Category; +import jdk.jfr.Event; +import jdk.jfr.Label; +import jdk.jfr.StackTrace; + +/** + * Flight recorder event variant of {@link ReauthEvent}. + * + * @author Ivo Gaydajiev + * @since 6.5.2 + */ +@Category({ "Lettuce", "Connection Events" }) +@Label("Reauthenticate to a Redis server failed") +@StackTrace(value = false) +class JfrReauthFailedEvent extends Event { + + private final String epId; + + public JfrReauthFailedEvent(ReauthenticateFailedEvent event) { + this.epId = event.getEpId(); + } + + public String getEpId() { + return epId; + } + +} diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthEvent.java deleted file mode 100644 index 6dd2e0b4b..000000000 --- a/src/main/java/io/lettuce/core/event/connection/ReauthEvent.java +++ /dev/null @@ -1,22 +0,0 @@ -package io.lettuce.core.event.connection; - -import io.lettuce.core.event.Event; - -/** - * Event fired on failed authentication caused either by I/O issues or during connection reauthentication. - * - * @author Ivo Gaydajiev - */ -public class ReauthEvent implements Event { - - private final String epId; - - public ReauthEvent(String epId) { - this.epId = epId; - } - - public String getEpId() { - return epId; - } - -} diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthenticateEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthenticateEvent.java new file mode 100644 index 000000000..caf1d2932 --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/ReauthenticateEvent.java @@ -0,0 +1,21 @@ +package io.lettuce.core.event.connection; + +/** + * Event fired on successfull connection re-authentication. see {@link io.lettuce.core.StreamingCredentialsProvider} + * + * @author Ivo Gaydajiev + * @since 6.5.2 + */ +public class ReauthenticateEvent implements AuthenticateEvent { + + private final String epId; + + public ReauthenticateEvent(String epId) { + this.epId = epId; + } + + public String getEpId() { + return epId; + } + +} diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthFailedEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthenticateFailedEvent.java similarity index 68% rename from src/main/java/io/lettuce/core/event/connection/ReauthFailedEvent.java rename to src/main/java/io/lettuce/core/event/connection/ReauthenticateFailedEvent.java index e7d282d2f..f808daa71 100644 --- a/src/main/java/io/lettuce/core/event/connection/ReauthFailedEvent.java +++ b/src/main/java/io/lettuce/core/event/connection/ReauthenticateFailedEvent.java @@ -1,21 +1,19 @@ package io.lettuce.core.event.connection; -import io.lettuce.core.event.Event; - -import java.net.SocketAddress; - /** - * Event fired on failed authentication caused either by I/O issues or during connection reauthentication. - * + * Event fired on failed authentication caused either by I/O issues or during connection re-authentication. see + * {@link io.lettuce.core.StreamingCredentialsProvider} + * * @author Ivo Gaydajiev + * @since 6.5.2 */ -public class ReauthFailedEvent implements Event { +public class ReauthenticateFailedEvent implements AuthenticateEvent { private final String epId; private final Throwable cause; - public ReauthFailedEvent(String epId, Throwable cause) { + public ReauthenticateFailedEvent(String epId, Throwable cause) { this.epId = epId; this.cause = cause; } diff --git a/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java b/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java index 34f60035e..6a2a43697 100644 --- a/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java +++ b/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java @@ -4,6 +4,12 @@ import reactor.core.publisher.Mono; import reactor.core.publisher.Sinks; +/** + * A provider for streaming credentials that can be used to authorize a Redis connection + * + * @author Ivo Gaydajiev + * @since 6.5.2 + */ public class MyStreamingRedisCredentialsProvider implements StreamingCredentialsProvider { private final Sinks.Many credentialsSink = Sinks.many().replay().latest(); diff --git a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java index 21d9eb5e8..102a00a55 100644 --- a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java +++ b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java @@ -6,6 +6,13 @@ import java.time.Duration; import java.time.temporal.ChronoUnit; +import io.lettuce.core.MyStreamingRedisCredentialsProvider; +import io.lettuce.core.event.connection.AuthenticateEvent; +import io.lettuce.core.event.connection.ReauthenticateEvent; +import io.lettuce.core.event.connection.ReauthenticateFailedEvent; +import io.lettuce.test.WithPassword; +import io.lettuce.test.settings.TestSettings; +import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -20,6 +27,7 @@ /** * @author Mark Paluch + * @author Ivo Gaydajiev */ @Tag(INTEGRATION_TEST) class ConnectionEventsTriggeredIntegrationTests extends TestSupport { @@ -41,4 +49,26 @@ void testConnectionEvents() { FastShutdown.shutdown(client); } + @Test + void testReauthConnectionEvents() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); + credentialsProvider.emitCredentials(TestSettings.username(), TestSettings.password().toString().toCharArray()); + RedisClient client = RedisClient.create(TestClientResources.get(), + RedisURI.Builder.redis(host, port).withAuthentication(credentialsProvider).build()); + + Flux publisher = client.getResources().eventBus().get() + .filter(event -> event instanceof AuthenticateEvent).cast(AuthenticateEvent.class); + + StepVerifier.create(publisher).then(() -> WithPassword.run(client, () -> client.connect().close())) + .assertNext(event -> assertThat(event).asInstanceOf(InstanceOfAssertFactories.type(ReauthenticateEvent.class)) + .extracting(ReauthenticateEvent::getEpId).isNotNull()) + .then(() -> credentialsProvider.emitCredentials(TestSettings.username(), "invalid".toCharArray())) + .assertNext( + event -> assertThat(event).asInstanceOf(InstanceOfAssertFactories.type(ReauthenticateFailedEvent.class)) + .extracting(ReauthenticateFailedEvent::getEpId).isNotNull()) + .thenCancel().verify(Duration.of(5, ChronoUnit.SECONDS)); + + FastShutdown.shutdown(client); + } + } From 58582864f3e2f20165553de807c5d1b289f45cd5 Mon Sep 17 00:00:00 2001 From: ggivo Date: Fri, 6 Dec 2024 17:22:13 +0200 Subject: [PATCH 04/28] Conditionally enable connection reauthentication based on client setting DEFAULT_REAUTHENTICATE_BEHAVIOUR --- .../core/BaseRedisAuthenticationHandler.java | 152 ---------------- .../java/io/lettuce/core/ClientOptions.java | 44 ++++- .../core/RedisAuthenticationHandler.java | 169 +++++++++++++++++- .../java/io/lettuce/core/RedisClient.java | 12 +- .../core/StatefulRedisConnectionImpl.java | 23 ++- .../core/cluster/ClusterClientOptions.java | 18 +- .../RedisClusterAuthenticationHandler.java | 54 ------ .../core/cluster/RedisClusterClient.java | 10 ++ .../StatefulRedisClusterConnectionImpl.java | 7 +- .../event/connection/AuthenticateEvent.java | 2 +- .../core/AuthenticationIntegrationTests.java | 20 ++- .../BaseRedisAuthenticationHandlerTest.java | 96 ---------- .../core/RedisAuthenticationHandlerTest.java | 108 +++++++++-- ...gCredentialsProviderlIntegrationTests.java | 6 + ...ectionEventsTriggeredIntegrationTests.java | 8 +- 15 files changed, 371 insertions(+), 358 deletions(-) delete mode 100644 src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java delete mode 100644 src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java delete mode 100644 src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java diff --git a/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java deleted file mode 100644 index 74e99045a..000000000 --- a/src/main/java/io/lettuce/core/BaseRedisAuthenticationHandler.java +++ /dev/null @@ -1,152 +0,0 @@ -package io.lettuce.core; - -import io.lettuce.core.codec.StringCodec; -import io.lettuce.core.event.EventBus; -import io.lettuce.core.event.connection.ReauthenticateFailedEvent; -import io.lettuce.core.event.connection.ReauthenticateEvent; -import io.lettuce.core.protocol.AsyncCommand; -import io.lettuce.core.protocol.Endpoint; -import io.lettuce.core.protocol.RedisCommand; -import io.netty.util.internal.logging.InternalLogger; -import io.netty.util.internal.logging.InternalLoggerFactory; -import reactor.core.Disposable; -import reactor.core.publisher.Flux; - -import java.nio.CharBuffer; -import java.util.concurrent.atomic.AtomicReference; - -/** - * Base class for Redis authentication handlers. - *

- * This class provides a mechanism to subscribe to a stream of credentials and re-authenticate the client when new credentials - * are received. Internal API. - * - * @author Ivor Gaydajiev - * @Since 6.5.2 - */ -public abstract class BaseRedisAuthenticationHandler> { - - private static final InternalLogger log = InternalLoggerFactory.getInstance(BaseRedisAuthenticationHandler.class); - - protected final T connection; - - private final RedisCommandBuilder commandBuilder = new RedisCommandBuilder<>(StringCodec.UTF8); - - private final AtomicReference credentialsSubscription = new AtomicReference<>(); - - protected final EventBus eventBus; - - public BaseRedisAuthenticationHandler(T connection, EventBus eventBus) { - this.connection = connection; - this.eventBus = eventBus; - } - - /** - * Subscribes to the provided `Flux` of credentials if the given `RedisCredentialsProvider` supports streaming credentials. - *

- * This method subscribes to a stream of credentials provided by the `StreamingCredentialsProvider`. Each time new - * credentials are received, the client is reauthenticated. If the connection is not supported, the method returns without - * subscribing. - *

- * The previous subscription, if any, is disposed of before setting the new subscription. - * - * @param credentialsProvider the credentials provider to subscribe to - */ - public void subscribe(RedisCredentialsProvider credentialsProvider) { - if (credentialsProvider == null) { - return; - } - - if (credentialsProvider instanceof StreamingCredentialsProvider) { - if (!isSupportedConnection()) { - return; - } - - Flux credentialsFlux = ((StreamingCredentialsProvider) credentialsProvider).credentials(); - - Disposable subscription = credentialsFlux.subscribe(this::onNext, this::onError, this::complete); - - Disposable oldSubscription = credentialsSubscription.getAndSet(subscription); - if (oldSubscription != null && !oldSubscription.isDisposed()) { - oldSubscription.dispose(); - } - } - } - - /** - * Unsubscribes from the current credentials stream. - */ - public void unsubscribe() { - Disposable subscription = credentialsSubscription.getAndSet(null); - if (subscription != null && !subscription.isDisposed()) { - subscription.dispose(); - } - } - - protected void complete() { - log.debug("Credentials stream completed"); - } - - protected void onNext(RedisCredentials credentials) { - reauthenticate(credentials); - } - - protected void onError(Throwable e) { - log.error("Credentials renew failed.", e); - } - - /** - * Performs re-authentication with the provided credentials. - * - * @param credentials the new credentials - */ - private void reauthenticate(RedisCredentials credentials) { - CharSequence password = CharBuffer.wrap(credentials.getPassword()); - - AsyncCommand authCmd; - if (credentials.hasUsername()) { - authCmd = new AsyncCommand<>(commandBuilder.auth(credentials.getUsername(), password)); - } else { - authCmd = new AsyncCommand<>(commandBuilder.auth(password)); - } - - dispatchAuth(authCmd).thenRun(() -> { - publishReauthEvent(); - log.info("Re-authentication succeeded for endpoint {}.", getEpid()); - }).exceptionally(throwable -> { - publishReauthFailedEvent(throwable); - log.error("Re-authentication failed for endpoint {}.", getEpid(), throwable); - return null; - }); - ; - } - - private void publishReauthEvent() { - eventBus.publish(new ReauthenticateEvent(getEpid())); - } - - private void publishReauthFailedEvent(Throwable throwable) { - eventBus.publish(new ReauthenticateFailedEvent(getEpid(), throwable)); - } - - protected boolean isSupportedConnection() { - return true; - } - - private AsyncCommand dispatchAuth(RedisCommand authCommand) { - AsyncCommand asyncCommand = new AsyncCommand<>(authCommand); - RedisCommand dispatched = connection.getChannelWriter().write(asyncCommand); - if (dispatched instanceof AsyncCommand) { - return (AsyncCommand) dispatched; - } - return asyncCommand; - } - - private String getEpid() { - if (connection.getChannelWriter() instanceof Endpoint) { - return ((Endpoint) connection.getChannelWriter()).getId(); - } - return "unknown"; - } - -} diff --git a/src/main/java/io/lettuce/core/ClientOptions.java b/src/main/java/io/lettuce/core/ClientOptions.java index 9f1f1c33d..eca9a3789 100644 --- a/src/main/java/io/lettuce/core/ClientOptions.java +++ b/src/main/java/io/lettuce/core/ClientOptions.java @@ -55,6 +55,8 @@ public class ClientOptions implements Serializable { public static final DisconnectedBehavior DEFAULT_DISCONNECTED_BEHAVIOR = DisconnectedBehavior.DEFAULT; + public static final ReauthenticateBehavior DEFAULT_REAUTHENTICATE_BEHAVIOUR = ReauthenticateBehavior.DEFAULT; + public static final boolean DEFAULT_PUBLISH_ON_SCHEDULER = false; public static final boolean DEFAULT_PING_BEFORE_ACTIVATE_CONNECTION = true; @@ -93,6 +95,8 @@ public class ClientOptions implements Serializable { private final DisconnectedBehavior disconnectedBehavior; + private final ReauthenticateBehavior reauthenticateBehavior; + private final boolean publishOnScheduler; private final boolean pingBeforeActivateConnection; @@ -120,6 +124,7 @@ protected ClientOptions(Builder builder) { this.cancelCommandsOnReconnectFailure = builder.cancelCommandsOnReconnectFailure; this.decodeBufferPolicy = builder.decodeBufferPolicy; this.disconnectedBehavior = builder.disconnectedBehavior; + this.reauthenticateBehavior = builder.reauthenticateBehavior; this.publishOnScheduler = builder.publishOnScheduler; this.pingBeforeActivateConnection = builder.pingBeforeActivateConnection; this.protocolVersion = builder.protocolVersion; @@ -138,6 +143,7 @@ protected ClientOptions(ClientOptions original) { this.cancelCommandsOnReconnectFailure = original.isCancelCommandsOnReconnectFailure(); this.decodeBufferPolicy = original.getDecodeBufferPolicy(); this.disconnectedBehavior = original.getDisconnectedBehavior(); + this.reauthenticateBehavior = original.getReauthenticateBehaviour(); this.publishOnScheduler = original.isPublishOnScheduler(); this.pingBeforeActivateConnection = original.isPingBeforeActivateConnection(); this.protocolVersion = original.getConfiguredProtocolVersion(); @@ -214,6 +220,8 @@ public static class Builder { private TimeoutOptions timeoutOptions = DEFAULT_TIMEOUT_OPTIONS; + private ReauthenticateBehavior reauthenticateBehavior = DEFAULT_REAUTHENTICATE_BEHAVIOUR; + protected Builder() { } @@ -293,6 +301,13 @@ public Builder disconnectedBehavior(DisconnectedBehavior disconnectedBehavior) { return this; } + public Builder reauthenticateBehavior(ReauthenticateBehavior reauthenticateBehavior) { + + LettuceAssert.notNull(reauthenticateBehavior, "ReuthenticatBehavior must not be null"); + this.reauthenticateBehavior = reauthenticateBehavior; + return this; + } + /** * Perform a lightweight {@literal PING} connection handshake when establishing a Redis connection. If {@code true} * (default is {@code true}, {@link #DEFAULT_PING_BEFORE_ACTIVATE_CONNECTION}), every connection and reconnect will @@ -484,11 +499,12 @@ public ClientOptions.Builder mutate() { builder.autoReconnect(isAutoReconnect()).cancelCommandsOnReconnectFailure(isCancelCommandsOnReconnectFailure()) .decodeBufferPolicy(getDecodeBufferPolicy()).disconnectedBehavior(getDisconnectedBehavior()) - .readOnlyCommands(getReadOnlyCommands()).publishOnScheduler(isPublishOnScheduler()) - .pingBeforeActivateConnection(isPingBeforeActivateConnection()).protocolVersion(getConfiguredProtocolVersion()) - .requestQueueSize(getRequestQueueSize()).scriptCharset(getScriptCharset()).jsonParser(getJsonParser()) - .socketOptions(getSocketOptions()).sslOptions(getSslOptions()) - .suspendReconnectOnProtocolFailure(isSuspendReconnectOnProtocolFailure()).timeoutOptions(getTimeoutOptions()); + .reauthenticateBehavior(getReauthenticateBehaviour()).readOnlyCommands(getReadOnlyCommands()) + .publishOnScheduler(isPublishOnScheduler()).pingBeforeActivateConnection(isPingBeforeActivateConnection()) + .protocolVersion(getConfiguredProtocolVersion()).requestQueueSize(getRequestQueueSize()) + .scriptCharset(getScriptCharset()).jsonParser(getJsonParser()).socketOptions(getSocketOptions()) + .sslOptions(getSslOptions()).suspendReconnectOnProtocolFailure(isSuspendReconnectOnProtocolFailure()) + .timeoutOptions(getTimeoutOptions()); return builder; } @@ -553,6 +569,10 @@ public DisconnectedBehavior getDisconnectedBehavior() { return disconnectedBehavior; } + public ReauthenticateBehavior getReauthenticateBehaviour() { + return reauthenticateBehavior; + } + /** * Predicate to identify commands as read-only. Defaults to {@link #DEFAULT_READ_ONLY_COMMANDS}. * @@ -684,6 +704,20 @@ public TimeoutOptions getTimeoutOptions() { return timeoutOptions; } + public enum ReauthenticateBehavior { + + /** + * This is the default behavior. The driver whenever needed will pull current credentials from the underlying + * CredentialsProvider. + */ + DEFAULT, + + /** + * CredentialsProvider might initiate re-authentication on its own. + */ + REAUTHENTICATE_ON_CREDENTIALS_CHANGE + } + /** * Behavior of connections in disconnected state. */ diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index 30bf55122..ca7d5e56d 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -19,33 +19,184 @@ */ package io.lettuce.core; +import io.lettuce.core.codec.StringCodec; import io.lettuce.core.event.EventBus; +import io.lettuce.core.event.connection.ReauthenticateEvent; +import io.lettuce.core.event.connection.ReauthenticateFailedEvent; +import io.lettuce.core.internal.LettuceAssert; +import io.lettuce.core.protocol.AsyncCommand; +import io.lettuce.core.protocol.Endpoint; import io.lettuce.core.protocol.ProtocolVersion; -import io.lettuce.core.pubsub.StatefulRedisPubSubConnection; +import io.lettuce.core.protocol.RedisCommand; import io.netty.util.internal.logging.InternalLogger; import io.netty.util.internal.logging.InternalLoggerFactory; +import reactor.core.Disposable; +import reactor.core.publisher.Flux; + +import java.nio.CharBuffer; +import java.util.concurrent.atomic.AtomicReference; /** - * Redis authentication handler. Internally used to authenticate a Redis connection. Internal API. + * Redis authentication handler. Internally used to authenticate a Redis connection. This class is part of the internal API. * * @author Ivo Gaydazhiev * @since 6.5.2 */ -class RedisAuthenticationHandler extends BaseRedisAuthenticationHandler> { +public class RedisAuthenticationHandler { + + private static final InternalLogger log = InternalLoggerFactory.getInstance(RedisAuthenticationHandler.class); + + private final RedisChannelWriter writer; + + private final ConnectionState state; + + private final RedisCommandBuilder commandBuilder = new RedisCommandBuilder<>(StringCodec.UTF8); + + private final RedisCredentialsProvider credentialsProvider; + + private final AtomicReference credentialsSubscription = new AtomicReference<>(); + + private final EventBus eventBus; + + private final Boolean isPubSubConnection; + + public RedisAuthenticationHandler(RedisChannelWriter writer, RedisCredentialsProvider credentialsProvider, + ConnectionState state, EventBus eventBus, Boolean isPubSubConnection) { + this.writer = writer; + this.state = state; + this.credentialsProvider = credentialsProvider; + this.eventBus = eventBus; + this.isPubSubConnection = isPubSubConnection; + } + + /** + * Subscribes to the provided `Flux` of credentials if the given `RedisCredentialsProvider` supports streaming credentials. + *

+ * This method subscribes to a stream of credentials provided by the `StreamingCredentialsProvider`. Each time new + * credentials are received, the client is reauthenticated. If the connection is not supported, the method returns without + * subscribing. + *

+ * The previous subscription, if any, is disposed of before setting the new subscription. + */ + public void subscribe() { + if (credentialsProvider == null) { + return; + } + + if (credentialsProvider instanceof StreamingCredentialsProvider) { + if (!isSupportedConnection()) { + return; + } + + Flux credentialsFlux = ((StreamingCredentialsProvider) credentialsProvider).credentials(); + + Disposable subscription = credentialsFlux.subscribe(this::onNext, this::onError, this::complete); + + Disposable oldSubscription = credentialsSubscription.getAndSet(subscription); + if (oldSubscription != null && !oldSubscription.isDisposed()) { + oldSubscription.dispose(); + } + } + } + + /** + * Unsubscribes from the current credentials stream. + */ + public void unsubscribe() { + Disposable subscription = credentialsSubscription.getAndSet(null); + if (subscription != null && !subscription.isDisposed()) { + subscription.dispose(); + } + } + + protected void complete() { + log.debug("Credentials stream completed"); + } - private static final InternalLogger logger = InternalLoggerFactory.getInstance(RedisAuthenticationHandler.class); + protected void onNext(RedisCredentials credentials) { + reauthenticate(credentials); + } + + protected void onError(Throwable e) { + log.error("Credentials renew failed.", e); + publishReauthFailedEvent(e); + } + + /** + * Performs re-authentication with the provided credentials. + * + * @param credentials the new credentials + */ + protected void reauthenticate(RedisCredentials credentials) { + CharSequence password = CharBuffer.wrap(credentials.getPassword()); + + AsyncCommand authCmd; + if (credentials.hasUsername()) { + authCmd = new AsyncCommand<>(commandBuilder.auth(credentials.getUsername(), password)); + } else { + authCmd = new AsyncCommand<>(commandBuilder.auth(password)); + } + + dispatchAuth(authCmd).thenRun(() -> { + publishReauthEvent(); + log.info("Re-authentication succeeded for endpoint {}.", getEpid()); + }).exceptionally(throwable -> { + publishReauthFailedEvent(throwable); + log.error("Re-authentication failed for endpoint {}.", getEpid(), throwable); + return null; + }); + } + + private AsyncCommand dispatchAuth(RedisCommand authCommand) { + AsyncCommand asyncCommand = new AsyncCommand<>(authCommand); + RedisCommand dispatched = writer.write(asyncCommand); + if (dispatched instanceof AsyncCommand) { + return (AsyncCommand) dispatched; + } + return asyncCommand; + } + + private void publishReauthEvent() { + eventBus.publish(new ReauthenticateEvent(getEpid())); + } - public RedisAuthenticationHandler(StatefulRedisConnectionImpl connection, EventBus eventBus) { - super(connection, eventBus); + private void publishReauthFailedEvent(Throwable throwable) { + eventBus.publish(new ReauthenticateFailedEvent(getEpid(), throwable)); } protected boolean isSupportedConnection() { - if (connection instanceof StatefulRedisPubSubConnection - && ProtocolVersion.RESP2 == connection.getConnectionState().getNegotiatedProtocolVersion()) { - logger.warn("Renewable credentials are not supported with RESP2 protocol on a pub/sub connection."); + if (isPubSubConnection && ProtocolVersion.RESP2 == state.getNegotiatedProtocolVersion()) { + log.warn("Renewable credentials are not supported with RESP2 protocol on a pub/sub connection."); return false; } return true; } + private String getEpid() { + if (writer instanceof Endpoint) { + return ((Endpoint) writer).getId(); + } + return "unknown"; + } + + public static boolean isSupported(ClientOptions clientOptions, RedisCredentialsProvider credentialsProvider) { + LettuceAssert.notNull(clientOptions, "ClientOptions must not be null"); + + if (credentialsProvider instanceof StreamingCredentialsProvider) { + + switch (clientOptions.getReauthenticateBehaviour()) { + case REAUTHENTICATE_ON_CREDENTIALS_CHANGE: + return true; + + case DEFAULT: + return false; + + default: + return false; + } + } else { + return false; + } + } + } diff --git a/src/main/java/io/lettuce/core/RedisClient.java b/src/main/java/io/lettuce/core/RedisClient.java index 4a2c3e7bd..b3eecd817 100644 --- a/src/main/java/io/lettuce/core/RedisClient.java +++ b/src/main/java/io/lettuce/core/RedisClient.java @@ -38,7 +38,6 @@ import io.lettuce.core.internal.ExceptionFactory; import io.lettuce.core.internal.Futures; import io.lettuce.core.internal.LettuceAssert; -import io.lettuce.core.json.JsonParser; import io.lettuce.core.masterreplica.MasterReplica; import io.lettuce.core.protocol.CommandExpiryWriter; import io.lettuce.core.protocol.CommandHandler; @@ -288,6 +287,12 @@ private ConnectionFuture> connectStandalone } StatefulRedisConnectionImpl connection = newStatefulRedisConnection(writer, endpoint, codec, timeout); + + if (RedisAuthenticationHandler.isSupported(getOptions(), redisURI.getCredentialsProvider())) { + connection.setAuthenticationHandler(new RedisAuthenticationHandler(writer, redisURI.getCredentialsProvider(), + connection.getConnectionState(), getResources().eventBus(), false)); + } + ConnectionFuture> future = connectStatefulAsync(connection, endpoint, redisURI, () -> new CommandHandler(getOptions(), getResources(), endpoint)); @@ -420,6 +425,11 @@ private ConnectionFuture> connectPubS StatefulRedisPubSubConnectionImpl connection = newStatefulRedisPubSubConnection(endpoint, writer, codec, timeout); + if (RedisAuthenticationHandler.isSupported(getOptions(), redisURI.getCredentialsProvider())) { + connection.setAuthenticationHandler(new RedisAuthenticationHandler(writer, redisURI.getCredentialsProvider(), + connection.getConnectionState(), getResources().eventBus(), true)); + } + ConnectionFuture> future = connectStatefulAsync(connection, endpoint, redisURI, () -> new PubSubCommandHandler<>(getOptions(), getResources(), codec, endpoint)); diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index ef084f398..b213936eb 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -67,7 +68,7 @@ public class StatefulRedisConnectionImpl extends RedisChannelHandler private final PushHandler pushHandler; - private final RedisAuthenticationHandler authHandler; + private final AtomicReference authenticationHandler = new AtomicReference<>(); private final Mono parser; @@ -106,8 +107,6 @@ public StatefulRedisConnectionImpl(RedisChannelWriter writer, PushHandler pushHa this.async = newRedisAsyncCommandsImpl(); this.sync = newRedisSyncCommandsImpl(); this.reactive = newRedisReactiveCommandsImpl(); - - this.authHandler = new RedisAuthenticationHandler(this, getResources().eventBus()); } public RedisCodec getCodec() { @@ -322,13 +321,27 @@ public ConnectionState getConnectionState() { @Override public void activated() { super.activated(); - authHandler.subscribe(state.getCredentialsProvider()); + RedisAuthenticationHandler currentHandler = authenticationHandler.get(); + if (currentHandler != null) { + currentHandler.subscribe(); + } } @Override public void deactivated() { - authHandler.unsubscribe(); + RedisAuthenticationHandler currentHandler = authenticationHandler.get(); + if (currentHandler != null) { + currentHandler.unsubscribe(); + } super.deactivated(); } + public void setAuthenticationHandler(RedisAuthenticationHandler authenticationHandler) { + RedisAuthenticationHandler currentHandler = this.authenticationHandler.getAndSet(authenticationHandler); + + if (currentHandler != null) { + currentHandler.unsubscribe(); + } + } + } diff --git a/src/main/java/io/lettuce/core/cluster/ClusterClientOptions.java b/src/main/java/io/lettuce/core/cluster/ClusterClientOptions.java index 11b90828f..b0f6cc018 100644 --- a/src/main/java/io/lettuce/core/cluster/ClusterClientOptions.java +++ b/src/main/java/io/lettuce/core/cluster/ClusterClientOptions.java @@ -218,6 +218,12 @@ public Builder disconnectedBehavior(DisconnectedBehavior disconnectedBehavior) { return this; } + @Override + public Builder reauthenticateBehavior(ReauthenticateBehavior reauthenticateBehavior) { + super.reauthenticateBehavior(reauthenticateBehavior); + return this; + } + /** * Number of maximal cluster redirects ({@literal -MOVED} and {@literal -ASK}) to follow in case a key was moved from * one node to another node. Defaults to {@literal 5}. See {@link ClusterClientOptions#DEFAULT_MAX_REDIRECTS}. @@ -355,12 +361,12 @@ public ClusterClientOptions.Builder mutate() { builder.autoReconnect(isAutoReconnect()).cancelCommandsOnReconnectFailure(isCancelCommandsOnReconnectFailure()) .decodeBufferPolicy(getDecodeBufferPolicy()).disconnectedBehavior(getDisconnectedBehavior()) - .maxRedirects(getMaxRedirects()).publishOnScheduler(isPublishOnScheduler()) - .pingBeforeActivateConnection(isPingBeforeActivateConnection()).protocolVersion(getConfiguredProtocolVersion()) - .readOnlyCommands(getReadOnlyCommands()).requestQueueSize(getRequestQueueSize()) - .scriptCharset(getScriptCharset()).socketOptions(getSocketOptions()).sslOptions(getSslOptions()) - .suspendReconnectOnProtocolFailure(isSuspendReconnectOnProtocolFailure()).timeoutOptions(getTimeoutOptions()) - .topologyRefreshOptions(getTopologyRefreshOptions()) + .reauthenticateBehavior(getReauthenticateBehaviour()).maxRedirects(getMaxRedirects()) + .publishOnScheduler(isPublishOnScheduler()).pingBeforeActivateConnection(isPingBeforeActivateConnection()) + .protocolVersion(getConfiguredProtocolVersion()).readOnlyCommands(getReadOnlyCommands()) + .requestQueueSize(getRequestQueueSize()).scriptCharset(getScriptCharset()).socketOptions(getSocketOptions()) + .sslOptions(getSslOptions()).suspendReconnectOnProtocolFailure(isSuspendReconnectOnProtocolFailure()) + .timeoutOptions(getTimeoutOptions()).topologyRefreshOptions(getTopologyRefreshOptions()) .validateClusterNodeMembership(isValidateClusterNodeMembership()).nodeFilter(getNodeFilter()); return builder; diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java b/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java deleted file mode 100644 index aae271985..000000000 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterAuthenticationHandler.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright 2019-Present, Redis Ltd. and Contributors - * All rights reserved. - * - * Licensed under the MIT License. - * - * This file contains contributions from third-party contributors - * licensed 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 - * - * https://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 io.lettuce.core.cluster; - -import io.lettuce.core.BaseRedisAuthenticationHandler; -import io.lettuce.core.cluster.pubsub.StatefulRedisClusterPubSubConnection; -import io.lettuce.core.event.EventBus; -import io.lettuce.core.protocol.ProtocolVersion; -import io.netty.util.internal.logging.InternalLogger; -import io.netty.util.internal.logging.InternalLoggerFactory; - -/** - * Redis Cluster authentication handler. Internally used to authenticate a Redis Cluster connection. Internal API. - * - *

- * - * @author Ivo Gaydajiev - * @since 6.5.2 - */ -class RedisClusterAuthenticationHandler extends BaseRedisAuthenticationHandler> { - - private static final InternalLogger logger = InternalLoggerFactory.getInstance(RedisClusterAuthenticationHandler.class); - - public RedisClusterAuthenticationHandler(StatefulRedisClusterConnectionImpl connection, EventBus eventBus) { - super(connection, eventBus); - } - - protected boolean isSupportedConnection() { - if (connection instanceof StatefulRedisClusterPubSubConnection - && ProtocolVersion.RESP2 == connection.getConnectionState().getNegotiatedProtocolVersion()) { - logger.warn("Renewable credentials are not supported with RESP2 protocol on a pub/sub connection."); - return false; - } - return true; - } - -} diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java index 577689cec..c3411bb40 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java @@ -556,6 +556,11 @@ ConnectionFuture> connectToNodeAsync(RedisC StatefulRedisConnectionImpl connection = newStatefulRedisConnection(writer, endpoint, codec, getFirstUri().getTimeout(), getClusterClientOptions().getJsonParser()); + if (RedisAuthenticationHandler.isSupported(getOptions(), getFirstUri().getCredentialsProvider())) { + connection.setAuthenticationHandler(new RedisAuthenticationHandler(writer, getFirstUri().getCredentialsProvider(), + connection.getConnectionState(), getResources().eventBus(), false)); + } + ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, () -> new CommandHandler(getClusterClientOptions(), getResources(), endpoint)); @@ -621,6 +626,11 @@ ConnectionFuture> connectPubSubToNode StatefulRedisPubSubConnectionImpl connection = new StatefulRedisPubSubConnectionImpl<>(endpoint, writer, codec, getFirstUri().getTimeout()); + if (RedisAuthenticationHandler.isSupported(getOptions(), getFirstUri().getCredentialsProvider())) { + connection.setAuthenticationHandler(new RedisAuthenticationHandler(writer, getFirstUri().getCredentialsProvider(), + connection.getConnectionState(), getResources().eventBus(), false)); + } + ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint)); diff --git a/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java b/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java index 7735a23b2..23a027315 100644 --- a/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java +++ b/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java @@ -89,8 +89,6 @@ public class StatefulRedisClusterConnectionImpl extends RedisChannelHandle private volatile Partitions partitions; - private final RedisClusterAuthenticationHandler authHandler; - /** * Initialize a new connection. * @@ -125,8 +123,6 @@ public StatefulRedisClusterConnectionImpl(RedisChannelWriter writer, ClusterPush this.async = newRedisAdvancedClusterAsyncCommandsImpl(); this.sync = newRedisAdvancedClusterCommandsImpl(); this.reactive = newRedisAdvancedClusterReactiveCommandsImpl(); - - this.authHandler = new RedisClusterAuthenticationHandler(this, getResources().eventBus()); } protected RedisAdvancedClusterReactiveCommandsImpl newRedisAdvancedClusterReactiveCommandsImpl() { @@ -234,12 +230,11 @@ public void activated() { super.activated(); async.clusterMyId().thenAccept(connectionState::setNodeId); - authHandler.subscribe(connectionState.getCredentialsProvider()); } @Override public void deactivated() { - authHandler.unsubscribe(); + super.deactivated(); } ClusterDistributionChannelWriter getClusterDistributionChannelWriter() { diff --git a/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java b/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java index 2d4c635e6..a2928cae5 100644 --- a/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java +++ b/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java @@ -6,7 +6,7 @@ * Interface for Connection authentication events * * @author Ivo Gaydajiev - * @since 3.4 + * @since 6.5.2 */ public interface AuthenticateEvent extends Event { diff --git a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java index 01ef7b853..badd948f3 100644 --- a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java +++ b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java @@ -90,6 +90,8 @@ void streamingCredentialProvider(RedisClient client) { TestCommandListener listener = new TestCommandListener(); client.addListener(listener); + client.setOptions(client.getOptions().mutate() + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.REAUTHENTICATE_ON_CREDENTIALS_CHANGE).build()); // Build RedisURI with streaming credentials provider MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); @@ -115,14 +117,8 @@ void streamingCredentialProvider(RedisClient client) { credentialsProvider.shutdown(); connection.close(); client.removeListener(listener); - } - - private boolean isAuthCommandWithCredentials(RedisCommand command, String username, char[] password) { - if (command.getType() == CommandType.AUTH) { - CommandArgs args = command.getArgs(); - return args.toCommandString().contains(username) && args.toCommandString().contains(String.valueOf(password)); - } - return false; + client.setOptions( + client.getOptions().mutate().reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.DEFAULT).build()); } static class TestCommandListener implements CommandListener { @@ -138,4 +134,12 @@ public void commandSucceeded(CommandSucceededEvent event) { } + private boolean isAuthCommandWithCredentials(RedisCommand command, String username, char[] password) { + if (command.getType() == CommandType.AUTH) { + CommandArgs args = command.getArgs(); + return args.toCommandString().contains(username) && args.toCommandString().contains(String.valueOf(password)); + } + return false; + } + } diff --git a/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java b/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java deleted file mode 100644 index 0b1a0e2f5..000000000 --- a/src/test/java/io/lettuce/core/BaseRedisAuthenticationHandlerTest.java +++ /dev/null @@ -1,96 +0,0 @@ -package io.lettuce.core; - -import io.lettuce.core.event.DefaultEventBus; -import io.lettuce.core.event.EventBus; -import io.lettuce.core.protocol.CommandType; -import io.lettuce.core.protocol.RedisCommand; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Sinks; -import reactor.core.scheduler.Schedulers; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -class BaseRedisAuthenticationHandlerTest { - - private BaseRedisAuthenticationHandler> handler; - - private RedisChannelHandler connection; - - private RedisChannelWriter channelWriter; - - @BeforeEach - void setUp() { - EventBus eventBus = new DefaultEventBus(Schedulers.immediate()); - connection = mock(RedisChannelHandler.class); - channelWriter = mock(RedisChannelWriter.class); - when(connection.getChannelWriter()).thenReturn(channelWriter); - handler = new BaseRedisAuthenticationHandler>(connection, eventBus) { - - }; - } - - @SuppressWarnings("unchecked") - @Test - void subscribeWithStreamingCredentialsProviderInvokesReauth() { - MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); - - // Subscribe to the provider - handler.subscribe(credentialsProvider); - credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); - - ArgumentCaptor> captor = ArgumentCaptor.forClass(RedisCommand.class); - verify(channelWriter).write(captor.capture()); - - RedisCommand capturedCommand = captor.getValue(); - assertThat(capturedCommand.getType()).isEqualTo(CommandType.AUTH); - assertThat(capturedCommand.getArgs().toCommandString()).contains("newuser"); - assertThat(capturedCommand.getArgs().toCommandString()).contains("newpassword"); - - credentialsProvider.shutdown(); - } - - @Test - void shouldHandleErrorInCredentialsStream() { - MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); - - // Subscribe to the provider and simulate an error - handler.subscribe(credentialsProvider); - credentialsProvider.tryEmitError(new RuntimeException("Test error")); - - verify(connection.getChannelWriter(), times(0)).write(any(RedisCommand.class)); // No command should be sent - - credentialsProvider.shutdown(); - } - - @Test - void shouldNotSubscribeIfConnectionIsNotSupported() { - EventBus eventBus = new DefaultEventBus(Schedulers.immediate()); - StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); - - BaseRedisAuthenticationHandler handler = new BaseRedisAuthenticationHandler>(connection, - eventBus) { - - @Override - protected boolean isSupportedConnection() { - // Simulate : Pub/Sub connections are not supported with RESP2 - return false; - } - - }; - - // Subscribe to the provider (it should not subscribe due to unsupported connection) - handler.subscribe(credentialsProvider); - - // Ensure credentials() was not called - verify(credentialsProvider, times(0)).credentials(); - } - -} diff --git a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java index 63a0d769b..28b7cedaf 100644 --- a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java +++ b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java @@ -1,39 +1,120 @@ package io.lettuce.core; +import io.lettuce.core.event.DefaultEventBus; import io.lettuce.core.event.EventBus; +import io.lettuce.core.event.connection.ReauthenticateFailedEvent; +import io.lettuce.core.protocol.CommandType; import io.lettuce.core.protocol.ProtocolVersion; -import io.lettuce.core.pubsub.StatefulRedisPubSubConnection; -import io.lettuce.core.pubsub.StatefulRedisPubSubConnectionImpl; +import io.lettuce.core.protocol.RedisCommand; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import reactor.core.scheduler.Schedulers; +import reactor.test.StepVerifier; +import java.time.Duration; + +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; 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 static org.mockito.Mockito.withSettings; public class RedisAuthenticationHandlerTest { + private RedisChannelWriter channelWriter; + + EventBus eventBus; + + ConnectionState connectionState; + + @BeforeEach + void setUp() { + eventBus = new DefaultEventBus(Schedulers.immediate()); + channelWriter = mock(RedisChannelWriter.class); + connectionState = mock(ConnectionState.class); + } + + @SuppressWarnings("unchecked") + @Test + void subscribeWithStreamingCredentialsProviderInvokesReauth() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelWriter, credentialsProvider, connectionState, + eventBus, false); + + // Subscribe to the provider + handler.subscribe(); + credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); + + ArgumentCaptor> captor = ArgumentCaptor.forClass(RedisCommand.class); + verify(channelWriter).write(captor.capture()); + + RedisCommand capturedCommand = captor.getValue(); + assertThat(capturedCommand.getType()).isEqualTo(CommandType.AUTH); + assertThat(capturedCommand.getArgs().toCommandString()).contains("newuser"); + assertThat(capturedCommand.getArgs().toCommandString()).contains("newpassword"); + + credentialsProvider.shutdown(); + } + + @Test + void shouldHandleErrorInCredentialsStream() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelWriter, credentialsProvider, connectionState, + eventBus, false); + + verify(channelWriter, times(0)).write(any(RedisCommand.class)); // No command should be sent + + // Verify the event was published + StepVerifier.create(eventBus.get()).then(() -> { + handler.subscribe(); + credentialsProvider.tryEmitError(new RuntimeException("Test error")); + }).expectNextMatches(event -> event instanceof ReauthenticateFailedEvent).thenCancel().verify(Duration.ofSeconds(1)); + + credentialsProvider.shutdown(); + } + + @Test + void shouldNotSubscribeIfConnectionIsNotSupported() { + StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); + + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelWriter, credentialsProvider, connectionState, + eventBus, true); + + // Subscribe to the provider (it should not subscribe due to unsupported connection) + handler.subscribe(); + + // Ensure credentials() was not called + verify(credentialsProvider, times(0)).credentials(); + } + @Test void testIsSupportedConnectionWithRESP2ProtocolOnPubSubConnection() { - StatefulRedisPubSubConnectionImpl connection = mock(StatefulRedisPubSubConnectionImpl.class, - withSettings().extraInterfaces(StatefulRedisPubSubConnection.class)); + RedisChannelWriter writer = mock(RedisChannelWriter.class); ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - when(connection.getConnectionState()).thenReturn(connectionState); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(EventBus.class)); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(writer, mock(RedisCredentialsProvider.class), + connectionState, mock(EventBus.class), true); assertFalse(handler.isSupportedConnection()); } @Test void testIsSupportedConnectionWithNonPubSubConnection() { - StatefulRedisConnectionImpl connection = mock(StatefulRedisConnectionImpl.class); + RedisChannelWriter writer = mock(RedisChannelWriter.class); ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - when(connection.getConnectionState()).thenReturn(connectionState); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(EventBus.class)); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(writer, mock(RedisCredentialsProvider.class), + connectionState, mock(EventBus.class), false); assertTrue(handler.isSupportedConnection()); } @@ -41,11 +122,12 @@ void testIsSupportedConnectionWithNonPubSubConnection() { @Test void testIsSupportedConnectionWithRESP3ProtocolOnPubSubConnection() { - StatefulRedisPubSubConnectionImpl connection = mock(StatefulRedisPubSubConnectionImpl.class); + RedisChannelWriter writer = mock(RedisChannelWriter.class); ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP3); - when(connection.getConnectionState()).thenReturn(connectionState); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(EventBus.class)); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(writer, mock(RedisCredentialsProvider.class), + connectionState, mock(EventBus.class), true); assertTrue(handler.isSupportedConnection()); } diff --git a/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java b/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java index d69882567..cedbe64e9 100644 --- a/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java +++ b/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java @@ -1,6 +1,7 @@ package io.lettuce.core.cluster; import io.lettuce.core.AclSetuserArgs; +import io.lettuce.core.ClientOptions; import io.lettuce.core.MyStreamingRedisCredentialsProvider; import io.lettuce.core.RedisCommandExecutionException; import io.lettuce.core.RedisURI; @@ -138,6 +139,11 @@ void nodeSelectionApiShouldWork() { @Test void shouldPerformNodeConnectionReauth() { + ClusterClientOptions origClientOptions = redisClient.getClusterClientOptions(); + origClientOptions.mutate() + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.REAUTHENTICATE_ON_CREDENTIALS_CHANGE).build(); + redisClient.setOptions(origClientOptions.mutate() + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.REAUTHENTICATE_ON_CREDENTIALS_CHANGE).build()); StatefulRedisClusterConnection connection = redisClient.connect(); connection.getPartitions().forEach( diff --git a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java index 102a00a55..bb59fe885 100644 --- a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java +++ b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java @@ -6,6 +6,7 @@ import java.time.Duration; import java.time.temporal.ChronoUnit; +import io.lettuce.core.ClientOptions; import io.lettuce.core.MyStreamingRedisCredentialsProvider; import io.lettuce.core.event.connection.AuthenticateEvent; import io.lettuce.core.event.connection.ReauthenticateEvent; @@ -50,11 +51,14 @@ void testConnectionEvents() { } @Test - void testReauthConnectionEvents() { + void testReauthenticateEvents() { MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); credentialsProvider.emitCredentials(TestSettings.username(), TestSettings.password().toString().toCharArray()); + RedisClient client = RedisClient.create(TestClientResources.get(), RedisURI.Builder.redis(host, port).withAuthentication(credentialsProvider).build()); + client.setOptions(ClientOptions.builder() + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.REAUTHENTICATE_ON_CREDENTIALS_CHANGE).build()); Flux publisher = client.getResources().eventBus().get() .filter(event -> event instanceof AuthenticateEvent).cast(AuthenticateEvent.class); @@ -66,7 +70,7 @@ void testReauthConnectionEvents() { .assertNext( event -> assertThat(event).asInstanceOf(InstanceOfAssertFactories.type(ReauthenticateFailedEvent.class)) .extracting(ReauthenticateFailedEvent::getEpId).isNotNull()) - .thenCancel().verify(Duration.of(5, ChronoUnit.SECONDS)); + .thenCancel().verify(Duration.of(1, ChronoUnit.SECONDS)); FastShutdown.shutdown(client); } From 779edca5c0b1a221f465945d927f35b253be9b1f Mon Sep 17 00:00:00 2001 From: ggivo Date: Sun, 8 Dec 2024 14:03:23 +0200 Subject: [PATCH 05/28] Client setting for enabling reauthentication - Moved Authentication handler to DefaultEndpoint - updated since 6.6.0 --- .../java/io/lettuce/core/ClientOptions.java | 27 +++++++++-- .../io/lettuce/core/ConnectionBuilder.java | 11 +++++ .../core/RedisAuthenticationHandler.java | 33 +++++-------- .../java/io/lettuce/core/RedisClient.java | 21 ++++----- .../core/StatefulRedisConnectionImpl.java | 18 ------- .../core/StreamingCredentialsProvider.java | 2 +- .../core/cluster/RedisClusterClient.java | 47 +++++++++---------- .../event/connection/AuthenticateEvent.java | 2 +- .../core/event/connection/JfrReauthEvent.java | 2 +- .../connection/JfrReauthFailedEvent.java | 2 +- .../event/connection/ReauthenticateEvent.java | 2 +- .../connection/ReauthenticateFailedEvent.java | 2 +- .../core/protocol/DefaultEndpoint.java | 14 ++++++ .../io/lettuce/core/protocol/Endpoint.java | 8 ++++ .../core/AuthenticationIntegrationTests.java | 2 +- .../MyStreamingRedisCredentialsProvider.java | 2 +- ...gCredentialsProviderlIntegrationTests.java | 4 +- ...ectionEventsTriggeredIntegrationTests.java | 2 +- 18 files changed, 109 insertions(+), 92 deletions(-) diff --git a/src/main/java/io/lettuce/core/ClientOptions.java b/src/main/java/io/lettuce/core/ClientOptions.java index eca9a3789..f8327a4c9 100644 --- a/src/main/java/io/lettuce/core/ClientOptions.java +++ b/src/main/java/io/lettuce/core/ClientOptions.java @@ -704,18 +704,37 @@ public TimeoutOptions getTimeoutOptions() { return timeoutOptions; } + /** + * Defines the re-authentication behavior of the Redis client in relation to the {@link CredentialsProvider}. + */s public enum ReauthenticateBehavior { /** - * This is the default behavior. The driver whenever needed will pull current credentials from the underlying - * CredentialsProvider. + * This is the default behavior. The client will fetch current credentials from the underlying + * {@link RedisCredentialsProvider} only when required. + * + *

No re-authentication is performed automatically when new credentials are emitted by the + * {@link StreamingCredentialsProvider} .

+ * + *

This behavior is suitable for use cases with static credentials or where explicit reconnection + * is required to change credentials.

*/ DEFAULT, /** - * CredentialsProvider might initiate re-authentication on its own. + * Automatically triggers re-authentication whenever new credentials are emitted by the + * {@link StreamingCredentialsProvider} or any other credentials manager. + * + *

When enabled, the client subscribes to the credentials stream provided by the + * {@link StreamingCredentialsProvider} (or other compatible sources) and issues an {@code AUTH} + * command to the Redis server each time new credentials are received. This behavior supports + * dynamic credential scenarios, such as token-based authentication, or credential rotation where credentials + * are refreshed periodically to maintain access.

+ * + *

Note: {@code AUTH} commands issued as part of this behavior may interleave with user-submitted + * commands, as the client performs re-authentication independently of user command flow.

*/ - REAUTHENTICATE_ON_CREDENTIALS_CHANGE + ON_NEW_CREDENTIALS } /** diff --git a/src/main/java/io/lettuce/core/ConnectionBuilder.java b/src/main/java/io/lettuce/core/ConnectionBuilder.java index 4bb3127b8..444c13f90 100644 --- a/src/main/java/io/lettuce/core/ConnectionBuilder.java +++ b/src/main/java/io/lettuce/core/ConnectionBuilder.java @@ -113,6 +113,17 @@ public void apply(RedisURI redisURI) { bootstrap.attr(REDIS_URI, redisURI.toString()); } + public void registerAuthenticationHandler(RedisCredentialsProvider credentialsProvider, ConnectionState state, + Boolean isPubSubConnection) { + LettuceAssert.assertState(endpoint != null, "Endpoint must be set"); + LettuceAssert.assertState(connection != null, "Connection must be set"); + LettuceAssert.assertState(clientResources != null, "ClientResources must be set"); + + RedisAuthenticationHandler authenticationHandler = new RedisAuthenticationHandler(connection.getChannelWriter(), + credentialsProvider, state, clientResources.eventBus(), isPubSubConnection); + endpoint.registerAuthenticationHandler(authenticationHandler); + } + protected List buildHandlers() { LettuceAssert.assertState(channelGroup != null, "ChannelGroup must be set"); diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index ca7d5e56d..e8cb1d5a0 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -40,7 +40,7 @@ * Redis authentication handler. Internally used to authenticate a Redis connection. This class is part of the internal API. * * @author Ivo Gaydazhiev - * @since 6.5.2 + * @since 6.6.0 */ public class RedisAuthenticationHandler { @@ -70,13 +70,10 @@ public RedisAuthenticationHandler(RedisChannelWriter writer, RedisCredentialsPro } /** - * Subscribes to the provided `Flux` of credentials if the given `RedisCredentialsProvider` supports streaming credentials. + * This method subscribes to a stream of credentials provided by the `StreamingCredentialsProvider`. *

- * This method subscribes to a stream of credentials provided by the `StreamingCredentialsProvider`. Each time new - * credentials are received, the client is reauthenticated. If the connection is not supported, the method returns without - * subscribing. - *

- * The previous subscription, if any, is disposed of before setting the new subscription. + * Each time new credentials are received, the client is re-authenticated. The previous subscription, if any, is disposed of + * before setting the new subscription. */ public void subscribe() { if (credentialsProvider == null) { @@ -179,23 +176,17 @@ private String getEpid() { return "unknown"; } - public static boolean isSupported(ClientOptions clientOptions, RedisCredentialsProvider credentialsProvider) { + public static boolean isSupported(ClientOptions clientOptions) { LettuceAssert.notNull(clientOptions, "ClientOptions must not be null"); + switch (clientOptions.getReauthenticateBehaviour()) { + case ON_NEW_CREDENTIALS: + return true; - if (credentialsProvider instanceof StreamingCredentialsProvider) { - - switch (clientOptions.getReauthenticateBehaviour()) { - case REAUTHENTICATE_ON_CREDENTIALS_CHANGE: - return true; - - case DEFAULT: - return false; + case DEFAULT: + return false; - default: - return false; - } - } else { - return false; + default: + return false; } } diff --git a/src/main/java/io/lettuce/core/RedisClient.java b/src/main/java/io/lettuce/core/RedisClient.java index b3eecd817..26801d949 100644 --- a/src/main/java/io/lettuce/core/RedisClient.java +++ b/src/main/java/io/lettuce/core/RedisClient.java @@ -288,13 +288,8 @@ private ConnectionFuture> connectStandalone StatefulRedisConnectionImpl connection = newStatefulRedisConnection(writer, endpoint, codec, timeout); - if (RedisAuthenticationHandler.isSupported(getOptions(), redisURI.getCredentialsProvider())) { - connection.setAuthenticationHandler(new RedisAuthenticationHandler(writer, redisURI.getCredentialsProvider(), - connection.getConnectionState(), getResources().eventBus(), false)); - } - ConnectionFuture> future = connectStatefulAsync(connection, endpoint, redisURI, - () -> new CommandHandler(getOptions(), getResources(), endpoint)); + () -> new CommandHandler(getOptions(), getResources(), endpoint), false); future.whenComplete((channelHandler, throwable) -> { @@ -308,7 +303,7 @@ private ConnectionFuture> connectStandalone @SuppressWarnings("unchecked") private ConnectionFuture connectStatefulAsync(StatefulRedisConnectionImpl connection, Endpoint endpoint, - RedisURI redisURI, Supplier commandHandlerSupplier) { + RedisURI redisURI, Supplier commandHandlerSupplier, Boolean isPubSub) { ConnectionBuilder connectionBuilder; if (redisURI.isSsl()) { @@ -331,6 +326,11 @@ private ConnectionFuture connectStatefulAsync(StatefulRedisConnecti connectionBuilder(getSocketAddressSupplier(redisURI), connectionBuilder, connection.getConnectionEvents(), redisURI); connectionBuilder.connectionInitializer(createHandshake(state)); + if (RedisAuthenticationHandler.isSupported(getOptions())) { + connectionBuilder.registerAuthenticationHandler(redisURI.getCredentialsProvider(), connection.getConnectionState(), + isPubSub); + } + ConnectionFuture> future = initializeChannelAsync(connectionBuilder); return future.thenApply(channelHandler -> (S) connection); @@ -425,13 +425,8 @@ private ConnectionFuture> connectPubS StatefulRedisPubSubConnectionImpl connection = newStatefulRedisPubSubConnection(endpoint, writer, codec, timeout); - if (RedisAuthenticationHandler.isSupported(getOptions(), redisURI.getCredentialsProvider())) { - connection.setAuthenticationHandler(new RedisAuthenticationHandler(writer, redisURI.getCredentialsProvider(), - connection.getConnectionState(), getResources().eventBus(), true)); - } - ConnectionFuture> future = connectStatefulAsync(connection, endpoint, redisURI, - () -> new PubSubCommandHandler<>(getOptions(), getResources(), codec, endpoint)); + () -> new PubSubCommandHandler<>(getOptions(), getResources(), codec, endpoint), true); return future.whenComplete((conn, throwable) -> { diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index b213936eb..3304cfdd4 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -68,8 +68,6 @@ public class StatefulRedisConnectionImpl extends RedisChannelHandler private final PushHandler pushHandler; - private final AtomicReference authenticationHandler = new AtomicReference<>(); - private final Mono parser; protected MultiOutput multi; @@ -321,27 +319,11 @@ public ConnectionState getConnectionState() { @Override public void activated() { super.activated(); - RedisAuthenticationHandler currentHandler = authenticationHandler.get(); - if (currentHandler != null) { - currentHandler.subscribe(); - } } @Override public void deactivated() { - RedisAuthenticationHandler currentHandler = authenticationHandler.get(); - if (currentHandler != null) { - currentHandler.unsubscribe(); - } super.deactivated(); } - public void setAuthenticationHandler(RedisAuthenticationHandler authenticationHandler) { - RedisAuthenticationHandler currentHandler = this.authenticationHandler.getAndSet(authenticationHandler); - - if (currentHandler != null) { - currentHandler.unsubscribe(); - } - } - } diff --git a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java index dba9302c7..999d35683 100644 --- a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java +++ b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java @@ -7,7 +7,7 @@ * new credentials are received. * * @author Ivo Gaydajiev - * @since 6.5.2 + * @since 6.6.0 */ public interface StreamingCredentialsProvider extends RedisCredentialsProvider { diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java index c3411bb40..f384cbda0 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java @@ -556,14 +556,9 @@ ConnectionFuture> connectToNodeAsync(RedisC StatefulRedisConnectionImpl connection = newStatefulRedisConnection(writer, endpoint, codec, getFirstUri().getTimeout(), getClusterClientOptions().getJsonParser()); - if (RedisAuthenticationHandler.isSupported(getOptions(), getFirstUri().getCredentialsProvider())) { - connection.setAuthenticationHandler(new RedisAuthenticationHandler(writer, getFirstUri().getCredentialsProvider(), - connection.getConnectionState(), getResources().eventBus(), false)); - } - ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, - () -> new CommandHandler(getClusterClientOptions(), getResources(), endpoint)); + () -> new CommandHandler(getClusterClientOptions(), getResources(), endpoint), false); return connectionFuture.whenComplete((conn, throwable) -> { if (throwable != null) { @@ -626,14 +621,9 @@ ConnectionFuture> connectPubSubToNode StatefulRedisPubSubConnectionImpl connection = new StatefulRedisPubSubConnectionImpl<>(endpoint, writer, codec, getFirstUri().getTimeout()); - if (RedisAuthenticationHandler.isSupported(getOptions(), getFirstUri().getCredentialsProvider())) { - connection.setAuthenticationHandler(new RedisAuthenticationHandler(writer, getFirstUri().getCredentialsProvider(), - connection.getConnectionState(), getResources().eventBus(), false)); - } - ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, - () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint)); + () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint), true); return connectionFuture.whenComplete((conn, throwable) -> { if (throwable != null) { connection.closeAsync(); @@ -689,11 +679,11 @@ private CompletableFuture> connectCl Mono socketAddressSupplier = getSocketAddressSupplier(connection::getPartitions, TopologyComparators::sortByClientCount); Mono> connectionMono = Mono - .defer(() -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier)); + .defer(() -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier, false)); for (int i = 1; i < getConnectionAttempts(); i++) { connectionMono = connectionMono - .onErrorResume(t -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier)); + .onErrorResume(t -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier, false)); } return connectionMono @@ -723,19 +713,20 @@ protected StatefulRedisClusterConnectionImpl newStatefulRedisCluste } private Mono connect(Mono socketAddressSupplier, DefaultEndpoint endpoint, - StatefulRedisClusterConnectionImpl connection, Supplier commandHandlerSupplier) { + StatefulRedisClusterConnectionImpl connection, Supplier commandHandlerSupplier, + Boolean isPubSub) { ConnectionFuture future = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, - commandHandlerSupplier); + commandHandlerSupplier, isPubSub); return Mono.fromCompletionStage(future).doOnError(t -> logger.warn(t.getMessage())); } private Mono connect(Mono socketAddressSupplier, DefaultEndpoint endpoint, - StatefulRedisConnectionImpl connection, Supplier commandHandlerSupplier) { + StatefulRedisConnectionImpl connection, Supplier commandHandlerSupplier, Boolean isPubSub) { ConnectionFuture future = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, - commandHandlerSupplier); + commandHandlerSupplier, isPubSub); return Mono.fromCompletionStage(future).doOnError(t -> logger.warn(t.getMessage())); } @@ -788,11 +779,11 @@ private CompletableFuture> con Mono socketAddressSupplier = getSocketAddressSupplier(connection::getPartitions, TopologyComparators::sortByClientCount); Mono> connectionMono = Mono - .defer(() -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier)); + .defer(() -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier, true)); for (int i = 1; i < getConnectionAttempts(); i++) { connectionMono = connectionMono - .onErrorResume(t -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier)); + .onErrorResume(t -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier, true)); } return connectionMono @@ -812,10 +803,10 @@ private int getConnectionAttempts() { @SuppressWarnings("unchecked") private , S> ConnectionFuture connectStatefulAsync(T connection, DefaultEndpoint endpoint, RedisURI connectionSettings, Mono socketAddressSupplier, - Supplier commandHandlerSupplier) { + Supplier commandHandlerSupplier, Boolean isPubSub) { ConnectionBuilder connectionBuilder = createConnectionBuilder(connection, connection.getConnectionState(), endpoint, - connectionSettings, socketAddressSupplier, commandHandlerSupplier); + connectionSettings, socketAddressSupplier, commandHandlerSupplier, isPubSub); ConnectionFuture> future = initializeChannelAsync(connectionBuilder); @@ -829,10 +820,10 @@ private , S> Connection @SuppressWarnings("unchecked") private , S> ConnectionFuture connectStatefulAsync(T connection, DefaultEndpoint endpoint, RedisURI connectionSettings, Mono socketAddressSupplier, - Supplier commandHandlerSupplier) { + Supplier commandHandlerSupplier, Boolean isPubSub) { ConnectionBuilder connectionBuilder = createConnectionBuilder(connection, connection.getConnectionState(), endpoint, - connectionSettings, socketAddressSupplier, commandHandlerSupplier); + connectionSettings, socketAddressSupplier, commandHandlerSupplier, isPubSub); ConnectionFuture> future = initializeChannelAsync(connectionBuilder); @@ -841,7 +832,7 @@ private , S> ConnectionFuture< private ConnectionBuilder createConnectionBuilder(RedisChannelHandler connection, ConnectionState state, DefaultEndpoint endpoint, RedisURI connectionSettings, Mono socketAddressSupplier, - Supplier commandHandlerSupplier) { + Supplier commandHandlerSupplier, Boolean isPubSub) { ConnectionBuilder connectionBuilder; if (connectionSettings.isSsl()) { @@ -853,6 +844,7 @@ private ConnectionBuilder createConnectionBuilder(RedisChannelHandler ConnectionBuilder createConnectionBuilder(RedisChannelHandler connection = redisClient.connect(); connection.getPartitions().forEach( diff --git a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java index bb59fe885..732e5dcda 100644 --- a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java +++ b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java @@ -58,7 +58,7 @@ void testReauthenticateEvents() { RedisClient client = RedisClient.create(TestClientResources.get(), RedisURI.Builder.redis(host, port).withAuthentication(credentialsProvider).build()); client.setOptions(ClientOptions.builder() - .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.REAUTHENTICATE_ON_CREDENTIALS_CHANGE).build()); + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); Flux publisher = client.getResources().eventBus().get() .filter(event -> event instanceof AuthenticateEvent).cast(AuthenticateEvent.class); From 21bf696c07231bf204e774dd7780d1e7997c8b06 Mon Sep 17 00:00:00 2001 From: ggivo Date: Sun, 8 Dec 2024 14:13:55 +0200 Subject: [PATCH 06/28] formating --- .../java/io/lettuce/core/ClientOptions.java | 31 ++++++++++++------- ...gCredentialsProviderlIntegrationTests.java | 3 +- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/main/java/io/lettuce/core/ClientOptions.java b/src/main/java/io/lettuce/core/ClientOptions.java index f8327a4c9..0c34eec77 100644 --- a/src/main/java/io/lettuce/core/ClientOptions.java +++ b/src/main/java/io/lettuce/core/ClientOptions.java @@ -706,18 +706,22 @@ public TimeoutOptions getTimeoutOptions() { /** * Defines the re-authentication behavior of the Redis client in relation to the {@link CredentialsProvider}. - */s + */ public enum ReauthenticateBehavior { /** * This is the default behavior. The client will fetch current credentials from the underlying * {@link RedisCredentialsProvider} only when required. * - *

No re-authentication is performed automatically when new credentials are emitted by the - * {@link StreamingCredentialsProvider} .

+ *

+ * No re-authentication is performed automatically when new credentials are emitted by the + * {@link StreamingCredentialsProvider} . + *

* - *

This behavior is suitable for use cases with static credentials or where explicit reconnection - * is required to change credentials.

+ *

+ * This behavior is suitable for use cases with static credentials or where explicit reconnection is required to change + * credentials. + *

*/ DEFAULT, @@ -725,14 +729,17 @@ public enum ReauthenticateBehavior { * Automatically triggers re-authentication whenever new credentials are emitted by the * {@link StreamingCredentialsProvider} or any other credentials manager. * - *

When enabled, the client subscribes to the credentials stream provided by the - * {@link StreamingCredentialsProvider} (or other compatible sources) and issues an {@code AUTH} - * command to the Redis server each time new credentials are received. This behavior supports - * dynamic credential scenarios, such as token-based authentication, or credential rotation where credentials - * are refreshed periodically to maintain access.

+ *

+ * When enabled, the client subscribes to the credentials stream provided by the {@link StreamingCredentialsProvider} + * (or other compatible sources) and issues an {@code AUTH} command to the Redis server each time new credentials are + * received. This behavior supports dynamic credential scenarios, such as token-based authentication, or credential + * rotation where credentials are refreshed periodically to maintain access. + *

* - *

Note: {@code AUTH} commands issued as part of this behavior may interleave with user-submitted - * commands, as the client performs re-authentication independently of user command flow.

+ *

+ * Note: {@code AUTH} commands issued as part of this behavior may interleave with user-submitted commands, as the + * client performs re-authentication independently of user command flow. + *

*/ ON_NEW_CREDENTIALS } diff --git a/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java b/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java index aeb7dd49b..3c8f20a96 100644 --- a/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java +++ b/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java @@ -140,8 +140,7 @@ void nodeSelectionApiShouldWork() { @Test void shouldPerformNodeConnectionReauth() { ClusterClientOptions origClientOptions = redisClient.getClusterClientOptions(); - origClientOptions.mutate() - .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); + origClientOptions.mutate().reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); redisClient.setOptions(origClientOptions.mutate() .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); From f3aef04a3f8b0ac1f34b4cc03466d78af4bbd398 Mon Sep 17 00:00:00 2001 From: ggivo Date: Sun, 8 Dec 2024 15:06:52 +0200 Subject: [PATCH 07/28] resolve conflict with main --- src/main/java/io/lettuce/core/ClientOptions.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/main/java/io/lettuce/core/ClientOptions.java b/src/main/java/io/lettuce/core/ClientOptions.java index 56362be8b..b700f6cae 100644 --- a/src/main/java/io/lettuce/core/ClientOptions.java +++ b/src/main/java/io/lettuce/core/ClientOptions.java @@ -230,7 +230,6 @@ public static class Builder { private boolean useHashIndexedQueue = DEFAULT_USE_HASH_INDEX_QUEUE; - protected Builder() { } @@ -521,12 +520,12 @@ public ClientOptions.Builder mutate() { builder.autoReconnect(isAutoReconnect()).cancelCommandsOnReconnectFailure(isCancelCommandsOnReconnectFailure()) .decodeBufferPolicy(getDecodeBufferPolicy()).disconnectedBehavior(getDisconnectedBehavior()) - .reauthenticateBehavior(getReauthenticateBehaviour()).readOnlyCommands(getReadOnlyCommands()) - .publishOnScheduler(isPublishOnScheduler()).pingBeforeActivateConnection(isPingBeforeActivateConnection()) - .protocolVersion(getConfiguredProtocolVersion()).requestQueueSize(getRequestQueueSize()) - .scriptCharset(getScriptCharset()).jsonParser(getJsonParser()).socketOptions(getSocketOptions()) - .sslOptions(getSslOptions()).suspendReconnectOnProtocolFailure(isSuspendReconnectOnProtocolFailure()) - .timeoutOptions(getTimeoutOptions()); + .reauthenticateBehavior(getReauthenticateBehaviour()) + .readOnlyCommands(getReadOnlyCommands()).publishOnScheduler(isPublishOnScheduler()) + .pingBeforeActivateConnection(isPingBeforeActivateConnection()).protocolVersion(getConfiguredProtocolVersion()) + .requestQueueSize(getRequestQueueSize()).scriptCharset(getScriptCharset()).jsonParser(getJsonParser()) + .socketOptions(getSocketOptions()).sslOptions(getSslOptions()) + .suspendReconnectOnProtocolFailure(isSuspendReconnectOnProtocolFailure()).timeoutOptions(getTimeoutOptions()); return builder; } @@ -764,7 +763,8 @@ public enum ReauthenticateBehavior { */ ON_NEW_CREDENTIALS } - + + /** * Whether we should use hash indexed queue, which provides O(1) remove(Object) * * @return if hash indexed queue should be used From 631d4209fbb39b87ac7db00c01b8fc5c41e2d1b4 Mon Sep 17 00:00:00 2001 From: ggivo Date: Sun, 8 Dec 2024 15:12:15 +0200 Subject: [PATCH 08/28] format --- src/main/java/io/lettuce/core/ClientOptions.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/java/io/lettuce/core/ClientOptions.java b/src/main/java/io/lettuce/core/ClientOptions.java index b700f6cae..8bc2c8109 100644 --- a/src/main/java/io/lettuce/core/ClientOptions.java +++ b/src/main/java/io/lettuce/core/ClientOptions.java @@ -520,12 +520,12 @@ public ClientOptions.Builder mutate() { builder.autoReconnect(isAutoReconnect()).cancelCommandsOnReconnectFailure(isCancelCommandsOnReconnectFailure()) .decodeBufferPolicy(getDecodeBufferPolicy()).disconnectedBehavior(getDisconnectedBehavior()) - .reauthenticateBehavior(getReauthenticateBehaviour()) - .readOnlyCommands(getReadOnlyCommands()).publishOnScheduler(isPublishOnScheduler()) - .pingBeforeActivateConnection(isPingBeforeActivateConnection()).protocolVersion(getConfiguredProtocolVersion()) - .requestQueueSize(getRequestQueueSize()).scriptCharset(getScriptCharset()).jsonParser(getJsonParser()) - .socketOptions(getSocketOptions()).sslOptions(getSslOptions()) - .suspendReconnectOnProtocolFailure(isSuspendReconnectOnProtocolFailure()).timeoutOptions(getTimeoutOptions()); + .reauthenticateBehavior(getReauthenticateBehaviour()).readOnlyCommands(getReadOnlyCommands()) + .publishOnScheduler(isPublishOnScheduler()).pingBeforeActivateConnection(isPingBeforeActivateConnection()) + .protocolVersion(getConfiguredProtocolVersion()).requestQueueSize(getRequestQueueSize()) + .scriptCharset(getScriptCharset()).jsonParser(getJsonParser()).socketOptions(getSocketOptions()) + .sslOptions(getSslOptions()).suspendReconnectOnProtocolFailure(isSuspendReconnectOnProtocolFailure()) + .timeoutOptions(getTimeoutOptions()); return builder; } From 6f460224a21bcd49521783ec85555847ec96f679 Mon Sep 17 00:00:00 2001 From: ggivo Date: Tue, 10 Dec 2024 07:13:12 +0200 Subject: [PATCH 09/28] dispath using connection handler --- .../io/lettuce/core/ConnectionBuilder.java | 4 +-- .../core/RedisAuthenticationHandler.java | 18 +++++----- .../core/StatefulRedisConnectionImpl.java | 1 - .../core/RedisAuthenticationHandlerTest.java | 36 +++++++++---------- 4 files changed, 29 insertions(+), 30 deletions(-) diff --git a/src/main/java/io/lettuce/core/ConnectionBuilder.java b/src/main/java/io/lettuce/core/ConnectionBuilder.java index 444c13f90..2a397b6e1 100644 --- a/src/main/java/io/lettuce/core/ConnectionBuilder.java +++ b/src/main/java/io/lettuce/core/ConnectionBuilder.java @@ -119,8 +119,8 @@ public void registerAuthenticationHandler(RedisCredentialsProvider credentialsPr LettuceAssert.assertState(connection != null, "Connection must be set"); LettuceAssert.assertState(clientResources != null, "ClientResources must be set"); - RedisAuthenticationHandler authenticationHandler = new RedisAuthenticationHandler(connection.getChannelWriter(), - credentialsProvider, state, clientResources.eventBus(), isPubSubConnection); + RedisAuthenticationHandler authenticationHandler = new RedisAuthenticationHandler(connection, credentialsProvider, + state, clientResources.eventBus(), isPubSubConnection); endpoint.registerAuthenticationHandler(authenticationHandler); } diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index e8cb1d5a0..cd3cf6239 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -46,7 +46,7 @@ public class RedisAuthenticationHandler { private static final InternalLogger log = InternalLoggerFactory.getInstance(RedisAuthenticationHandler.class); - private final RedisChannelWriter writer; + private final RedisChannelHandler connection; private final ConnectionState state; @@ -60,9 +60,9 @@ public class RedisAuthenticationHandler { private final Boolean isPubSubConnection; - public RedisAuthenticationHandler(RedisChannelWriter writer, RedisCredentialsProvider credentialsProvider, + public RedisAuthenticationHandler(RedisChannelHandler connection, RedisCredentialsProvider credentialsProvider, ConnectionState state, EventBus eventBus, Boolean isPubSubConnection) { - this.writer = writer; + this.connection = connection; this.state = state; this.credentialsProvider = credentialsProvider; this.eventBus = eventBus; @@ -144,11 +144,11 @@ protected void reauthenticate(RedisCredentials credentials) { }); } - private AsyncCommand dispatchAuth(RedisCommand authCommand) { - AsyncCommand asyncCommand = new AsyncCommand<>(authCommand); - RedisCommand dispatched = writer.write(asyncCommand); + private AsyncCommand dispatchAuth(RedisCommand authCommand) { + AsyncCommand asyncCommand = new AsyncCommand<>(authCommand); + RedisCommand dispatched = connection.dispatch(asyncCommand); if (dispatched instanceof AsyncCommand) { - return (AsyncCommand) dispatched; + return (AsyncCommand) dispatched; } return asyncCommand; } @@ -170,8 +170,8 @@ protected boolean isSupportedConnection() { } private String getEpid() { - if (writer instanceof Endpoint) { - return ((Endpoint) writer).getId(); + if (connection.getChannelWriter() instanceof Endpoint) { + return ((Endpoint) connection.getChannelWriter()).getId(); } return "unknown"; } diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index 3304cfdd4..853f9ee24 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -26,7 +26,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.stream.Collectors; diff --git a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java index 28b7cedaf..98b5b0102 100644 --- a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java +++ b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java @@ -25,7 +25,7 @@ public class RedisAuthenticationHandlerTest { - private RedisChannelWriter channelWriter; + private RedisChannelHandler channelHandler; EventBus eventBus; @@ -34,7 +34,7 @@ public class RedisAuthenticationHandlerTest { @BeforeEach void setUp() { eventBus = new DefaultEventBus(Schedulers.immediate()); - channelWriter = mock(RedisChannelWriter.class); + channelHandler = mock(RedisChannelHandler.class); connectionState = mock(ConnectionState.class); } @@ -43,17 +43,17 @@ void setUp() { void subscribeWithStreamingCredentialsProviderInvokesReauth() { MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelWriter, credentialsProvider, connectionState, - eventBus, false); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelHandler, credentialsProvider, + connectionState, eventBus, false); // Subscribe to the provider handler.subscribe(); credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); - ArgumentCaptor> captor = ArgumentCaptor.forClass(RedisCommand.class); - verify(channelWriter).write(captor.capture()); + ArgumentCaptor> captor = ArgumentCaptor.forClass(RedisCommand.class); + verify(channelHandler).dispatch(captor.capture()); - RedisCommand capturedCommand = captor.getValue(); + RedisCommand capturedCommand = captor.getValue(); assertThat(capturedCommand.getType()).isEqualTo(CommandType.AUTH); assertThat(capturedCommand.getArgs().toCommandString()).contains("newuser"); assertThat(capturedCommand.getArgs().toCommandString()).contains("newpassword"); @@ -65,10 +65,10 @@ void subscribeWithStreamingCredentialsProviderInvokesReauth() { void shouldHandleErrorInCredentialsStream() { MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelWriter, credentialsProvider, connectionState, - eventBus, false); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelHandler, credentialsProvider, + connectionState, eventBus, false); - verify(channelWriter, times(0)).write(any(RedisCommand.class)); // No command should be sent + verify(channelHandler, times(0)).dispatch(any(RedisCommand.class)); // No command should be sent // Verify the event was published StepVerifier.create(eventBus.get()).then(() -> { @@ -84,8 +84,8 @@ void shouldNotSubscribeIfConnectionIsNotSupported() { StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelWriter, credentialsProvider, connectionState, - eventBus, true); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelHandler, credentialsProvider, + connectionState, eventBus, true); // Subscribe to the provider (it should not subscribe due to unsupported connection) handler.subscribe(); @@ -96,12 +96,12 @@ void shouldNotSubscribeIfConnectionIsNotSupported() { @Test void testIsSupportedConnectionWithRESP2ProtocolOnPubSubConnection() { - RedisChannelWriter writer = mock(RedisChannelWriter.class); + RedisChannelHandler connection = mock(RedisChannelHandler.class); ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(writer, mock(RedisCredentialsProvider.class), + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), connectionState, mock(EventBus.class), true); assertFalse(handler.isSupportedConnection()); @@ -109,11 +109,11 @@ void testIsSupportedConnectionWithRESP2ProtocolOnPubSubConnection() { @Test void testIsSupportedConnectionWithNonPubSubConnection() { - RedisChannelWriter writer = mock(RedisChannelWriter.class); + RedisChannelHandler connection = mock(RedisChannelHandler.class); ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(writer, mock(RedisCredentialsProvider.class), + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), connectionState, mock(EventBus.class), false); assertTrue(handler.isSupportedConnection()); @@ -122,11 +122,11 @@ void testIsSupportedConnectionWithNonPubSubConnection() { @Test void testIsSupportedConnectionWithRESP3ProtocolOnPubSubConnection() { - RedisChannelWriter writer = mock(RedisChannelWriter.class); + RedisChannelHandler connection = mock(RedisChannelHandler.class); ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP3); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(writer, mock(RedisCredentialsProvider.class), + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), connectionState, mock(EventBus.class), true); assertTrue(handler.isSupportedConnection()); From b32f84c2890f89dd44edaf0e913fde988836f4b2 Mon Sep 17 00:00:00 2001 From: ggivo Date: Wed, 11 Dec 2024 13:44:41 +0200 Subject: [PATCH 10/28] Support multi with re-auth Defer the re-auth operation in case there is on-going multi Tx in lettuce need to be externally synchronised when used in multithreaded env. Since re-auth happens from different thread we need to make sure it does not happen while there is ongoing transaction. --- .../io/lettuce/core/ConnectionBuilder.java | 11 -- .../core/RedisAuthenticationHandler.java | 58 ++------- .../java/io/lettuce/core/RedisClient.java | 10 +- .../core/StatefulRedisConnectionImpl.java | 113 +++++++++++++++++- .../core/api/StatefulRedisConnection.java | 1 + .../core/cluster/RedisClusterClient.java | 46 +++---- .../connection/ReauthenticateFailedEvent.java | 5 + .../core/protocol/DefaultEndpoint.java | 14 --- .../io/lettuce/core/protocol/Endpoint.java | 8 -- .../core/RedisAuthenticationHandlerTest.java | 50 ++++---- ...ectionEventsTriggeredIntegrationTests.java | 14 ++- 11 files changed, 187 insertions(+), 143 deletions(-) diff --git a/src/main/java/io/lettuce/core/ConnectionBuilder.java b/src/main/java/io/lettuce/core/ConnectionBuilder.java index 2a397b6e1..4bb3127b8 100644 --- a/src/main/java/io/lettuce/core/ConnectionBuilder.java +++ b/src/main/java/io/lettuce/core/ConnectionBuilder.java @@ -113,17 +113,6 @@ public void apply(RedisURI redisURI) { bootstrap.attr(REDIS_URI, redisURI.toString()); } - public void registerAuthenticationHandler(RedisCredentialsProvider credentialsProvider, ConnectionState state, - Boolean isPubSubConnection) { - LettuceAssert.assertState(endpoint != null, "Endpoint must be set"); - LettuceAssert.assertState(connection != null, "Connection must be set"); - LettuceAssert.assertState(clientResources != null, "ClientResources must be set"); - - RedisAuthenticationHandler authenticationHandler = new RedisAuthenticationHandler(connection, credentialsProvider, - state, clientResources.eventBus(), isPubSubConnection); - endpoint.registerAuthenticationHandler(authenticationHandler); - } - protected List buildHandlers() { LettuceAssert.assertState(channelGroup != null, "ChannelGroup must be set"); diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index cd3cf6239..1bf329e86 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -46,26 +46,18 @@ public class RedisAuthenticationHandler { private static final InternalLogger log = InternalLoggerFactory.getInstance(RedisAuthenticationHandler.class); - private final RedisChannelHandler connection; - - private final ConnectionState state; - - private final RedisCommandBuilder commandBuilder = new RedisCommandBuilder<>(StringCodec.UTF8); + private final StatefulRedisConnectionImpl connection; private final RedisCredentialsProvider credentialsProvider; private final AtomicReference credentialsSubscription = new AtomicReference<>(); - private final EventBus eventBus; - private final Boolean isPubSubConnection; - public RedisAuthenticationHandler(RedisChannelHandler connection, RedisCredentialsProvider credentialsProvider, - ConnectionState state, EventBus eventBus, Boolean isPubSubConnection) { + public RedisAuthenticationHandler(StatefulRedisConnectionImpl connection, + RedisCredentialsProvider credentialsProvider, Boolean isPubSubConnection) { this.connection = connection; - this.state = state; this.credentialsProvider = credentialsProvider; - this.eventBus = eventBus; this.isPubSubConnection = isPubSubConnection; } @@ -125,55 +117,19 @@ protected void onError(Throwable e) { * @param credentials the new credentials */ protected void reauthenticate(RedisCredentials credentials) { - CharSequence password = CharBuffer.wrap(credentials.getPassword()); - - AsyncCommand authCmd; - if (credentials.hasUsername()) { - authCmd = new AsyncCommand<>(commandBuilder.auth(credentials.getUsername(), password)); - } else { - authCmd = new AsyncCommand<>(commandBuilder.auth(password)); - } - - dispatchAuth(authCmd).thenRun(() -> { - publishReauthEvent(); - log.info("Re-authentication succeeded for endpoint {}.", getEpid()); - }).exceptionally(throwable -> { - publishReauthFailedEvent(throwable); - log.error("Re-authentication failed for endpoint {}.", getEpid(), throwable); - return null; - }); - } - - private AsyncCommand dispatchAuth(RedisCommand authCommand) { - AsyncCommand asyncCommand = new AsyncCommand<>(authCommand); - RedisCommand dispatched = connection.dispatch(asyncCommand); - if (dispatched instanceof AsyncCommand) { - return (AsyncCommand) dispatched; - } - return asyncCommand; - } - - private void publishReauthEvent() { - eventBus.publish(new ReauthenticateEvent(getEpid())); - } - - private void publishReauthFailedEvent(Throwable throwable) { - eventBus.publish(new ReauthenticateFailedEvent(getEpid(), throwable)); + connection.setCredentials(credentials); } protected boolean isSupportedConnection() { - if (isPubSubConnection && ProtocolVersion.RESP2 == state.getNegotiatedProtocolVersion()) { + if (isPubSubConnection && ProtocolVersion.RESP2 == connection.getConnectionState().getNegotiatedProtocolVersion()) { log.warn("Renewable credentials are not supported with RESP2 protocol on a pub/sub connection."); return false; } return true; } - private String getEpid() { - if (connection.getChannelWriter() instanceof Endpoint) { - return ((Endpoint) connection.getChannelWriter()).getId(); - } - return "unknown"; + private void publishReauthFailedEvent(Throwable throwable) { + connection.getResources().eventBus().publish(new ReauthenticateFailedEvent(throwable)); } public static boolean isSupported(ClientOptions clientOptions) { diff --git a/src/main/java/io/lettuce/core/RedisClient.java b/src/main/java/io/lettuce/core/RedisClient.java index 26801d949..d2d04e307 100644 --- a/src/main/java/io/lettuce/core/RedisClient.java +++ b/src/main/java/io/lettuce/core/RedisClient.java @@ -317,7 +317,10 @@ private ConnectionFuture connectStatefulAsync(StatefulRedisConnecti ConnectionState state = connection.getConnectionState(); state.apply(redisURI); state.setDb(redisURI.getDatabase()); - + if (RedisAuthenticationHandler.isSupported(getOptions())) { + connection.setAuthenticationHandler( + new RedisAuthenticationHandler(connection, redisURI.getCredentialsProvider(), isPubSub)); + } connectionBuilder.connection(connection); connectionBuilder.clientOptions(getOptions()); connectionBuilder.clientResources(getResources()); @@ -326,11 +329,6 @@ private ConnectionFuture connectStatefulAsync(StatefulRedisConnecti connectionBuilder(getSocketAddressSupplier(redisURI), connectionBuilder, connection.getConnectionEvents(), redisURI); connectionBuilder.connectionInitializer(createHandshake(state)); - if (RedisAuthenticationHandler.isSupported(getOptions())) { - connectionBuilder.registerAuthenticationHandler(redisURI.getCredentialsProvider(), connection.getConnectionState(), - isPubSub); - } - ConnectionFuture> future = initializeChannelAsync(connectionBuilder); return future.thenApply(channelHandler -> (S) connection); diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index 853f9ee24..de78c26d0 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -22,10 +22,14 @@ import static io.lettuce.core.ClientOptions.DEFAULT_JSON_PARSER; import static io.lettuce.core.protocol.CommandType.*; +import java.nio.CharBuffer; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -37,10 +41,14 @@ import io.lettuce.core.cluster.api.sync.RedisClusterCommands; import io.lettuce.core.codec.RedisCodec; import io.lettuce.core.codec.StringCodec; +import io.lettuce.core.event.connection.ReauthenticateEvent; +import io.lettuce.core.event.connection.ReauthenticateFailedEvent; import io.lettuce.core.json.JsonParser; import io.lettuce.core.output.MultiOutput; import io.lettuce.core.output.StatusOutput; import io.lettuce.core.protocol.*; +import io.netty.util.internal.logging.InternalLogger; +import io.netty.util.internal.logging.InternalLoggerFactory; import reactor.core.publisher.Mono; /** @@ -55,6 +63,8 @@ */ public class StatefulRedisConnectionImpl extends RedisChannelHandler implements StatefulRedisConnection { + private static final InternalLogger logger = InternalLoggerFactory.getInstance(StatefulRedisConnectionImpl.class); + protected final RedisCodec codec; protected final RedisCommands sync; @@ -71,6 +81,14 @@ public class StatefulRedisConnectionImpl extends RedisChannelHandler protected MultiOutput multi; + private RedisAuthenticationHandler authHandler; + + private AtomicReference credentialsRef = new AtomicReference<>(); + + private final ReentrantLock reAuthSafety = new ReentrantLock(); + + private AtomicBoolean inTransaction = new AtomicBoolean(false); + /** * Initialize a new connection. * @@ -181,7 +199,13 @@ public boolean isMulti() { public RedisCommand dispatch(RedisCommand command) { RedisCommand toSend = preProcessCommand(command); - return super.dispatch(toSend); + RedisCommand result = super.dispatch(toSend); + if (toSend.getType() == EXEC || toSend.getType() == DISCARD) { + inTransaction.set(false); + setCredentials(credentialsRef.getAndSet(null)); + } + + return result; } @Override @@ -189,12 +213,24 @@ public RedisCommand dispatch(RedisCommand command) { List> sentCommands = new ArrayList<>(commands.size()); - commands.forEach(o -> { + boolean transactionComplete = false; + for (RedisCommand o : commands) { RedisCommand command = preProcessCommand(o); sentCommands.add(command); - }); + if (command.getType() == EXEC) { + transactionComplete = true; + } + if (command.getType() == MULTI || command.getType() == DISCARD) { + transactionComplete = false; + } + } - return super.dispatch(sentCommands); + Collection> result = super.dispatch(sentCommands); + if (transactionComplete) { + inTransaction.set(false); + setCredentials(credentialsRef.getAndSet(null)); + } + return result; } // TODO [tihomir.mateev] Refactor to include as part of the Command interface @@ -273,12 +309,20 @@ protected RedisCommand preProcessCommand(RedisCommand comm if (commandType.equals(MULTI.name())) { + reAuthSafety.lock(); + try { + inTransaction.set(true); + } finally { + reAuthSafety.unlock(); + } multi = (multi == null ? new MultiOutput<>(codec) : multi); if (command instanceof CompleteableCommand) { ((CompleteableCommand) command).onComplete((ignored, e) -> { if (e != null) { multi = null; + inTransaction.set(false); + setCredentials(credentialsRef.getAndSet(null)); } }); } @@ -318,11 +362,72 @@ public ConnectionState getConnectionState() { @Override public void activated() { super.activated(); + if (authHandler != null) { + authHandler.subscribe(); + } } @Override public void deactivated() { + if (authHandler != null) { + authHandler.unsubscribe(); + } super.deactivated(); } + public void setAuthenticationHandler(RedisAuthenticationHandler handler) { + authHandler = handler; + } + + public void setCredentials(RedisCredentials credentials) { + if (credentials == null) { + return; + } + reAuthSafety.lock(); + try { + credentialsRef.set(credentials); + if (!inTransaction.get()) { + dispatchAuthCommand(credentialsRef.getAndSet(null)); + } + } finally { + reAuthSafety.unlock(); + } + } + + private void dispatchAuthCommand(RedisCredentials credentials) { + if (credentials == null) { + return; + } + + RedisFuture auth; + if (credentials.getUsername() != null) { + auth = async().auth(credentials.getUsername(), CharBuffer.wrap(credentials.getPassword())); + } else { + auth = async().auth(CharBuffer.wrap(credentials.getPassword())); + } + auth.thenRun(() -> { + publishReauthEvent(); + logger.info("Re-authentication succeeded {}.", getEpid()); + }).exceptionally(throwable -> { + publishReauthFailedEvent(throwable); + logger.error("Re-authentication failed {}.", getEpid(), throwable); + return null; + }); + } + + private void publishReauthEvent() { + getResources().eventBus().publish(new ReauthenticateEvent(getEpid())); + } + + private void publishReauthFailedEvent(Throwable throwable) { + getResources().eventBus().publish(new ReauthenticateFailedEvent(getEpid(), throwable)); + } + + private String getEpid() { + if (getChannelWriter() instanceof Endpoint) { + return ((Endpoint) getChannelWriter()).getId(); + } + return ""; + } + } diff --git a/src/main/java/io/lettuce/core/api/StatefulRedisConnection.java b/src/main/java/io/lettuce/core/api/StatefulRedisConnection.java index 1be962ba3..7cac57887 100644 --- a/src/main/java/io/lettuce/core/api/StatefulRedisConnection.java +++ b/src/main/java/io/lettuce/core/api/StatefulRedisConnection.java @@ -1,5 +1,6 @@ package io.lettuce.core.api; +import io.lettuce.core.RedisCredentials; import io.lettuce.core.api.async.RedisAsyncCommands; import io.lettuce.core.api.push.PushListener; import io.lettuce.core.api.reactive.RedisReactiveCommands; diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java index f384cbda0..422dcbb15 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java @@ -558,7 +558,12 @@ ConnectionFuture> connectToNodeAsync(RedisC ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, - () -> new CommandHandler(getClusterClientOptions(), getResources(), endpoint), false); + () -> new CommandHandler(getClusterClientOptions(), getResources(), endpoint)); + + if (RedisAuthenticationHandler.isSupported(getOptions())) { + connection.setAuthenticationHandler( + new RedisAuthenticationHandler(connection, getFirstUri().getCredentialsProvider(), false)); + } return connectionFuture.whenComplete((conn, throwable) -> { if (throwable != null) { @@ -623,7 +628,13 @@ ConnectionFuture> connectPubSubToNode ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, - () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint), true); + () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint)); + + if (RedisAuthenticationHandler.isSupported(getOptions())) { + connection.setAuthenticationHandler( + new RedisAuthenticationHandler(connection, getFirstUri().getCredentialsProvider(), true)); + } + return connectionFuture.whenComplete((conn, throwable) -> { if (throwable != null) { connection.closeAsync(); @@ -679,11 +690,11 @@ private CompletableFuture> connectCl Mono socketAddressSupplier = getSocketAddressSupplier(connection::getPartitions, TopologyComparators::sortByClientCount); Mono> connectionMono = Mono - .defer(() -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier, false)); + .defer(() -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier)); for (int i = 1; i < getConnectionAttempts(); i++) { connectionMono = connectionMono - .onErrorResume(t -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier, false)); + .onErrorResume(t -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier)); } return connectionMono @@ -713,20 +724,19 @@ protected StatefulRedisClusterConnectionImpl newStatefulRedisCluste } private Mono connect(Mono socketAddressSupplier, DefaultEndpoint endpoint, - StatefulRedisClusterConnectionImpl connection, Supplier commandHandlerSupplier, - Boolean isPubSub) { + StatefulRedisClusterConnectionImpl connection, Supplier commandHandlerSupplier) { ConnectionFuture future = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, - commandHandlerSupplier, isPubSub); + commandHandlerSupplier); return Mono.fromCompletionStage(future).doOnError(t -> logger.warn(t.getMessage())); } private Mono connect(Mono socketAddressSupplier, DefaultEndpoint endpoint, - StatefulRedisConnectionImpl connection, Supplier commandHandlerSupplier, Boolean isPubSub) { + StatefulRedisConnectionImpl connection, Supplier commandHandlerSupplier) { ConnectionFuture future = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, - commandHandlerSupplier, isPubSub); + commandHandlerSupplier); return Mono.fromCompletionStage(future).doOnError(t -> logger.warn(t.getMessage())); } @@ -779,11 +789,11 @@ private CompletableFuture> con Mono socketAddressSupplier = getSocketAddressSupplier(connection::getPartitions, TopologyComparators::sortByClientCount); Mono> connectionMono = Mono - .defer(() -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier, true)); + .defer(() -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier)); for (int i = 1; i < getConnectionAttempts(); i++) { connectionMono = connectionMono - .onErrorResume(t -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier, true)); + .onErrorResume(t -> connect(socketAddressSupplier, endpoint, connection, commandHandlerSupplier)); } return connectionMono @@ -803,10 +813,10 @@ private int getConnectionAttempts() { @SuppressWarnings("unchecked") private , S> ConnectionFuture connectStatefulAsync(T connection, DefaultEndpoint endpoint, RedisURI connectionSettings, Mono socketAddressSupplier, - Supplier commandHandlerSupplier, Boolean isPubSub) { + Supplier commandHandlerSupplier) { ConnectionBuilder connectionBuilder = createConnectionBuilder(connection, connection.getConnectionState(), endpoint, - connectionSettings, socketAddressSupplier, commandHandlerSupplier, isPubSub); + connectionSettings, socketAddressSupplier, commandHandlerSupplier); ConnectionFuture> future = initializeChannelAsync(connectionBuilder); @@ -820,10 +830,10 @@ private , S> Connection @SuppressWarnings("unchecked") private , S> ConnectionFuture connectStatefulAsync(T connection, DefaultEndpoint endpoint, RedisURI connectionSettings, Mono socketAddressSupplier, - Supplier commandHandlerSupplier, Boolean isPubSub) { + Supplier commandHandlerSupplier) { ConnectionBuilder connectionBuilder = createConnectionBuilder(connection, connection.getConnectionState(), endpoint, - connectionSettings, socketAddressSupplier, commandHandlerSupplier, isPubSub); + connectionSettings, socketAddressSupplier, commandHandlerSupplier); ConnectionFuture> future = initializeChannelAsync(connectionBuilder); @@ -832,7 +842,7 @@ private , S> ConnectionFuture< private ConnectionBuilder createConnectionBuilder(RedisChannelHandler connection, ConnectionState state, DefaultEndpoint endpoint, RedisURI connectionSettings, Mono socketAddressSupplier, - Supplier commandHandlerSupplier, Boolean isPubSub) { + Supplier commandHandlerSupplier) { ConnectionBuilder connectionBuilder; if (connectionSettings.isSsl()) { @@ -854,10 +864,6 @@ private ConnectionBuilder createConnectionBuilder(RedisChannelHandler channelHandler; + private StatefulRedisConnectionImpl connection; + + ClientResources resources; EventBus eventBus; @@ -34,8 +38,13 @@ public class RedisAuthenticationHandlerTest { @BeforeEach void setUp() { eventBus = new DefaultEventBus(Schedulers.immediate()); - channelHandler = mock(RedisChannelHandler.class); + connection = mock(StatefulRedisConnectionImpl.class); + resources = mock(ClientResources.class); + when(resources.eventBus()).thenReturn(eventBus); + connectionState = mock(ConnectionState.class); + when(connection.getResources()).thenReturn(resources); + when(connection.getConnectionState()).thenReturn(connectionState); } @SuppressWarnings("unchecked") @@ -43,20 +52,18 @@ void setUp() { void subscribeWithStreamingCredentialsProviderInvokesReauth() { MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelHandler, credentialsProvider, - connectionState, eventBus, false); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, credentialsProvider, false); // Subscribe to the provider handler.subscribe(); credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); - ArgumentCaptor> captor = ArgumentCaptor.forClass(RedisCommand.class); - verify(channelHandler).dispatch(captor.capture()); + ArgumentCaptor captor = ArgumentCaptor.forClass(RedisCredentials.class); + verify(connection).setCredentials(captor.capture()); - RedisCommand capturedCommand = captor.getValue(); - assertThat(capturedCommand.getType()).isEqualTo(CommandType.AUTH); - assertThat(capturedCommand.getArgs().toCommandString()).contains("newuser"); - assertThat(capturedCommand.getArgs().toCommandString()).contains("newpassword"); + RedisCredentials credentials = captor.getValue(); + assertThat(credentials.getUsername()).isEqualTo("newuser"); + assertThat(credentials.getPassword()).isEqualTo("newpassword".toCharArray()); credentialsProvider.shutdown(); } @@ -65,10 +72,9 @@ void subscribeWithStreamingCredentialsProviderInvokesReauth() { void shouldHandleErrorInCredentialsStream() { MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelHandler, credentialsProvider, - connectionState, eventBus, false); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, credentialsProvider, false); - verify(channelHandler, times(0)).dispatch(any(RedisCommand.class)); // No command should be sent + verify(connection, times(0)).dispatch(any(RedisCommand.class)); // No command should be sent // Verify the event was published StepVerifier.create(eventBus.get()).then(() -> { @@ -82,10 +88,9 @@ void shouldHandleErrorInCredentialsStream() { @Test void shouldNotSubscribeIfConnectionIsNotSupported() { StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); - when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(channelHandler, credentialsProvider, - connectionState, eventBus, true); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, credentialsProvider, true); // Subscribe to the provider (it should not subscribe due to unsupported connection) handler.subscribe(); @@ -96,25 +101,22 @@ void shouldNotSubscribeIfConnectionIsNotSupported() { @Test void testIsSupportedConnectionWithRESP2ProtocolOnPubSubConnection() { - RedisChannelHandler connection = mock(RedisChannelHandler.class); - ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), - connectionState, mock(EventBus.class), true); + true); assertFalse(handler.isSupportedConnection()); } @Test void testIsSupportedConnectionWithNonPubSubConnection() { - RedisChannelHandler connection = mock(RedisChannelHandler.class); - ConnectionState connectionState = mock(ConnectionState.class); + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), - connectionState, mock(EventBus.class), false); + false); assertTrue(handler.isSupportedConnection()); } @@ -122,12 +124,10 @@ void testIsSupportedConnectionWithNonPubSubConnection() { @Test void testIsSupportedConnectionWithRESP3ProtocolOnPubSubConnection() { - RedisChannelHandler connection = mock(RedisChannelHandler.class); - ConnectionState connectionState = mock(ConnectionState.class); when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP3); RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), - connectionState, mock(EventBus.class), true); + true); assertTrue(handler.isSupportedConnection()); } diff --git a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java index 732e5dcda..28dab306b 100644 --- a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java +++ b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java @@ -11,12 +11,14 @@ import io.lettuce.core.event.connection.AuthenticateEvent; import io.lettuce.core.event.connection.ReauthenticateEvent; import io.lettuce.core.event.connection.ReauthenticateFailedEvent; +import io.lettuce.test.LettuceExtension; import io.lettuce.test.WithPassword; import io.lettuce.test.settings.TestSettings; import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import reactor.core.publisher.Flux; import reactor.test.StepVerifier; import io.lettuce.core.RedisClient; @@ -26,11 +28,14 @@ import io.lettuce.test.resource.FastShutdown; import io.lettuce.test.resource.TestClientResources; +import javax.inject.Inject; + /** * @author Mark Paluch * @author Ivo Gaydajiev */ @Tag(INTEGRATION_TEST) +@ExtendWith(LettuceExtension.class) class ConnectionEventsTriggeredIntegrationTests extends TestSupport { @Test @@ -52,25 +57,26 @@ void testConnectionEvents() { @Test void testReauthenticateEvents() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); credentialsProvider.emitCredentials(TestSettings.username(), TestSettings.password().toString().toCharArray()); - RedisClient client = RedisClient.create(TestClientResources.get(), - RedisURI.Builder.redis(host, port).withAuthentication(credentialsProvider).build()); + RedisClient client = RedisClient.create(RedisURI.create(TestSettings.host(), TestSettings.port())); client.setOptions(ClientOptions.builder() .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); + RedisURI uri = RedisURI.Builder.redis(host, port).withAuthentication(credentialsProvider).build(); Flux publisher = client.getResources().eventBus().get() .filter(event -> event instanceof AuthenticateEvent).cast(AuthenticateEvent.class); - StepVerifier.create(publisher).then(() -> WithPassword.run(client, () -> client.connect().close())) + WithPassword.run(client, () -> StepVerifier.create(publisher).then(() -> client.connect(uri)) .assertNext(event -> assertThat(event).asInstanceOf(InstanceOfAssertFactories.type(ReauthenticateEvent.class)) .extracting(ReauthenticateEvent::getEpId).isNotNull()) .then(() -> credentialsProvider.emitCredentials(TestSettings.username(), "invalid".toCharArray())) .assertNext( event -> assertThat(event).asInstanceOf(InstanceOfAssertFactories.type(ReauthenticateFailedEvent.class)) .extracting(ReauthenticateFailedEvent::getEpId).isNotNull()) - .thenCancel().verify(Duration.of(1, ChronoUnit.SECONDS)); + .thenCancel().verify(Duration.of(1, ChronoUnit.SECONDS))); FastShutdown.shutdown(client); } From 7eaaf6b3249be37bd1a7d0a5a81ca47aad689361 Mon Sep 17 00:00:00 2001 From: ggivo Date: Thu, 12 Dec 2024 11:10:02 +0200 Subject: [PATCH 11/28] Fix EndpointId missing in events --- .../core/StatefulRedisConnectionImpl.java | 47 ++++++++++++------- .../core/cluster/RedisClusterClient.java | 20 ++++---- 2 files changed, 42 insertions(+), 25 deletions(-) diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index de78c26d0..038d23a96 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -355,6 +355,21 @@ public void setClientName(String clientName) { dispatch((RedisCommand) async); } + public void setCredentials(RedisCredentials credentials) { + if (credentials == null) { + return; + } + reAuthSafety.lock(); + try { + credentialsRef.set(credentials); + if (!inTransaction.get()) { + dispatchAuthCommand(credentialsRef.getAndSet(null)); + } + } finally { + reAuthSafety.unlock(); + } + } + public ConnectionState getConnectionState() { return state; } @@ -379,20 +394,6 @@ public void setAuthenticationHandler(RedisAuthenticationHandler handler) { authHandler = handler; } - public void setCredentials(RedisCredentials credentials) { - if (credentials == null) { - return; - } - reAuthSafety.lock(); - try { - credentialsRef.set(credentials); - if (!inTransaction.get()) { - dispatchAuthCommand(credentialsRef.getAndSet(null)); - } - } finally { - reAuthSafety.unlock(); - } - } private void dispatchAuthCommand(RedisCredentials credentials) { if (credentials == null) { @@ -424,10 +425,22 @@ private void publishReauthFailedEvent(Throwable throwable) { } private String getEpid() { - if (getChannelWriter() instanceof Endpoint) { - return ((Endpoint) getChannelWriter()).getId(); + RedisChannelWriter writer = getChannelWriter(); + while (!(writer instanceof Endpoint)) { + + if (writer instanceof CommandListenerWriter) { + writer = ((CommandListenerWriter) writer).getDelegate(); + continue; + } + + if (writer instanceof CommandExpiryWriter) { + writer = ((CommandExpiryWriter) writer).getDelegate(); + continue; + } + return null; } - return ""; + + return ((Endpoint) writer).getId(); } } diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java index 422dcbb15..0a08389c8 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java @@ -556,15 +556,17 @@ ConnectionFuture> connectToNodeAsync(RedisC StatefulRedisConnectionImpl connection = newStatefulRedisConnection(writer, endpoint, codec, getFirstUri().getTimeout(), getClusterClientOptions().getJsonParser()); - ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, - getFirstUri(), socketAddressSupplier, - () -> new CommandHandler(getClusterClientOptions(), getResources(), endpoint)); - if (RedisAuthenticationHandler.isSupported(getOptions())) { connection.setAuthenticationHandler( new RedisAuthenticationHandler(connection, getFirstUri().getCredentialsProvider(), false)); } + ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, + getFirstUri(), socketAddressSupplier, + () -> new CommandHandler(getClusterClientOptions(), getResources(), endpoint)); + + + return connectionFuture.whenComplete((conn, throwable) -> { if (throwable != null) { connection.closeAsync(); @@ -626,15 +628,17 @@ ConnectionFuture> connectPubSubToNode StatefulRedisPubSubConnectionImpl connection = new StatefulRedisPubSubConnectionImpl<>(endpoint, writer, codec, getFirstUri().getTimeout()); - ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, - getFirstUri(), socketAddressSupplier, - () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint)); - if (RedisAuthenticationHandler.isSupported(getOptions())) { connection.setAuthenticationHandler( new RedisAuthenticationHandler(connection, getFirstUri().getCredentialsProvider(), true)); } + ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, + getFirstUri(), socketAddressSupplier, + () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint)); + + + return connectionFuture.whenComplete((conn, throwable) -> { if (throwable != null) { connection.closeAsync(); From 086ccf36880c4ef74df1e065a1276f07e72e00b6 Mon Sep 17 00:00:00 2001 From: ggivo Date: Thu, 12 Dec 2024 11:44:06 +0200 Subject: [PATCH 12/28] format --- .../java/io/lettuce/core/StatefulRedisConnectionImpl.java | 1 - src/main/java/io/lettuce/core/cluster/RedisClusterClient.java | 4 ---- 2 files changed, 5 deletions(-) diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index 038d23a96..57d164dc2 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -394,7 +394,6 @@ public void setAuthenticationHandler(RedisAuthenticationHandler handler) { authHandler = handler; } - private void dispatchAuthCommand(RedisCredentials credentials) { if (credentials == null) { return; diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java index 0a08389c8..ef8690a2b 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java @@ -565,8 +565,6 @@ ConnectionFuture> connectToNodeAsync(RedisC getFirstUri(), socketAddressSupplier, () -> new CommandHandler(getClusterClientOptions(), getResources(), endpoint)); - - return connectionFuture.whenComplete((conn, throwable) -> { if (throwable != null) { connection.closeAsync(); @@ -637,8 +635,6 @@ ConnectionFuture> connectPubSubToNode getFirstUri(), socketAddressSupplier, () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint)); - - return connectionFuture.whenComplete((conn, throwable) -> { if (throwable != null) { connection.closeAsync(); From 61158f27d92ab4926cbcfe161deda02c29bb5b0c Mon Sep 17 00:00:00 2001 From: ggivo Date: Fri, 13 Dec 2024 09:28:29 +0200 Subject: [PATCH 13/28] Add unit tests for setCredenatials --- .../core/StatefulRedisConnectionImpl.java | 33 +++- .../StatefulRedisConnectionImplUnitTests.java | 142 ++++++++++++++++++ 2 files changed, 169 insertions(+), 6 deletions(-) create mode 100644 src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index 57d164dc2..c85191a1e 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -22,7 +22,6 @@ import static io.lettuce.core.ClientOptions.DEFAULT_JSON_PARSER; import static io.lettuce.core.protocol.CommandType.*; -import java.nio.CharBuffer; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -355,6 +354,29 @@ public void setClientName(String clientName) { dispatch((RedisCommand) async); } + /** + * Authenticates the current connection using the provided credentials. + *

+ * Unlike using dispatch of {@link RedisAsyncCommands#auth}, this method defers the {@code AUTH} command if the connection is within an active + * transaction. The authentication command will only be dispatched after the enclosing {@code DISCARD} or {@code EXEC} + * command is executed, ensuring that authentication does not interfere with ongoing transactions. + *

+ * + * @param credentials the {@link RedisCredentials} to authenticate the connection. If {@code null}, no action is performed. + * + *

+ * Behavior: + *

    + *
  • If the provided credentials are {@code null}, the method exits immediately.
  • + *
  • If a transaction is active (as indicated by {@code inTransaction}), the {@code AUTH} command is not dispatched + * immediately but deferred until the transaction ends.
  • + *
  • If no transaction is active, the {@code AUTH} command is dispatched immediately using the provided + * credentials.
  • + *
+ *

+ * + * @see RedisAsyncCommands#auth + */ public void setCredentials(RedisCredentials credentials) { if (credentials == null) { return; @@ -363,7 +385,7 @@ public void setCredentials(RedisCredentials credentials) { try { credentialsRef.set(credentials); if (!inTransaction.get()) { - dispatchAuthCommand(credentialsRef.getAndSet(null)); + dispatchAuth(credentialsRef.getAndSet(null)); } } finally { reAuthSafety.unlock(); @@ -394,16 +416,16 @@ public void setAuthenticationHandler(RedisAuthenticationHandler handler) { authHandler = handler; } - private void dispatchAuthCommand(RedisCredentials credentials) { + protected void dispatchAuth(RedisCredentials credentials) { if (credentials == null) { return; } RedisFuture auth; if (credentials.getUsername() != null) { - auth = async().auth(credentials.getUsername(), CharBuffer.wrap(credentials.getPassword())); + auth = async().auth(credentials.getUsername(), String.valueOf(credentials.getPassword())); } else { - auth = async().auth(CharBuffer.wrap(credentials.getPassword())); + auth = async().auth(String.valueOf(credentials.getPassword())); } auth.thenRun(() -> { publishReauthEvent(); @@ -441,5 +463,4 @@ private String getEpid() { return ((Endpoint) writer).getId(); } - } diff --git a/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java b/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java new file mode 100644 index 000000000..245eef09b --- /dev/null +++ b/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java @@ -0,0 +1,142 @@ +package io.lettuce.core; + +import io.lettuce.core.codec.StringCodec; +import io.lettuce.core.protocol.AsyncCommand; +import io.lettuce.core.protocol.PushHandler; +import io.lettuce.core.resource.ClientResources; +import io.lettuce.core.tracing.Tracing; +import io.lettuce.test.ReflectionTestUtils; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +import java.lang.reflect.Field; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +public class StatefulRedisConnectionImplUnitTests extends TestSupport { + + RedisCommandBuilder commandBuilder = new RedisCommandBuilder<>(StringCodec.UTF8); + StatefulRedisConnectionImpl connection; + + @Mock + RedisAsyncCommandsImpl asyncCommands; + + @Mock + PushHandler pushHandler; + + @Mock + RedisChannelWriter writer; + + @Mock + ClientResources clientResources; + + @Mock + Tracing tracing; + + @BeforeEach + void setup() throws NoSuchFieldException, IllegalAccessException { + when(writer.getClientResources()).thenReturn(clientResources); + when(clientResources.tracing()).thenReturn(tracing); + when(tracing.isEnabled()).thenReturn(false); + when(asyncCommands.auth(any(CharSequence.class))) + .thenAnswer( invocation -> { + String pass = invocation.getArgument(0); + AsyncCommand auth = new AsyncCommand<>(commandBuilder.auth(pass)); + auth.complete(); + return auth; + }); + when(asyncCommands.auth(anyString(), any(CharSequence.class))) + .thenAnswer( invocation -> { + String user = invocation.getArgument(0); // Capture username + String pass = invocation.getArgument(1); // Capture password + AsyncCommand auth = new AsyncCommand<>(commandBuilder.auth(user, pass)); + auth.complete(); + return auth; + }); + + Field asyncField = StatefulRedisConnectionImpl.class.getDeclaredField("async"); + asyncField.setAccessible(true); + + + connection = new StatefulRedisConnectionImpl<>(writer, pushHandler, StringCodec.UTF8, Duration.ofSeconds(1)); + asyncField.set(connection,asyncCommands); + } + + @Test + public void testSetCredentialsWhenCredentialsAreNull() { + connection.setCredentials(null); + + verify(asyncCommands, never()).auth(any(CharSequence.class)); + verify(asyncCommands, never()).auth(anyString(), any(CharSequence.class)); + } + + @Test + void testSetCredentialsDispatchesAuthWhenNotInTransaction() { + connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); + verify(asyncCommands).auth(eq("user"), eq("pass")); + } + + + @Test + void testSetCredentialsDoesNotDispatchAuthIfInTransaction() { + AtomicBoolean inTransaction = ReflectionTestUtils.getField(connection, "inTransaction"); + inTransaction.set(true); + + connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); + + verify(asyncCommands, never()).auth(any(CharSequence.class)); + verify(asyncCommands, never()).auth(anyString(), any(CharSequence.class)); + } + + + @Test + void testSetCredentialsDispatchesAuthAfterTransaction() { + AtomicBoolean inTransaction = ReflectionTestUtils.getField(connection, "inTransaction"); + + connection.dispatch(commandBuilder.multi()); + assertThat(inTransaction.get()).isTrue(); + + connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); + connection.dispatch(commandBuilder.discard()); + + assertThat(inTransaction.get()).isFalse(); + + verify(asyncCommands).auth(eq("user"), eq("pass")); + } + + @Test + void testSetCredentialsDispatchesAuthAfterTransactionInAnotherThread() throws InterruptedException { + AtomicBoolean inTransaction = ReflectionTestUtils.getField(connection, "inTransaction"); + + connection.dispatch(commandBuilder.multi()); + assertThat(inTransaction.get()).isTrue(); + + Thread thread = new Thread(() -> { + connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); + }); + thread.start(); + + connection.dispatch(commandBuilder.discard()); + + thread.join(); + + assertThat(inTransaction.get()).isFalse(); + verify(asyncCommands).auth(eq("user"), eq("pass")); + } + +} From 9a0e51365f0b431e1ed776c4463c9eccde30c447 Mon Sep 17 00:00:00 2001 From: ggivo Date: Fri, 13 Dec 2024 11:56:28 +0200 Subject: [PATCH 14/28] Skip preProcessing of auth command to avoid replacing the credential provider with static one provider Add unit tests for setCredentials --- .../core/StatefulRedisConnectionImpl.java | 38 ++++++----- .../StatefulRedisConnectionImplUnitTests.java | 63 ++++++++++--------- 2 files changed, 58 insertions(+), 43 deletions(-) diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index c85191a1e..1b54ee8a4 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -357,9 +357,9 @@ public void setClientName(String clientName) { /** * Authenticates the current connection using the provided credentials. *

- * Unlike using dispatch of {@link RedisAsyncCommands#auth}, this method defers the {@code AUTH} command if the connection is within an active - * transaction. The authentication command will only be dispatched after the enclosing {@code DISCARD} or {@code EXEC} - * command is executed, ensuring that authentication does not interfere with ongoing transactions. + * Unlike using dispatch of {@link RedisAsyncCommands#auth}, this method defers the {@code AUTH} command if the connection + * is within an active transaction. The authentication command will only be dispatched after the enclosing {@code DISCARD} + * or {@code EXEC} command is executed, ensuring that authentication does not interfere with ongoing transactions. *

* * @param credentials the {@link RedisCredentials} to authenticate the connection. If {@code null}, no action is performed. @@ -421,20 +421,29 @@ protected void dispatchAuth(RedisCredentials credentials) { return; } - RedisFuture auth; + // dispatch directly to avoid AUTH preprocessing overrides credentials provider + RedisCommand auth = super.dispatch(authCommand(credentials)); + if (auth instanceof CompleteableCommand) { + ((CompleteableCommand) auth).onComplete((status, throwable) -> { + if (throwable != null) { + logger.error("Re-authentication failed {}.", getEpid(), throwable); + publishReauthFailedEvent(throwable); + } else { + logger.info("Re-authentication succeeded {}.", getEpid()); + publishReauthEvent(); + } + }); + } + } + + private AsyncCommand authCommand(RedisCredentials credentials) { + CommandArgs args = new CommandArgs<>(codec); if (credentials.getUsername() != null) { - auth = async().auth(credentials.getUsername(), String.valueOf(credentials.getPassword())); + args.add(credentials.getUsername()).add(credentials.getPassword()); } else { - auth = async().auth(String.valueOf(credentials.getPassword())); + args.add(credentials.getPassword()); } - auth.thenRun(() -> { - publishReauthEvent(); - logger.info("Re-authentication succeeded {}.", getEpid()); - }).exceptionally(throwable -> { - publishReauthFailedEvent(throwable); - logger.error("Re-authentication failed {}.", getEpid(), throwable); - return null; - }); + return new AsyncCommand<>(new Command<>(AUTH, new StatusOutput<>(codec), args)); } private void publishReauthEvent() { @@ -463,4 +472,5 @@ private String getEpid() { return ((Endpoint) writer).getId(); } + } diff --git a/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java b/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java index 245eef09b..d418c4251 100644 --- a/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java +++ b/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java @@ -2,13 +2,17 @@ import io.lettuce.core.codec.StringCodec; import io.lettuce.core.protocol.AsyncCommand; +import io.lettuce.core.protocol.CommandType; import io.lettuce.core.protocol.PushHandler; +import io.lettuce.core.protocol.RedisCommand; import io.lettuce.core.resource.ClientResources; import io.lettuce.core.tracing.Tracing; import io.lettuce.test.ReflectionTestUtils; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentMatcher; +import org.mockito.ArgumentMatchers; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; @@ -21,6 +25,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; @@ -31,10 +36,8 @@ public class StatefulRedisConnectionImplUnitTests extends TestSupport { RedisCommandBuilder commandBuilder = new RedisCommandBuilder<>(StringCodec.UTF8); - StatefulRedisConnectionImpl connection; - @Mock - RedisAsyncCommandsImpl asyncCommands; + StatefulRedisConnectionImpl connection; @Mock PushHandler pushHandler; @@ -53,45 +56,26 @@ void setup() throws NoSuchFieldException, IllegalAccessException { when(writer.getClientResources()).thenReturn(clientResources); when(clientResources.tracing()).thenReturn(tracing); when(tracing.isEnabled()).thenReturn(false); - when(asyncCommands.auth(any(CharSequence.class))) - .thenAnswer( invocation -> { - String pass = invocation.getArgument(0); - AsyncCommand auth = new AsyncCommand<>(commandBuilder.auth(pass)); - auth.complete(); - return auth; - }); - when(asyncCommands.auth(anyString(), any(CharSequence.class))) - .thenAnswer( invocation -> { - String user = invocation.getArgument(0); // Capture username - String pass = invocation.getArgument(1); // Capture password - AsyncCommand auth = new AsyncCommand<>(commandBuilder.auth(user, pass)); - auth.complete(); - return auth; - }); Field asyncField = StatefulRedisConnectionImpl.class.getDeclaredField("async"); asyncField.setAccessible(true); - connection = new StatefulRedisConnectionImpl<>(writer, pushHandler, StringCodec.UTF8, Duration.ofSeconds(1)); - asyncField.set(connection,asyncCommands); } @Test public void testSetCredentialsWhenCredentialsAreNull() { connection.setCredentials(null); - verify(asyncCommands, never()).auth(any(CharSequence.class)); - verify(asyncCommands, never()).auth(anyString(), any(CharSequence.class)); + verify(writer, never()).write(ArgumentMatchers.> any()); } @Test void testSetCredentialsDispatchesAuthWhenNotInTransaction() { connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); - verify(asyncCommands).auth(eq("user"), eq("pass")); + verify(writer).write(argThat(isAuthCommand("user", "pass"))); } - @Test void testSetCredentialsDoesNotDispatchAuthIfInTransaction() { AtomicBoolean inTransaction = ReflectionTestUtils.getField(connection, "inTransaction"); @@ -99,11 +83,9 @@ void testSetCredentialsDoesNotDispatchAuthIfInTransaction() { connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); - verify(asyncCommands, never()).auth(any(CharSequence.class)); - verify(asyncCommands, never()).auth(anyString(), any(CharSequence.class)); + verify(writer, never()).write(ArgumentMatchers.> any()); } - @Test void testSetCredentialsDispatchesAuthAfterTransaction() { AtomicBoolean inTransaction = ReflectionTestUtils.getField(connection, "inTransaction"); @@ -116,7 +98,7 @@ void testSetCredentialsDispatchesAuthAfterTransaction() { assertThat(inTransaction.get()).isFalse(); - verify(asyncCommands).auth(eq("user"), eq("pass")); + verify(writer).write(argThat(isAuthCommand("user", "pass"))); } @Test @@ -136,7 +118,30 @@ void testSetCredentialsDispatchesAuthAfterTransactionInAnotherThread() throws In thread.join(); assertThat(inTransaction.get()).isFalse(); - verify(asyncCommands).auth(eq("user"), eq("pass")); + verify(writer).write(argThat(isAuthCommand("user", "pass"))); + } + + public static ArgumentMatcher> isAuthCommand(String expectedUsername, + String expectedPassword) { + return new ArgumentMatcher>() { + + @Override + public boolean matches(RedisCommand command) { + if (command.getType() != CommandType.AUTH) { + return false; + } + + // Retrieve arguments (adjust based on your RedisCommand implementation) + return command.getArgs().toCommandString().equals(expectedUsername + " " + expectedPassword); + } + + @Override + public String toString() { + return String.format("Expected AUTH command with username=%s and password=%s", expectedUsername, + expectedPassword); + } + + }; } } From 6ec2846e356c8e5e9c23824fc744ca2a4a46c9ee Mon Sep 17 00:00:00 2001 From: ggivo Date: Fri, 13 Dec 2024 12:17:20 +0200 Subject: [PATCH 15/28] clean up - remove dead code --- .../java/io/lettuce/core/api/StatefulRedisConnection.java | 1 - .../core/cluster/StatefulRedisClusterConnectionImpl.java | 5 ----- 2 files changed, 6 deletions(-) diff --git a/src/main/java/io/lettuce/core/api/StatefulRedisConnection.java b/src/main/java/io/lettuce/core/api/StatefulRedisConnection.java index 7cac57887..1be962ba3 100644 --- a/src/main/java/io/lettuce/core/api/StatefulRedisConnection.java +++ b/src/main/java/io/lettuce/core/api/StatefulRedisConnection.java @@ -1,6 +1,5 @@ package io.lettuce.core.api; -import io.lettuce.core.RedisCredentials; import io.lettuce.core.api.async.RedisAsyncCommands; import io.lettuce.core.api.push.PushListener; import io.lettuce.core.api.reactive.RedisReactiveCommands; diff --git a/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java b/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java index 23a027315..c84193491 100644 --- a/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java +++ b/src/main/java/io/lettuce/core/cluster/StatefulRedisClusterConnectionImpl.java @@ -232,11 +232,6 @@ public void activated() { async.clusterMyId().thenAccept(connectionState::setNodeId); } - @Override - public void deactivated() { - super.deactivated(); - } - ClusterDistributionChannelWriter getClusterDistributionChannelWriter() { return (ClusterDistributionChannelWriter) super.getChannelWriter(); } From 110eb1ab101dc7282b71426bf1af0e076f1a57a7 Mon Sep 17 00:00:00 2001 From: Tihomir Mateev Date: Tue, 17 Dec 2024 14:53:12 +0200 Subject: [PATCH 16/28] Moved almost all code inside the new handler --- .../java/io/lettuce/core/ClientOptions.java | 45 +-- .../core/RedisAuthenticationHandler.java | 292 +++++++++++++++--- .../java/io/lettuce/core/RedisClient.java | 7 +- .../core/RedisCredentialsProvider.java | 11 + .../core/StatefulRedisConnectionImpl.java | 172 ++--------- .../core/StreamingCredentialsProvider.java | 4 + .../core/cluster/ClusterClientOptions.java | 1 + .../core/cluster/RedisClusterClient.java | 15 +- .../event/connection/AuthenticateEvent.java | 15 - .../event/connection/AuthenticationEvent.java | 24 ++ .../core/event/connection/JfrReauthEvent.java | 18 +- .../connection/JfrReauthFailedEvent.java | 18 +- .../event/connection/ReauthenticateEvent.java | 21 -- .../connection/ReauthenticateFailedEvent.java | 39 --- .../connection/ReauthenticationEvent.java | 34 ++ .../ReauthenticationFailedEvent.java | 56 ++++ .../core/RedisAuthenticationHandlerTest.java | 135 -------- .../RedisAuthenticationHandlerUnitTests.java | 210 +++++++++++++ .../StatefulRedisConnectionImplUnitTests.java | 147 --------- ...ectionEventsTriggeredIntegrationTests.java | 22 +- 20 files changed, 690 insertions(+), 596 deletions(-) delete mode 100644 src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java create mode 100644 src/main/java/io/lettuce/core/event/connection/AuthenticationEvent.java delete mode 100644 src/main/java/io/lettuce/core/event/connection/ReauthenticateEvent.java delete mode 100644 src/main/java/io/lettuce/core/event/connection/ReauthenticateFailedEvent.java create mode 100644 src/main/java/io/lettuce/core/event/connection/ReauthenticationEvent.java create mode 100644 src/main/java/io/lettuce/core/event/connection/ReauthenticationFailedEvent.java delete mode 100644 src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java create mode 100644 src/test/java/io/lettuce/core/RedisAuthenticationHandlerUnitTests.java delete mode 100644 src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java diff --git a/src/main/java/io/lettuce/core/ClientOptions.java b/src/main/java/io/lettuce/core/ClientOptions.java index 8bc2c8109..1248800a5 100644 --- a/src/main/java/io/lettuce/core/ClientOptions.java +++ b/src/main/java/io/lettuce/core/ClientOptions.java @@ -309,6 +309,13 @@ public Builder disconnectedBehavior(DisconnectedBehavior disconnectedBehavior) { return this; } + /** + * Configure the {@link ReauthenticateBehavior} of the Lettuce driver. Defaults to + * {@link ReauthenticateBehavior#DEFAULT}. + * + * @param reauthenticateBehavior the {@link ReauthenticateBehavior} to use. Must not be {@code null}. + * @return {@code this} + */ public Builder reauthenticateBehavior(ReauthenticateBehavior reauthenticateBehavior) { LettuceAssert.notNull(reauthenticateBehavior, "ReuthenticatBehavior must not be null"); @@ -589,6 +596,12 @@ public DisconnectedBehavior getDisconnectedBehavior() { return disconnectedBehavior; } + /** + * Behavior for re-authentication when the {@link RedisCredentialsProvider} emits new credentials. Defaults to + * {@link ReauthenticateBehavior#DEFAULT}. + * + * @return the currently set {@link ReauthenticateBehavior}. + */ public ReauthenticateBehavior getReauthenticateBehaviour() { return reauthenticateBehavior; } @@ -725,35 +738,31 @@ public TimeoutOptions getTimeoutOptions() { } /** - * Defines the re-authentication behavior of the Redis client in relation to the {@link CredentialsProvider}. + * Defines the re-authentication behavior of the Redis client. + *

+ * Certain implementations of the {@link RedisCredentialsProvider} such as the {@link StreamingCredentialsProvider} could + * emit new credentials at runtime. This setting controls how the driver reacts to these newly emitted credentials. */ public enum ReauthenticateBehavior { /** * This is the default behavior. The client will fetch current credentials from the underlying - * {@link RedisCredentialsProvider} only when required. - * - *

- * No re-authentication is performed automatically when new credentials are emitted by the - * {@link StreamingCredentialsProvider} . - *

- * - *

- * This behavior is suitable for use cases with static credentials or where explicit reconnection is required to change - * credentials. - *

+ * {@link RedisCredentialsProvider} only when the driver needs to, e.g. when the connection is first established or when + * it is re-established after a disconnect. + *

+ * No re-authentication is performed when new credentials are emitted by the {@link StreamingCredentialsProvider} . */ DEFAULT, /** - * Automatically triggers re-authentication whenever new credentials are emitted by the - * {@link StreamingCredentialsProvider} or any other credentials manager. + * Automatically triggers re-authentication whenever new credentials are emitted by any implementation of the + * {@link StreamingCredentialsProvider} interface. * *

- * When enabled, the client subscribes to the credentials stream provided by the {@link StreamingCredentialsProvider} - * (or other compatible sources) and issues an {@code AUTH} command to the Redis server each time new credentials are - * received. This behavior supports dynamic credential scenarios, such as token-based authentication, or credential - * rotation where credentials are refreshed periodically to maintain access. + * When enabled, the client subscribes to the credential stream provided by the {@link StreamingCredentialsProvider} and + * issues an {@code AUTH} command to the Redis server each time new credentials are received. This behavior supports + * dynamic credential scenarios, such as token-based authentication, or credential rotation where credentials are + * refreshed periodically to maintain access. *

* *

diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index 1bf329e86..cbdc12c8a 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -1,30 +1,22 @@ /* - * Copyright 2019-Present, Redis Ltd. and Contributors + * Copyright 2024, Redis Ltd. and Contributors * All rights reserved. * * Licensed under the MIT License. - * - * This file contains contributions from third-party contributors - * licensed 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 - * - * https://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 io.lettuce.core; -import io.lettuce.core.codec.StringCodec; -import io.lettuce.core.event.EventBus; -import io.lettuce.core.event.connection.ReauthenticateEvent; -import io.lettuce.core.event.connection.ReauthenticateFailedEvent; +import io.lettuce.core.api.async.RedisAsyncCommands; +import io.lettuce.core.codec.RedisCodec; +import io.lettuce.core.event.connection.ReauthenticationEvent; +import io.lettuce.core.event.connection.ReauthenticationFailedEvent; import io.lettuce.core.internal.LettuceAssert; +import io.lettuce.core.output.StatusOutput; import io.lettuce.core.protocol.AsyncCommand; +import io.lettuce.core.protocol.Command; +import io.lettuce.core.protocol.CommandArgs; +import io.lettuce.core.protocol.CommandExpiryWriter; +import io.lettuce.core.protocol.CompleteableCommand; import io.lettuce.core.protocol.Endpoint; import io.lettuce.core.protocol.ProtocolVersion; import io.lettuce.core.protocol.RedisCommand; @@ -33,8 +25,13 @@ import reactor.core.Disposable; import reactor.core.publisher.Flux; -import java.nio.CharBuffer; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; + +import static io.lettuce.core.protocol.CommandType.AUTH; +import static io.lettuce.core.protocol.CommandType.DISCARD; +import static io.lettuce.core.protocol.CommandType.EXEC; /** * Redis authentication handler. Internally used to authenticate a Redis connection. This class is part of the internal API. @@ -42,11 +39,11 @@ * @author Ivo Gaydazhiev * @since 6.6.0 */ -public class RedisAuthenticationHandler { +public class RedisAuthenticationHandler { private static final InternalLogger log = InternalLoggerFactory.getInstance(RedisAuthenticationHandler.class); - private final StatefulRedisConnectionImpl connection; + private final StatefulRedisConnectionImpl connection; private final RedisCredentialsProvider credentialsProvider; @@ -54,13 +51,61 @@ public class RedisAuthenticationHandler { private final Boolean isPubSubConnection; - public RedisAuthenticationHandler(StatefulRedisConnectionImpl connection, + private final AtomicReference credentialsRef = new AtomicReference<>(); + + private final ReentrantLock reAuthSafety = new ReentrantLock(); + + private final AtomicBoolean inTransaction = new AtomicBoolean(false); + + /** + * Creates a new {@link RedisAuthenticationHandler}. + * + * @param connection the connection to authenticate + * @param credentialsProvider the implementation of {@link RedisCredentialsProvider} to use + * @param isPubSubConnection {@code true} if the connection is a pub/sub connection + */ + public RedisAuthenticationHandler(StatefulRedisConnectionImpl connection, RedisCredentialsProvider credentialsProvider, Boolean isPubSubConnection) { this.connection = connection; this.credentialsProvider = credentialsProvider; this.isPubSubConnection = isPubSubConnection; } + /** + * Creates a new {@link RedisAuthenticationHandler} if the connection supports re-authentication. + * + * @param connection the connection to authenticate + * @param credentialsProvider the implementation of {@link RedisCredentialsProvider} to use + * @param isPubSubConnection {@code true} if the connection is a pub/sub connection + * @param options the {@link ClientOptions} to use + * @return a new {@link RedisAuthenticationHandler} if the connection supports re-authentication, otherwise an + * implementation of the {@link RedisAuthenticationHandler} that does nothing + * @since 6.6.0 + * @see StreamingCredentialsProvider + */ + public static RedisAuthenticationHandler createHandler(StatefulRedisConnectionImpl connection, + RedisCredentialsProvider credentialsProvider, Boolean isPubSubConnection, ClientOptions options) { + + if (isSupported(options)) { + return new RedisAuthenticationHandler<>(connection, credentialsProvider, isPubSubConnection); + } + + return null; + } + + /** + * Creates a new default {@link RedisAuthenticationHandler}. + *

+ * The default {@link RedisAuthenticationHandler} is used when re-authentication is not supported. + * + * @return a new {@link RedisAuthenticationHandler} + * @since 6.6.0 + * @see StreamingCredentialsProvider + */ + public static RedisAuthenticationHandler createDefaultAuthenticationHandler() { + return new DisabledAuthenticationHandler<>(); + } + /** * This method subscribes to a stream of credentials provided by the `StreamingCredentialsProvider`. *

@@ -68,23 +113,21 @@ public RedisAuthenticationHandler(StatefulRedisConnectionImpl connection, * before setting the new subscription. */ public void subscribe() { - if (credentialsProvider == null) { + if (credentialsProvider == null || !credentialsProvider.supportsStreaming()) { return; } - if (credentialsProvider instanceof StreamingCredentialsProvider) { - if (!isSupportedConnection()) { - return; - } + if (!isSupportedConnection()) { + return; + } - Flux credentialsFlux = ((StreamingCredentialsProvider) credentialsProvider).credentials(); + Flux credentialsFlux = ((StreamingCredentialsProvider) credentialsProvider).credentials(); - Disposable subscription = credentialsFlux.subscribe(this::onNext, this::onError, this::complete); + Disposable subscription = credentialsFlux.subscribe(this::onNext, this::onError, this::complete); - Disposable oldSubscription = credentialsSubscription.getAndSet(subscription); - if (oldSubscription != null && !oldSubscription.isDisposed()) { - oldSubscription.dispose(); - } + Disposable oldSubscription = credentialsSubscription.getAndSet(subscription); + if (oldSubscription != null && !oldSubscription.isDisposed()) { + oldSubscription.dispose(); } } @@ -117,10 +160,10 @@ protected void onError(Throwable e) { * @param credentials the new credentials */ protected void reauthenticate(RedisCredentials credentials) { - connection.setCredentials(credentials); + setCredentials(credentials); } - protected boolean isSupportedConnection() { + boolean isSupportedConnection() { if (isPubSubConnection && ProtocolVersion.RESP2 == connection.getConnectionState().getNegotiatedProtocolVersion()) { log.warn("Renewable credentials are not supported with RESP2 protocol on a pub/sub connection."); return false; @@ -128,22 +171,185 @@ protected boolean isSupportedConnection() { return true; } - private void publishReauthFailedEvent(Throwable throwable) { - connection.getResources().eventBus().publish(new ReauthenticateFailedEvent(throwable)); - } - - public static boolean isSupported(ClientOptions clientOptions) { + private static boolean isSupported(ClientOptions clientOptions) { LettuceAssert.notNull(clientOptions, "ClientOptions must not be null"); switch (clientOptions.getReauthenticateBehaviour()) { case ON_NEW_CREDENTIALS: return true; - case DEFAULT: - return false; - default: return false; } } + public void postProcess(RedisCommand toSend) { + if (toSend.getType() == EXEC || toSend.getType() == DISCARD) { + inTransaction.set(false); + setCredentials(credentialsRef.getAndSet(null)); + } + } + + /** + * Marks that the current connection has started a transaction. + *

+ * During transactions, any re-authentication attempts are deferred until the transaction ends. + */ + public void startTransaction() { + reAuthSafety.lock(); + try { + inTransaction.set(true); + } finally { + reAuthSafety.unlock(); + } + } + + /** + * Marks that the current connection has ended the transaction. + *

+ * After a transaction is completed, any deferred re-authentication attempts are dispatched. + */ + public void endTransaction() { + inTransaction.set(false); + setCredentials(credentialsRef.getAndSet(null)); + } + + /** + * Authenticates the current connection using the provided credentials. + *

+ * Unlike using dispatch of {@link RedisAsyncCommands#auth}, this method defers the {@code AUTH} command if the connection + * is within an active transaction. The authentication command will only be dispatched after the enclosing {@code DISCARD} + * or {@code EXEC} command is executed, ensuring that authentication does not interfere with ongoing transactions. + *

+ * + * @param credentials the {@link RedisCredentials} to authenticate the connection. If {@code null}, no action is performed. + * + *

+ * Behavior: + *

    + *
  • If the provided credentials are {@code null}, the method exits immediately.
  • + *
  • If a transaction is active (as indicated by {@code inTransaction}), the {@code AUTH} command is not dispatched + * immediately but deferred until the transaction ends.
  • + *
  • If no transaction is active, the {@code AUTH} command is dispatched immediately using the provided + * credentials.
  • + *
+ *

+ * + * @see RedisAsyncCommands#auth + */ + public void setCredentials(RedisCredentials credentials) { + if (credentials == null) { + return; + } + reAuthSafety.lock(); + try { + credentialsRef.set(credentials); + if (!inTransaction.get()) { + dispatchAuth(credentialsRef.getAndSet(null)); + } + } finally { + reAuthSafety.unlock(); + } + } + + protected void dispatchAuth(RedisCredentials credentials) { + if (credentials == null) { + return; + } + + // dispatch directly to avoid AUTH preprocessing overrides credentials provider + RedisCommand auth = connection.dispatch(authCommand(credentials)); + if (auth instanceof CompleteableCommand) { + ((CompleteableCommand) auth).onComplete((status, throwable) -> { + if (throwable != null) { + log.error("Re-authentication failed {}.", getEpid(), throwable); + publishReauthFailedEvent(throwable); + } else { + log.info("Re-authentication succeeded {}.", getEpid()); + publishReauthEvent(); + } + }); + } + } + + private AsyncCommand authCommand(RedisCredentials credentials) { + RedisCodec codec = connection.getCodec(); + CommandArgs args = new CommandArgs<>(codec); + if (credentials.getUsername() != null) { + args.add(credentials.getUsername()).add(credentials.getPassword()); + } else { + args.add(credentials.getPassword()); + } + return new AsyncCommand<>(new Command<>(AUTH, new StatusOutput<>(codec), args)); + } + + private void publishReauthEvent() { + connection.getResources().eventBus().publish(new ReauthenticationEvent(getEpid())); + } + + private void publishReauthFailedEvent(Throwable throwable) { + connection.getResources().eventBus().publish(new ReauthenticationFailedEvent(getEpid(), throwable)); + } + + private String getEpid() { + RedisChannelWriter writer = connection.getChannelWriter(); + while (!(writer instanceof Endpoint)) { + + if (writer instanceof CommandListenerWriter) { + writer = ((CommandListenerWriter) writer).getDelegate(); + continue; + } + + if (writer instanceof CommandExpiryWriter) { + writer = ((CommandExpiryWriter) writer).getDelegate(); + continue; + } + return null; + } + + return ((Endpoint) writer).getId(); + } + + private static final class DisabledAuthenticationHandler extends RedisAuthenticationHandler { + + public DisabledAuthenticationHandler(StatefulRedisConnectionImpl connection, + RedisCredentialsProvider credentialsProvider, Boolean isPubSubConnection) { + super(null, null, null); + } + + public DisabledAuthenticationHandler() { + super(null, null, null); + } + + @Override + public void postProcess(RedisCommand toSend) { + // No-op + } + + @Override + public void startTransaction() { + // No-op + } + + @Override + public void endTransaction() { + // No-op + } + + @Override + public void setCredentials(RedisCredentials credentials) { + // No-op + } + + @Override + public void unsubscribe() { + // No-op + } + + @Override + public void subscribe() { + // No-op + } + + } + } diff --git a/src/main/java/io/lettuce/core/RedisClient.java b/src/main/java/io/lettuce/core/RedisClient.java index d2d04e307..78b28d3a1 100644 --- a/src/main/java/io/lettuce/core/RedisClient.java +++ b/src/main/java/io/lettuce/core/RedisClient.java @@ -19,6 +19,7 @@ */ package io.lettuce.core; +import static io.lettuce.core.RedisAuthenticationHandler.createHandler; import static io.lettuce.core.internal.LettuceStrings.*; import java.net.InetSocketAddress; @@ -317,10 +318,8 @@ private ConnectionFuture connectStatefulAsync(StatefulRedisConnecti ConnectionState state = connection.getConnectionState(); state.apply(redisURI); state.setDb(redisURI.getDatabase()); - if (RedisAuthenticationHandler.isSupported(getOptions())) { - connection.setAuthenticationHandler( - new RedisAuthenticationHandler(connection, redisURI.getCredentialsProvider(), isPubSub)); - } + connection + .setAuthenticationHandler(createHandler(connection, redisURI.getCredentialsProvider(), isPubSub, getOptions())); connectionBuilder.connection(connection); connectionBuilder.clientOptions(getOptions()); connectionBuilder.clientResources(getResources()); diff --git a/src/main/java/io/lettuce/core/RedisCredentialsProvider.java b/src/main/java/io/lettuce/core/RedisCredentialsProvider.java index afaef0ae7..556483edf 100644 --- a/src/main/java/io/lettuce/core/RedisCredentialsProvider.java +++ b/src/main/java/io/lettuce/core/RedisCredentialsProvider.java @@ -41,6 +41,17 @@ static RedisCredentialsProvider from(Supplier supplier) { return () -> Mono.fromSupplier(supplier); } + /** + * Some implementations of the {@link RedisCredentialsProvider} may support streaming new credentials, based on some event + * that originates outside the driver. In this case they should indicate that so the {@link RedisAuthenticationHandler} is + * able to process these new credentials. + * + * @return whether the {@link RedisCredentialsProvider} supports streaming credentials. + */ + default boolean supportsStreaming() { + return false; + } + /** * Extension to {@link RedisCredentialsProvider} that resolves credentials immediately without the need to defer the * credential resolution. diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index 1b54ee8a4..58d5b36c8 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -26,9 +26,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -40,14 +37,10 @@ import io.lettuce.core.cluster.api.sync.RedisClusterCommands; import io.lettuce.core.codec.RedisCodec; import io.lettuce.core.codec.StringCodec; -import io.lettuce.core.event.connection.ReauthenticateEvent; -import io.lettuce.core.event.connection.ReauthenticateFailedEvent; import io.lettuce.core.json.JsonParser; import io.lettuce.core.output.MultiOutput; import io.lettuce.core.output.StatusOutput; import io.lettuce.core.protocol.*; -import io.netty.util.internal.logging.InternalLogger; -import io.netty.util.internal.logging.InternalLoggerFactory; import reactor.core.publisher.Mono; /** @@ -62,8 +55,6 @@ */ public class StatefulRedisConnectionImpl extends RedisChannelHandler implements StatefulRedisConnection { - private static final InternalLogger logger = InternalLoggerFactory.getInstance(StatefulRedisConnectionImpl.class); - protected final RedisCodec codec; protected final RedisCommands sync; @@ -80,13 +71,7 @@ public class StatefulRedisConnectionImpl extends RedisChannelHandler protected MultiOutput multi; - private RedisAuthenticationHandler authHandler; - - private AtomicReference credentialsRef = new AtomicReference<>(); - - private final ReentrantLock reAuthSafety = new ReentrantLock(); - - private AtomicBoolean inTransaction = new AtomicBoolean(false); + private RedisAuthenticationHandler authHandler = RedisAuthenticationHandler.createDefaultAuthenticationHandler(); /** * Initialize a new connection. @@ -199,12 +184,9 @@ public RedisCommand dispatch(RedisCommand command) { RedisCommand toSend = preProcessCommand(command); RedisCommand result = super.dispatch(toSend); - if (toSend.getType() == EXEC || toSend.getType() == DISCARD) { - inTransaction.set(false); - setCredentials(credentialsRef.getAndSet(null)); - } + RedisCommand finalCommand = postProcessCommand(result); - return result; + return finalCommand; } @Override @@ -212,24 +194,20 @@ public RedisCommand dispatch(RedisCommand command) { List> sentCommands = new ArrayList<>(commands.size()); - boolean transactionComplete = false; - for (RedisCommand o : commands) { - RedisCommand command = preProcessCommand(o); - sentCommands.add(command); - if (command.getType() == EXEC) { - transactionComplete = true; - } - if (command.getType() == MULTI || command.getType() == DISCARD) { - transactionComplete = false; - } - } + commands.forEach(o -> { + RedisCommand preprocessed = preProcessCommand(o); + sentCommands.add(preprocessed); + }); - Collection> result = super.dispatch(sentCommands); - if (transactionComplete) { - inTransaction.set(false); - setCredentials(credentialsRef.getAndSet(null)); - } - return result; + super.dispatch(sentCommands); + + sentCommands.forEach(this::postProcessCommand); + return sentCommands; + } + + protected RedisCommand postProcessCommand(RedisCommand command) { + authHandler.postProcess(command); + return command; } // TODO [tihomir.mateev] Refactor to include as part of the Command interface @@ -307,21 +285,14 @@ protected RedisCommand preProcessCommand(RedisCommand comm } if (commandType.equals(MULTI.name())) { - - reAuthSafety.lock(); - try { - inTransaction.set(true); - } finally { - reAuthSafety.unlock(); - } + authHandler.startTransaction(); multi = (multi == null ? new MultiOutput<>(codec) : multi); if (command instanceof CompleteableCommand) { ((CompleteableCommand) command).onComplete((ignored, e) -> { if (e != null) { multi = null; - inTransaction.set(false); - setCredentials(credentialsRef.getAndSet(null)); + authHandler.endTransaction(); } }); } @@ -354,44 +325,6 @@ public void setClientName(String clientName) { dispatch((RedisCommand) async); } - /** - * Authenticates the current connection using the provided credentials. - *

- * Unlike using dispatch of {@link RedisAsyncCommands#auth}, this method defers the {@code AUTH} command if the connection - * is within an active transaction. The authentication command will only be dispatched after the enclosing {@code DISCARD} - * or {@code EXEC} command is executed, ensuring that authentication does not interfere with ongoing transactions. - *

- * - * @param credentials the {@link RedisCredentials} to authenticate the connection. If {@code null}, no action is performed. - * - *

- * Behavior: - *

    - *
  • If the provided credentials are {@code null}, the method exits immediately.
  • - *
  • If a transaction is active (as indicated by {@code inTransaction}), the {@code AUTH} command is not dispatched - * immediately but deferred until the transaction ends.
  • - *
  • If no transaction is active, the {@code AUTH} command is dispatched immediately using the provided - * credentials.
  • - *
- *

- * - * @see RedisAsyncCommands#auth - */ - public void setCredentials(RedisCredentials credentials) { - if (credentials == null) { - return; - } - reAuthSafety.lock(); - try { - credentialsRef.set(credentials); - if (!inTransaction.get()) { - dispatchAuth(credentialsRef.getAndSet(null)); - } - } finally { - reAuthSafety.unlock(); - } - } - public ConnectionState getConnectionState() { return state; } @@ -399,78 +332,19 @@ public ConnectionState getConnectionState() { @Override public void activated() { super.activated(); - if (authHandler != null) { - authHandler.subscribe(); - } + authHandler.subscribe(); } @Override public void deactivated() { - if (authHandler != null) { - authHandler.unsubscribe(); - } + authHandler.unsubscribe(); super.deactivated(); } - public void setAuthenticationHandler(RedisAuthenticationHandler handler) { - authHandler = handler; - } - - protected void dispatchAuth(RedisCredentials credentials) { - if (credentials == null) { - return; - } - - // dispatch directly to avoid AUTH preprocessing overrides credentials provider - RedisCommand auth = super.dispatch(authCommand(credentials)); - if (auth instanceof CompleteableCommand) { - ((CompleteableCommand) auth).onComplete((status, throwable) -> { - if (throwable != null) { - logger.error("Re-authentication failed {}.", getEpid(), throwable); - publishReauthFailedEvent(throwable); - } else { - logger.info("Re-authentication succeeded {}.", getEpid()); - publishReauthEvent(); - } - }); - } - } - - private AsyncCommand authCommand(RedisCredentials credentials) { - CommandArgs args = new CommandArgs<>(codec); - if (credentials.getUsername() != null) { - args.add(credentials.getUsername()).add(credentials.getPassword()); - } else { - args.add(credentials.getPassword()); + public void setAuthenticationHandler(RedisAuthenticationHandler handler) { + if (handler != null) { + authHandler = handler; } - return new AsyncCommand<>(new Command<>(AUTH, new StatusOutput<>(codec), args)); - } - - private void publishReauthEvent() { - getResources().eventBus().publish(new ReauthenticateEvent(getEpid())); - } - - private void publishReauthFailedEvent(Throwable throwable) { - getResources().eventBus().publish(new ReauthenticateFailedEvent(getEpid(), throwable)); - } - - private String getEpid() { - RedisChannelWriter writer = getChannelWriter(); - while (!(writer instanceof Endpoint)) { - - if (writer instanceof CommandListenerWriter) { - writer = ((CommandListenerWriter) writer).getDelegate(); - continue; - } - - if (writer instanceof CommandExpiryWriter) { - writer = ((CommandExpiryWriter) writer).getDelegate(); - continue; - } - return null; - } - - return ((Endpoint) writer).getId(); } } diff --git a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java index 999d35683..77a131f37 100644 --- a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java +++ b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java @@ -19,4 +19,8 @@ public interface StreamingCredentialsProvider extends RedisCredentialsProvider { */ Flux credentials(); + default boolean supportsStreaming() { + return true; + } + } diff --git a/src/main/java/io/lettuce/core/cluster/ClusterClientOptions.java b/src/main/java/io/lettuce/core/cluster/ClusterClientOptions.java index b0f6cc018..832c310f5 100644 --- a/src/main/java/io/lettuce/core/cluster/ClusterClientOptions.java +++ b/src/main/java/io/lettuce/core/cluster/ClusterClientOptions.java @@ -134,6 +134,7 @@ public static ClusterClientOptions.Builder builder(ClientOptions clientOptions) .cancelCommandsOnReconnectFailure(clientOptions.isCancelCommandsOnReconnectFailure()) .decodeBufferPolicy(clientOptions.getDecodeBufferPolicy()) .disconnectedBehavior(clientOptions.getDisconnectedBehavior()) + .reauthenticateBehavior(clientOptions.getReauthenticateBehaviour()) .pingBeforeActivateConnection(clientOptions.isPingBeforeActivateConnection()) .publishOnScheduler(clientOptions.isPublishOnScheduler()) .protocolVersion(clientOptions.getConfiguredProtocolVersion()) diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java index ef8690a2b..e62fbc731 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java @@ -74,6 +74,8 @@ import io.netty.util.internal.logging.InternalLoggerFactory; import reactor.core.publisher.Mono; +import static io.lettuce.core.RedisAuthenticationHandler.createHandler; + /** * A scalable and thread-safe Redis cluster client supporting synchronous, asynchronous and * reactive execution models. Multiple threads may share one connection. The cluster client handles command routing based on the @@ -556,10 +558,8 @@ ConnectionFuture> connectToNodeAsync(RedisC StatefulRedisConnectionImpl connection = newStatefulRedisConnection(writer, endpoint, codec, getFirstUri().getTimeout(), getClusterClientOptions().getJsonParser()); - if (RedisAuthenticationHandler.isSupported(getOptions())) { - connection.setAuthenticationHandler( - new RedisAuthenticationHandler(connection, getFirstUri().getCredentialsProvider(), false)); - } + connection.setAuthenticationHandler( + createHandler(connection, getFirstUri().getCredentialsProvider(), false, getOptions())); ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, @@ -625,11 +625,8 @@ ConnectionFuture> connectPubSubToNode StatefulRedisPubSubConnectionImpl connection = new StatefulRedisPubSubConnectionImpl<>(endpoint, writer, codec, getFirstUri().getTimeout()); - - if (RedisAuthenticationHandler.isSupported(getOptions())) { - connection.setAuthenticationHandler( - new RedisAuthenticationHandler(connection, getFirstUri().getCredentialsProvider(), true)); - } + connection.setAuthenticationHandler( + createHandler(connection, getFirstUri().getCredentialsProvider(), true, getOptions())); ConnectionFuture> connectionFuture = connectStatefulAsync(connection, endpoint, getFirstUri(), socketAddressSupplier, diff --git a/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java b/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java deleted file mode 100644 index 6a02be399..000000000 --- a/src/main/java/io/lettuce/core/event/connection/AuthenticateEvent.java +++ /dev/null @@ -1,15 +0,0 @@ -package io.lettuce.core.event.connection; - -import io.lettuce.core.event.Event; - -/** - * Interface for Connection authentication events - * - * @author Ivo Gaydajiev - * @since 6.6.0 - */ -public interface AuthenticateEvent extends Event { - - String getEpId(); - -} diff --git a/src/main/java/io/lettuce/core/event/connection/AuthenticationEvent.java b/src/main/java/io/lettuce/core/event/connection/AuthenticationEvent.java new file mode 100644 index 000000000..ba13a975d --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/AuthenticationEvent.java @@ -0,0 +1,24 @@ +/* + * Copyright 2024, Redis Ltd. and Contributors + * All rights reserved. + * + * Licensed under the MIT License. + */ +package io.lettuce.core.event.connection; + +import io.lettuce.core.event.Event; + +/** + * Interface for Connection authentication events + * + * @author Ivo Gaydajiev + * @since 6.6.0 + */ +public interface AuthenticationEvent extends Event { + + /** + * @return the endpoint ID associated with this event + */ + String getEpId(); + +} diff --git a/src/main/java/io/lettuce/core/event/connection/JfrReauthEvent.java b/src/main/java/io/lettuce/core/event/connection/JfrReauthEvent.java index 8faaad8f6..e167843d1 100644 --- a/src/main/java/io/lettuce/core/event/connection/JfrReauthEvent.java +++ b/src/main/java/io/lettuce/core/event/connection/JfrReauthEvent.java @@ -1,3 +1,9 @@ +/* + * Copyright 2024, Redis Ltd. and Contributors + * All rights reserved. + * + * Licensed under the MIT License. + */ package io.lettuce.core.event.connection; import jdk.jfr.Category; @@ -6,7 +12,7 @@ import jdk.jfr.StackTrace; /** - * Flight recorder event variant of {@link ReauthenticateEvent}. + * Flight recorder event variant of {@link ReauthenticationEvent}. * * @author Ivo Gaydajiev * @since 6.6.0 @@ -18,10 +24,18 @@ class JfrReauthEvent extends Event { private final String epId; - public JfrReauthEvent(ReauthenticateEvent event) { + /** + * Create a new {@link JfrReauthEvent} given a {@link ReauthenticationEvent}. + * + * @param event the {@link ReauthenticationEvent} + */ + public JfrReauthEvent(ReauthenticationEvent event) { this.epId = event.getEpId(); } + /** + * @return the connection endpoint ID + */ public String getEpId() { return epId; } diff --git a/src/main/java/io/lettuce/core/event/connection/JfrReauthFailedEvent.java b/src/main/java/io/lettuce/core/event/connection/JfrReauthFailedEvent.java index b1b5d7899..183d75a63 100644 --- a/src/main/java/io/lettuce/core/event/connection/JfrReauthFailedEvent.java +++ b/src/main/java/io/lettuce/core/event/connection/JfrReauthFailedEvent.java @@ -1,3 +1,9 @@ +/* + * Copyright 2024, Redis Ltd. and Contributors + * All rights reserved. + * + * Licensed under the MIT License. + */ package io.lettuce.core.event.connection; import jdk.jfr.Category; @@ -6,7 +12,7 @@ import jdk.jfr.StackTrace; /** - * Flight recorder event variant of {@link ReauthEvent}. + * Flight recorder event variant of {@link ReauthenticationFailedEvent}. * * @author Ivo Gaydajiev * @since 6.6.0 @@ -18,10 +24,18 @@ class JfrReauthFailedEvent extends Event { private final String epId; - public JfrReauthFailedEvent(ReauthenticateFailedEvent event) { + /** + * Create a new {@link JfrReauthFailedEvent} given a {@link ReauthenticationFailedEvent}. + * + * @param event the {@link ReauthenticationFailedEvent} + */ + public JfrReauthFailedEvent(ReauthenticationFailedEvent event) { this.epId = event.getEpId(); } + /** + * @return the connection endpoint ID + */ public String getEpId() { return epId; } diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthenticateEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthenticateEvent.java deleted file mode 100644 index 20d737066..000000000 --- a/src/main/java/io/lettuce/core/event/connection/ReauthenticateEvent.java +++ /dev/null @@ -1,21 +0,0 @@ -package io.lettuce.core.event.connection; - -/** - * Event fired on successfull connection re-authentication. see {@link io.lettuce.core.StreamingCredentialsProvider} - * - * @author Ivo Gaydajiev - * @since 6.6.0 - */ -public class ReauthenticateEvent implements AuthenticateEvent { - - private final String epId; - - public ReauthenticateEvent(String epId) { - this.epId = epId; - } - - public String getEpId() { - return epId; - } - -} diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthenticateFailedEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthenticateFailedEvent.java deleted file mode 100644 index 36d7cc24c..000000000 --- a/src/main/java/io/lettuce/core/event/connection/ReauthenticateFailedEvent.java +++ /dev/null @@ -1,39 +0,0 @@ -package io.lettuce.core.event.connection; - -/** - * Event fired on failed authentication caused either by I/O issues or during connection re-authentication. see - * {@link io.lettuce.core.StreamingCredentialsProvider} - * - * @author Ivo Gaydajiev - * @since 6.6.0 - */ -public class ReauthenticateFailedEvent implements AuthenticateEvent { - - private final String epId; - - private final Throwable cause; - - public ReauthenticateFailedEvent(Throwable cause) { - this.cause = cause; - this.epId = null; - } - - public ReauthenticateFailedEvent(String epId, Throwable cause) { - this.epId = epId; - this.cause = cause; - } - - public String getEpId() { - return epId; - } - - /** - * Returns the {@link Throwable} that describes the reauth failure cause. - * - * @return the {@link Throwable} that describes the reauth failure cause. - */ - public Throwable getCause() { - return cause; - } - -} diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthenticationEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthenticationEvent.java new file mode 100644 index 000000000..745046dd9 --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/ReauthenticationEvent.java @@ -0,0 +1,34 @@ +/* + * Copyright 2024, Redis Ltd. and Contributors + * All rights reserved. + * + * Licensed under the MIT License. + */ +package io.lettuce.core.event.connection; + +/** + * Event fired on successful connection re-authentication + * + * @author Ivo Gaydajiev + * @since 6.6.0 + * @see io.lettuce.core.StreamingCredentialsProvider + */ +public class ReauthenticationEvent implements AuthenticationEvent { + + private final String epId; + + /** + * Create a new {@link ReauthenticationEvent} given a connection endpoint ID + * + * @param epId the connection endpoint ID + */ + public ReauthenticationEvent(String epId) { + this.epId = epId; + } + + @Override + public String getEpId() { + return epId; + } + +} diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthenticationFailedEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthenticationFailedEvent.java new file mode 100644 index 000000000..4ee1e56dd --- /dev/null +++ b/src/main/java/io/lettuce/core/event/connection/ReauthenticationFailedEvent.java @@ -0,0 +1,56 @@ +/* + * Copyright 2024, Redis Ltd. and Contributors + * All rights reserved. + * + * Licensed under the MIT License. + */ +package io.lettuce.core.event.connection; + +/** + * Event fired on failed authentication caused either by I/O issues or during connection re-authentication. + * + * @author Ivo Gaydajiev + * @since 6.6.0 + * @see io.lettuce.core.StreamingCredentialsProvider + */ +public class ReauthenticationFailedEvent implements AuthenticationEvent { + + private final String epId; + + private final Throwable cause; + + /** + * Create a new {@link ReauthenticationFailedEvent} given a {@link Throwable} that describes the re-authentication failure + * cause. + * + * @param cause the {@link Throwable} that describes the re-authentication failure cause. + */ + public ReauthenticationFailedEvent(Throwable cause) { + this(null, cause); + } + + /** + * Create a new {@link ReauthenticationFailedEvent} given a connection endpoint ID and a {@link Throwable} that describes + * the re-authentication failure cause. + * + * @param epId the connection endpoint ID + * @param cause the {@link Throwable} that describes the re-authentication failure cause. + */ + public ReauthenticationFailedEvent(String epId, Throwable cause) { + this.epId = epId; + this.cause = cause; + } + + @Override + public String getEpId() { + return epId; + } + + /** + * @return the {@link Throwable} that describes the re-authentication failure cause. + */ + public Throwable getCause() { + return cause; + } + +} diff --git a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java deleted file mode 100644 index 608c23d0a..000000000 --- a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerTest.java +++ /dev/null @@ -1,135 +0,0 @@ -package io.lettuce.core; - -import io.lettuce.core.api.StatefulRedisConnection; -import io.lettuce.core.event.DefaultEventBus; -import io.lettuce.core.event.EventBus; -import io.lettuce.core.event.connection.ReauthenticateFailedEvent; -import io.lettuce.core.protocol.CommandType; -import io.lettuce.core.protocol.ProtocolVersion; -import io.lettuce.core.protocol.RedisCommand; -import io.lettuce.core.resource.ClientResources; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import reactor.core.scheduler.Schedulers; -import reactor.test.StepVerifier; - -import java.time.Duration; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class RedisAuthenticationHandlerTest { - - private StatefulRedisConnectionImpl connection; - - ClientResources resources; - - EventBus eventBus; - - ConnectionState connectionState; - - @BeforeEach - void setUp() { - eventBus = new DefaultEventBus(Schedulers.immediate()); - connection = mock(StatefulRedisConnectionImpl.class); - resources = mock(ClientResources.class); - when(resources.eventBus()).thenReturn(eventBus); - - connectionState = mock(ConnectionState.class); - when(connection.getResources()).thenReturn(resources); - when(connection.getConnectionState()).thenReturn(connectionState); - } - - @SuppressWarnings("unchecked") - @Test - void subscribeWithStreamingCredentialsProviderInvokesReauth() { - MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); - - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, credentialsProvider, false); - - // Subscribe to the provider - handler.subscribe(); - credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); - - ArgumentCaptor captor = ArgumentCaptor.forClass(RedisCredentials.class); - verify(connection).setCredentials(captor.capture()); - - RedisCredentials credentials = captor.getValue(); - assertThat(credentials.getUsername()).isEqualTo("newuser"); - assertThat(credentials.getPassword()).isEqualTo("newpassword".toCharArray()); - - credentialsProvider.shutdown(); - } - - @Test - void shouldHandleErrorInCredentialsStream() { - MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); - - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, credentialsProvider, false); - - verify(connection, times(0)).dispatch(any(RedisCommand.class)); // No command should be sent - - // Verify the event was published - StepVerifier.create(eventBus.get()).then(() -> { - handler.subscribe(); - credentialsProvider.tryEmitError(new RuntimeException("Test error")); - }).expectNextMatches(event -> event instanceof ReauthenticateFailedEvent).thenCancel().verify(Duration.ofSeconds(1)); - - credentialsProvider.shutdown(); - } - - @Test - void shouldNotSubscribeIfConnectionIsNotSupported() { - StreamingCredentialsProvider credentialsProvider = mock(StreamingCredentialsProvider.class); - when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, credentialsProvider, true); - - // Subscribe to the provider (it should not subscribe due to unsupported connection) - handler.subscribe(); - - // Ensure credentials() was not called - verify(credentialsProvider, times(0)).credentials(); - } - - @Test - void testIsSupportedConnectionWithRESP2ProtocolOnPubSubConnection() { - - when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), - true); - - assertFalse(handler.isSupportedConnection()); - } - - @Test - void testIsSupportedConnectionWithNonPubSubConnection() { - - when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); - - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), - false); - - assertTrue(handler.isSupportedConnection()); - } - - @Test - void testIsSupportedConnectionWithRESP3ProtocolOnPubSubConnection() { - - when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP3); - - RedisAuthenticationHandler handler = new RedisAuthenticationHandler(connection, mock(RedisCredentialsProvider.class), - true); - - assertTrue(handler.isSupportedConnection()); - } - -} diff --git a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerUnitTests.java b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerUnitTests.java new file mode 100644 index 000000000..6b001aabc --- /dev/null +++ b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerUnitTests.java @@ -0,0 +1,210 @@ +package io.lettuce.core; + +import io.lettuce.core.codec.StringCodec; +import io.lettuce.core.event.DefaultEventBus; +import io.lettuce.core.event.EventBus; +import io.lettuce.core.event.connection.ReauthenticationFailedEvent; +import io.lettuce.core.protocol.AsyncCommand; +import io.lettuce.core.protocol.CommandType; +import io.lettuce.core.protocol.ProtocolVersion; +import io.lettuce.core.protocol.RedisCommand; +import io.lettuce.core.resource.ClientResources; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatcher; +import reactor.core.scheduler.Schedulers; +import reactor.test.StepVerifier; + +import java.time.Duration; + +import static io.lettuce.TestTags.UNIT_TEST; +import static io.lettuce.core.protocol.CommandType.AUTH; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for the {@link RedisAuthenticationHandler} + */ +@Tag(UNIT_TEST) +public class RedisAuthenticationHandlerUnitTests { + + private StatefulRedisConnectionImpl connection; + + ClientResources resources; + + EventBus eventBus; + + ConnectionState connectionState; + + @BeforeEach + void setUp() { + eventBus = new DefaultEventBus(Schedulers.immediate()); + connection = mock(StatefulRedisConnectionImpl.class); + resources = mock(ClientResources.class); + when(resources.eventBus()).thenReturn(eventBus); + + connectionState = mock(ConnectionState.class); + when(connection.getResources()).thenReturn(resources); + when(connection.getCodec()).thenReturn(StringCodec.UTF8); + when(connection.getConnectionState()).thenReturn(connectionState); + } + + @SuppressWarnings("unchecked") + @Test + void subscribeWithStreamingCredentialsProviderInvokesReauth() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler<>(connection, credentialsProvider, + false); + + // Subscribe to the provider + handler.subscribe(); + credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); + + ArgumentCaptor> captor = ArgumentCaptor.forClass(AsyncCommand.class); + verify(connection).dispatch(captor.capture()); + + AsyncCommand credentialsCommand = captor.getValue(); + assertThat(credentialsCommand.getType()).isEqualTo(AUTH); + assertThat(credentialsCommand.getArgs().count()).isEqualTo(2); + assertThat(credentialsCommand.getArgs().toCommandString()).isEqualTo("newuser" + " " + "newpassword"); + + credentialsProvider.shutdown(); + } + + @Test + void shouldHandleErrorInCredentialsStream() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler<>(connection, credentialsProvider, false); + + verify(connection, times(0)).dispatch(any(RedisCommand.class)); // No command should be sent + + // Verify the event was published + StepVerifier.create(eventBus.get()).then(() -> { + handler.subscribe(); + credentialsProvider.tryEmitError(new RuntimeException("Test error")); + }).expectNextMatches(event -> event instanceof ReauthenticationFailedEvent).thenCancel().verify(Duration.ofSeconds(1)); + + credentialsProvider.shutdown(); + } + + @Test + void shouldNotSubscribeIfConnectionIsNotSupported() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler<>(connection, credentialsProvider, true); + + // Subscribe to the provider (it should not subscribe due to unsupported connection) + handler.subscribe(); + credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); + + // Ensure credentials() was not called + verify(connection, times(0)).dispatch(any(RedisCommand.class)); // No command should be sent + } + + @Test + void testIsSupportedConnectionWithRESP2ProtocolOnPubSubConnection() { + + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler<>(connection, + mock(RedisCredentialsProvider.class), true); + + assertFalse(handler.isSupportedConnection()); + } + + @Test + void testIsSupportedConnectionWithNonPubSubConnection() { + + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP2); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler<>(connection, + mock(RedisCredentialsProvider.class), false); + + assertTrue(handler.isSupportedConnection()); + } + + @Test + void testIsSupportedConnectionWithRESP3ProtocolOnPubSubConnection() { + + when(connectionState.getNegotiatedProtocolVersion()).thenReturn(ProtocolVersion.RESP3); + + RedisAuthenticationHandler handler = new RedisAuthenticationHandler<>(connection, + mock(RedisCredentialsProvider.class), true); + + assertTrue(handler.isSupportedConnection()); + } + + @Test + public void testSetCredentialsWhenCredentialsAreNull() { + RedisAuthenticationHandler handler = new RedisAuthenticationHandler<>(connection, + mock(RedisCredentialsProvider.class), false); + + handler.setCredentials(null); + + verify(connection, times(0)).dispatch(any(RedisCommand.class)); // No command should be sent + } + + @Test + void testSetCredentialsDoesNotDispatchAuthIfInTransaction() { + MyStreamingRedisCredentialsProvider credentialsProvider = new MyStreamingRedisCredentialsProvider(); + RedisAuthenticationHandler handler = new RedisAuthenticationHandler<>(connection, credentialsProvider, false); + + // Subscribe to the provider + handler.subscribe(); + + // Indicate a transaction is ongoing + handler.startTransaction(); + + // Attempt to authenticate + credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); + + // verify that the AUTH command was not sent + verify(connection, times(0)).dispatch(any(RedisCommand.class)); + + // Indicate a transaction is ongoing + handler.endTransaction(); + + ArgumentCaptor> captor = ArgumentCaptor.forClass(AsyncCommand.class); + verify(connection).dispatch(captor.capture()); + + AsyncCommand credentialsCommand = captor.getValue(); + assertThat(credentialsCommand.getType()).isEqualTo(AUTH); + assertThat(credentialsCommand.getArgs().count()).isEqualTo(2); + assertThat(credentialsCommand.getArgs().toCommandString()).isEqualTo("newuser" + " " + "newpassword"); + } + + public static ArgumentMatcher> isAuthCommand(String expectedUsername, + String expectedPassword) { + return new ArgumentMatcher>() { + + @Override + public boolean matches(RedisCommand command) { + if (command.getType() != CommandType.AUTH) { + return false; + } + + // Retrieve arguments (adjust based on your RedisCommand implementation) + return command.getArgs().toCommandString().equals(expectedUsername + " " + expectedPassword); + } + + @Override + public String toString() { + return String.format("Expected AUTH command with username=%s and password=%s", expectedUsername, + expectedPassword); + } + + }; + } + +} diff --git a/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java b/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java deleted file mode 100644 index d418c4251..000000000 --- a/src/test/java/io/lettuce/core/StatefulRedisConnectionImplUnitTests.java +++ /dev/null @@ -1,147 +0,0 @@ -package io.lettuce.core; - -import io.lettuce.core.codec.StringCodec; -import io.lettuce.core.protocol.AsyncCommand; -import io.lettuce.core.protocol.CommandType; -import io.lettuce.core.protocol.PushHandler; -import io.lettuce.core.protocol.RedisCommand; -import io.lettuce.core.resource.ClientResources; -import io.lettuce.core.tracing.Tracing; -import io.lettuce.test.ReflectionTestUtils; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.ArgumentMatcher; -import org.mockito.ArgumentMatchers; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; -import org.mockito.junit.jupiter.MockitoSettings; -import org.mockito.quality.Strictness; - -import java.lang.reflect.Field; -import java.time.Duration; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.argThat; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@ExtendWith(MockitoExtension.class) -@MockitoSettings(strictness = Strictness.LENIENT) -public class StatefulRedisConnectionImplUnitTests extends TestSupport { - - RedisCommandBuilder commandBuilder = new RedisCommandBuilder<>(StringCodec.UTF8); - - StatefulRedisConnectionImpl connection; - - @Mock - PushHandler pushHandler; - - @Mock - RedisChannelWriter writer; - - @Mock - ClientResources clientResources; - - @Mock - Tracing tracing; - - @BeforeEach - void setup() throws NoSuchFieldException, IllegalAccessException { - when(writer.getClientResources()).thenReturn(clientResources); - when(clientResources.tracing()).thenReturn(tracing); - when(tracing.isEnabled()).thenReturn(false); - - Field asyncField = StatefulRedisConnectionImpl.class.getDeclaredField("async"); - asyncField.setAccessible(true); - - connection = new StatefulRedisConnectionImpl<>(writer, pushHandler, StringCodec.UTF8, Duration.ofSeconds(1)); - } - - @Test - public void testSetCredentialsWhenCredentialsAreNull() { - connection.setCredentials(null); - - verify(writer, never()).write(ArgumentMatchers.> any()); - } - - @Test - void testSetCredentialsDispatchesAuthWhenNotInTransaction() { - connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); - verify(writer).write(argThat(isAuthCommand("user", "pass"))); - } - - @Test - void testSetCredentialsDoesNotDispatchAuthIfInTransaction() { - AtomicBoolean inTransaction = ReflectionTestUtils.getField(connection, "inTransaction"); - inTransaction.set(true); - - connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); - - verify(writer, never()).write(ArgumentMatchers.> any()); - } - - @Test - void testSetCredentialsDispatchesAuthAfterTransaction() { - AtomicBoolean inTransaction = ReflectionTestUtils.getField(connection, "inTransaction"); - - connection.dispatch(commandBuilder.multi()); - assertThat(inTransaction.get()).isTrue(); - - connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); - connection.dispatch(commandBuilder.discard()); - - assertThat(inTransaction.get()).isFalse(); - - verify(writer).write(argThat(isAuthCommand("user", "pass"))); - } - - @Test - void testSetCredentialsDispatchesAuthAfterTransactionInAnotherThread() throws InterruptedException { - AtomicBoolean inTransaction = ReflectionTestUtils.getField(connection, "inTransaction"); - - connection.dispatch(commandBuilder.multi()); - assertThat(inTransaction.get()).isTrue(); - - Thread thread = new Thread(() -> { - connection.setCredentials(new StaticRedisCredentials("user", "pass".toCharArray())); - }); - thread.start(); - - connection.dispatch(commandBuilder.discard()); - - thread.join(); - - assertThat(inTransaction.get()).isFalse(); - verify(writer).write(argThat(isAuthCommand("user", "pass"))); - } - - public static ArgumentMatcher> isAuthCommand(String expectedUsername, - String expectedPassword) { - return new ArgumentMatcher>() { - - @Override - public boolean matches(RedisCommand command) { - if (command.getType() != CommandType.AUTH) { - return false; - } - - // Retrieve arguments (adjust based on your RedisCommand implementation) - return command.getArgs().toCommandString().equals(expectedUsername + " " + expectedPassword); - } - - @Override - public String toString() { - return String.format("Expected AUTH command with username=%s and password=%s", expectedUsername, - expectedPassword); - } - - }; - } - -} diff --git a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java index 28dab306b..12bbbedda 100644 --- a/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java +++ b/src/test/java/io/lettuce/core/event/ConnectionEventsTriggeredIntegrationTests.java @@ -8,9 +8,9 @@ import io.lettuce.core.ClientOptions; import io.lettuce.core.MyStreamingRedisCredentialsProvider; -import io.lettuce.core.event.connection.AuthenticateEvent; -import io.lettuce.core.event.connection.ReauthenticateEvent; -import io.lettuce.core.event.connection.ReauthenticateFailedEvent; +import io.lettuce.core.event.connection.AuthenticationEvent; +import io.lettuce.core.event.connection.ReauthenticationEvent; +import io.lettuce.core.event.connection.ReauthenticationFailedEvent; import io.lettuce.test.LettuceExtension; import io.lettuce.test.WithPassword; import io.lettuce.test.settings.TestSettings; @@ -28,8 +28,6 @@ import io.lettuce.test.resource.FastShutdown; import io.lettuce.test.resource.TestClientResources; -import javax.inject.Inject; - /** * @author Mark Paluch * @author Ivo Gaydajiev @@ -66,16 +64,16 @@ void testReauthenticateEvents() { .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); RedisURI uri = RedisURI.Builder.redis(host, port).withAuthentication(credentialsProvider).build(); - Flux publisher = client.getResources().eventBus().get() - .filter(event -> event instanceof AuthenticateEvent).cast(AuthenticateEvent.class); + Flux publisher = client.getResources().eventBus().get() + .filter(event -> event instanceof AuthenticationEvent).cast(AuthenticationEvent.class); WithPassword.run(client, () -> StepVerifier.create(publisher).then(() -> client.connect(uri)) - .assertNext(event -> assertThat(event).asInstanceOf(InstanceOfAssertFactories.type(ReauthenticateEvent.class)) - .extracting(ReauthenticateEvent::getEpId).isNotNull()) + .assertNext(event -> assertThat(event).asInstanceOf(InstanceOfAssertFactories.type(ReauthenticationEvent.class)) + .extracting(ReauthenticationEvent::getEpId).isNotNull()) .then(() -> credentialsProvider.emitCredentials(TestSettings.username(), "invalid".toCharArray())) - .assertNext( - event -> assertThat(event).asInstanceOf(InstanceOfAssertFactories.type(ReauthenticateFailedEvent.class)) - .extracting(ReauthenticateFailedEvent::getEpId).isNotNull()) + .assertNext(event -> assertThat(event) + .asInstanceOf(InstanceOfAssertFactories.type(ReauthenticationFailedEvent.class)) + .extracting(ReauthenticationFailedEvent::getEpId).isNotNull()) .thenCancel().verify(Duration.of(1, ChronoUnit.SECONDS))); FastShutdown.shutdown(client); From 8e9ab4853cfd25a56c991eb789ca9d1e15c21e55 Mon Sep 17 00:00:00 2001 From: ggivo Date: Tue, 17 Dec 2024 17:16:07 +0200 Subject: [PATCH 17/28] fix inTransaction lock with dispatch command batch --- .../core/RedisAuthenticationHandler.java | 23 +++++++++++++++- .../core/StatefulRedisConnectionImpl.java | 26 +++++++++++++------ .../RedisAuthenticationHandlerUnitTests.java | 8 ++++-- 3 files changed, 46 insertions(+), 11 deletions(-) diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index cbdc12c8a..b2467002e 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -25,6 +25,7 @@ import reactor.core.Disposable; import reactor.core.publisher.Flux; +import java.util.Collection; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; @@ -32,6 +33,7 @@ import static io.lettuce.core.protocol.CommandType.AUTH; import static io.lettuce.core.protocol.CommandType.DISCARD; import static io.lettuce.core.protocol.CommandType.EXEC; +import static io.lettuce.core.protocol.CommandType.MULTI; /** * Redis authentication handler. Internally used to authenticate a Redis connection. This class is part of the internal API. @@ -189,6 +191,25 @@ public void postProcess(RedisCommand toSend) { } } + public void postProcess(Collection> dispatched) { + Boolean transactionComplete = null; + for (RedisCommand command : dispatched) { + if (command.getType() == EXEC || command.getType() == DISCARD) { + transactionComplete = true; + } + if (command.getType() == MULTI) { + transactionComplete = false; + } + } + + if (transactionComplete != null) { + if (transactionComplete) { + inTransaction.set(false); + setCredentials(credentialsRef.getAndSet(null)); + } + } + } + /** * Marks that the current connection has started a transaction. *

@@ -257,7 +278,7 @@ protected void dispatchAuth(RedisCredentials credentials) { } // dispatch directly to avoid AUTH preprocessing overrides credentials provider - RedisCommand auth = connection.dispatch(authCommand(credentials)); + RedisCommand auth = connection.getChannelWriter().write(authCommand(credentials)); if (auth instanceof CompleteableCommand) { ((CompleteableCommand) auth).onComplete((status, throwable) -> { if (throwable != null) { diff --git a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java index 58d5b36c8..b51ee8ffa 100644 --- a/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java +++ b/src/main/java/io/lettuce/core/StatefulRedisConnectionImpl.java @@ -192,17 +192,16 @@ public RedisCommand dispatch(RedisCommand command) { @Override public Collection> dispatch(Collection> commands) { - List> sentCommands = new ArrayList<>(commands.size()); + Collection> sentCommands = preProcessCommands(commands); - commands.forEach(o -> { - RedisCommand preprocessed = preProcessCommand(o); - sentCommands.add(preprocessed); - }); + Collection> dispatchedCommands = super.dispatch(sentCommands); - super.dispatch(sentCommands); + return this.postProcessCommands(dispatchedCommands); + } - sentCommands.forEach(this::postProcessCommand); - return sentCommands; + protected Collection> postProcessCommands(Collection> commands) { + authHandler.postProcess(commands); + return commands; } protected RedisCommand postProcessCommand(RedisCommand command) { @@ -210,6 +209,17 @@ protected RedisCommand postProcessCommand(RedisCommand com return command; } + protected Collection> preProcessCommands(Collection> commands) { + List> sentCommands = new ArrayList<>(commands.size()); + + commands.forEach(o -> { + RedisCommand preprocessed = preProcessCommand(o); + sentCommands.add(preprocessed); + }); + + return sentCommands; + } + // TODO [tihomir.mateev] Refactor to include as part of the Command interface // All these if statements clearly indicate this is a problem best solve by each command // (defining a pre and post processing behaviour of the command) diff --git a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerUnitTests.java b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerUnitTests.java index 6b001aabc..07cea7a15 100644 --- a/src/test/java/io/lettuce/core/RedisAuthenticationHandlerUnitTests.java +++ b/src/test/java/io/lettuce/core/RedisAuthenticationHandlerUnitTests.java @@ -38,6 +38,8 @@ public class RedisAuthenticationHandlerUnitTests { private StatefulRedisConnectionImpl connection; + RedisChannelWriter writer; + ClientResources resources; EventBus eventBus; @@ -47,6 +49,7 @@ public class RedisAuthenticationHandlerUnitTests { @BeforeEach void setUp() { eventBus = new DefaultEventBus(Schedulers.immediate()); + writer = mock(RedisChannelWriter.class); connection = mock(StatefulRedisConnectionImpl.class); resources = mock(ClientResources.class); when(resources.eventBus()).thenReturn(eventBus); @@ -55,6 +58,7 @@ void setUp() { when(connection.getResources()).thenReturn(resources); when(connection.getCodec()).thenReturn(StringCodec.UTF8); when(connection.getConnectionState()).thenReturn(connectionState); + when(connection.getChannelWriter()).thenReturn(writer); } @SuppressWarnings("unchecked") @@ -70,7 +74,7 @@ void subscribeWithStreamingCredentialsProviderInvokesReauth() { credentialsProvider.emitCredentials("newuser", "newpassword".toCharArray()); ArgumentCaptor> captor = ArgumentCaptor.forClass(AsyncCommand.class); - verify(connection).dispatch(captor.capture()); + verify(writer).write(captor.capture()); AsyncCommand credentialsCommand = captor.getValue(); assertThat(credentialsCommand.getType()).isEqualTo(AUTH); @@ -176,7 +180,7 @@ void testSetCredentialsDoesNotDispatchAuthIfInTransaction() { handler.endTransaction(); ArgumentCaptor> captor = ArgumentCaptor.forClass(AsyncCommand.class); - verify(connection).dispatch(captor.capture()); + verify(writer).write(captor.capture()); AsyncCommand credentialsCommand = captor.getValue(); assertThat(credentialsCommand.getType()).isEqualTo(AUTH); From 746dd82e701366be4cfc932d1f960adc8bf9a1d4 Mon Sep 17 00:00:00 2001 From: ggivo Date: Wed, 18 Dec 2024 10:54:42 +0200 Subject: [PATCH 18/28] Remove StreamingCredentialsProvider interface. move credentials() method to RedisCredentialsProvider. Resolve issue with unsafe cast after extending RedisCredentialsProvider with supportsStreaming() method --- .../java/io/lettuce/core/ClientOptions.java | 22 +++++++++------- .../core/RedisAuthenticationHandler.java | 6 ++--- .../core/RedisCredentialsProvider.java | 18 +++++++++++++ .../core/StreamingCredentialsProvider.java | 26 ------------------- .../connection/ReauthenticationEvent.java | 2 +- .../ReauthenticationFailedEvent.java | 2 +- .../MyStreamingRedisCredentialsProvider.java | 7 ++++- 7 files changed, 42 insertions(+), 41 deletions(-) delete mode 100644 src/main/java/io/lettuce/core/StreamingCredentialsProvider.java diff --git a/src/main/java/io/lettuce/core/ClientOptions.java b/src/main/java/io/lettuce/core/ClientOptions.java index 1248800a5..3fd635e4e 100644 --- a/src/main/java/io/lettuce/core/ClientOptions.java +++ b/src/main/java/io/lettuce/core/ClientOptions.java @@ -740,8 +740,8 @@ public TimeoutOptions getTimeoutOptions() { /** * Defines the re-authentication behavior of the Redis client. *

- * Certain implementations of the {@link RedisCredentialsProvider} such as the {@link StreamingCredentialsProvider} could - * emit new credentials at runtime. This setting controls how the driver reacts to these newly emitted credentials. + * Certain implementations of the {@link RedisCredentialsProvider} could emit new credentials at runtime. This setting + * controls how the driver reacts to these newly emitted credentials. */ public enum ReauthenticateBehavior { @@ -750,19 +750,23 @@ public enum ReauthenticateBehavior { * {@link RedisCredentialsProvider} only when the driver needs to, e.g. when the connection is first established or when * it is re-established after a disconnect. *

- * No re-authentication is performed when new credentials are emitted by the {@link StreamingCredentialsProvider} . + *

+ * No re-authentication is performed when new credentials are emitted by a {@link RedisCredentialsProvider} that + * supports streaming. The client does not subscribe to or react to any updates in the credential stream provided by + * {@link RedisCredentialsProvider#credentials()}. + *

*/ DEFAULT, /** - * Automatically triggers re-authentication whenever new credentials are emitted by any implementation of the - * {@link StreamingCredentialsProvider} interface. + * Automatically triggers re-authentication whenever new credentials are emitted by a {@link RedisCredentialsProvider} + * that supports streaming, as indicated by {@link RedisCredentialsProvider#supportsStreaming()}. * *

- * When enabled, the client subscribes to the credential stream provided by the {@link StreamingCredentialsProvider} and - * issues an {@code AUTH} command to the Redis server each time new credentials are received. This behavior supports - * dynamic credential scenarios, such as token-based authentication, or credential rotation where credentials are - * refreshed periodically to maintain access. + * When this behavior is enabled, the client subscribes to the credential stream provided by + * {@link RedisCredentialsProvider#credentials()} and issues an {@code AUTH} command to the Redis server each time new + * credentials are received. This behavior supports dynamic credential scenarios, such as token-based authentication, or + * credential rotation where credentials are refreshed periodically to maintain access. *

* *

diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index b2467002e..7a05a7a60 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -83,7 +83,7 @@ public RedisAuthenticationHandler(StatefulRedisConnectionImpl connection, * @return a new {@link RedisAuthenticationHandler} if the connection supports re-authentication, otherwise an * implementation of the {@link RedisAuthenticationHandler} that does nothing * @since 6.6.0 - * @see StreamingCredentialsProvider + * @see RedisCredentialsProvider */ public static RedisAuthenticationHandler createHandler(StatefulRedisConnectionImpl connection, RedisCredentialsProvider credentialsProvider, Boolean isPubSubConnection, ClientOptions options) { @@ -102,7 +102,7 @@ public static RedisAuthenticationHandler createHandler(StatefulRedi * * @return a new {@link RedisAuthenticationHandler} * @since 6.6.0 - * @see StreamingCredentialsProvider + * @see RedisCredentialsProvider */ public static RedisAuthenticationHandler createDefaultAuthenticationHandler() { return new DisabledAuthenticationHandler<>(); @@ -123,7 +123,7 @@ public void subscribe() { return; } - Flux credentialsFlux = ((StreamingCredentialsProvider) credentialsProvider).credentials(); + Flux credentialsFlux = credentialsProvider.credentials(); Disposable subscription = credentialsFlux.subscribe(this::onNext, this::onError, this::complete); diff --git a/src/main/java/io/lettuce/core/RedisCredentialsProvider.java b/src/main/java/io/lettuce/core/RedisCredentialsProvider.java index 556483edf..9c57a280a 100644 --- a/src/main/java/io/lettuce/core/RedisCredentialsProvider.java +++ b/src/main/java/io/lettuce/core/RedisCredentialsProvider.java @@ -2,6 +2,7 @@ import java.util.function.Supplier; +import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import io.lettuce.core.internal.LettuceAssert; @@ -52,6 +53,23 @@ default boolean supportsStreaming() { return false; } + /** + * Returns a {@link Flux} emitting {@link RedisCredentials} that can be used to authorize a Redis connection. + * + * For implementations that support streaming credentials (as indicated by {@link #supportsStreaming()} returning + * {@code true}), this method can emit multiple credentials over time, typically based on external events like token renewal + * or rotation. + * + * For implementations that do not support streaming credentials (where {@link #supportsStreaming()} returns {@code false}), + * this method throws an {@link UnsupportedOperationException} by default. + * + * @return a {@link Flux} emitting {@link RedisCredentials}, or throws an exception if streaming is not supported. + * @throws UnsupportedOperationException if the provider does not support streaming credentials. + */ + default Flux credentials() { + throw new UnsupportedOperationException("Streaming credentials are not supported by this provider."); + } + /** * Extension to {@link RedisCredentialsProvider} that resolves credentials immediately without the need to defer the * credential resolution. diff --git a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java b/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java deleted file mode 100644 index 77a131f37..000000000 --- a/src/main/java/io/lettuce/core/StreamingCredentialsProvider.java +++ /dev/null @@ -1,26 +0,0 @@ -package io.lettuce.core; - -import reactor.core.publisher.Flux; - -/** - * A provider for streaming credentials that can be used to authorize a Redis connection and re-authenticate the connection when - * new credentials are received. - * - * @author Ivo Gaydajiev - * @since 6.6.0 - */ -public interface StreamingCredentialsProvider extends RedisCredentialsProvider { - - /** - * Returns a {@link Flux} emitting {@link RedisCredentials} that can be used to authorize a Redis connection. This - * credential provider supports streaming credentials, meaning that it can emit multiple credentials over time. - * - * @return - */ - Flux credentials(); - - default boolean supportsStreaming() { - return true; - } - -} diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthenticationEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthenticationEvent.java index 745046dd9..bb9ab0bc0 100644 --- a/src/main/java/io/lettuce/core/event/connection/ReauthenticationEvent.java +++ b/src/main/java/io/lettuce/core/event/connection/ReauthenticationEvent.java @@ -11,7 +11,7 @@ * * @author Ivo Gaydajiev * @since 6.6.0 - * @see io.lettuce.core.StreamingCredentialsProvider + * @see io.lettuce.core.RedisCredentialsProvider */ public class ReauthenticationEvent implements AuthenticationEvent { diff --git a/src/main/java/io/lettuce/core/event/connection/ReauthenticationFailedEvent.java b/src/main/java/io/lettuce/core/event/connection/ReauthenticationFailedEvent.java index 4ee1e56dd..79164aa96 100644 --- a/src/main/java/io/lettuce/core/event/connection/ReauthenticationFailedEvent.java +++ b/src/main/java/io/lettuce/core/event/connection/ReauthenticationFailedEvent.java @@ -11,7 +11,7 @@ * * @author Ivo Gaydajiev * @since 6.6.0 - * @see io.lettuce.core.StreamingCredentialsProvider + * @see io.lettuce.core.RedisCredentialsProvider */ public class ReauthenticationFailedEvent implements AuthenticationEvent { diff --git a/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java b/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java index e5b0eaa93..12e9e37d1 100644 --- a/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java +++ b/src/test/java/io/lettuce/core/MyStreamingRedisCredentialsProvider.java @@ -10,10 +10,15 @@ * @author Ivo Gaydajiev * @since 6.6.0 */ -public class MyStreamingRedisCredentialsProvider implements StreamingCredentialsProvider { +public class MyStreamingRedisCredentialsProvider implements RedisCredentialsProvider { private final Sinks.Many credentialsSink = Sinks.many().replay().latest(); + @Override + public boolean supportsStreaming() { + return true; + } + @Override public Mono resolveCredentials() { From 31341f161817091c0beaf9eb82a4bc3359ff1cd6 Mon Sep 17 00:00:00 2001 From: ggivo Date: Wed, 18 Dec 2024 11:22:47 +0200 Subject: [PATCH 19/28] Add authentication handler to ClusterPubSub connections --- .../core/RedisAuthenticationHandler.java | 6 +++++ .../core/cluster/RedisClusterClient.java | 2 ++ .../RedisClientConnectIntegrationTests.java | 18 +++++++++++++++ .../ClusterClientOptionsIntegrationTests.java | 23 +++++++++++++++++++ 4 files changed, 49 insertions(+) diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index 7a05a7a60..5a3733381 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -89,6 +89,12 @@ public static RedisAuthenticationHandler createHandler(StatefulRedi RedisCredentialsProvider credentialsProvider, Boolean isPubSubConnection, ClientOptions options) { if (isSupported(options)) { + + if (isPubSubConnection && options.getConfiguredProtocolVersion() == ProtocolVersion.RESP2) { + throw new RedisConnectionException( + "Renewable credentials are not supported with RESP2 protocol on a pub/sub connection."); + } + return new RedisAuthenticationHandler<>(connection, credentialsProvider, isPubSubConnection); } diff --git a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java index e62fbc731..e9e8a2ede 100644 --- a/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java +++ b/src/main/java/io/lettuce/core/cluster/RedisClusterClient.java @@ -780,6 +780,8 @@ private CompletableFuture> con clusterWriter.setClusterConnectionProvider(pooledClusterConnectionProvider); connection.setPartitions(partitions); + connection.setAuthenticationHandler( + createHandler(connection, getFirstUri().getCredentialsProvider(), true, getOptions())); Supplier commandHandlerSupplier = () -> new PubSubCommandHandler<>(getClusterClientOptions(), getResources(), codec, endpoint); diff --git a/src/test/java/io/lettuce/core/RedisClientConnectIntegrationTests.java b/src/test/java/io/lettuce/core/RedisClientConnectIntegrationTests.java index 4e7c281e4..416ffa3a4 100644 --- a/src/test/java/io/lettuce/core/RedisClientConnectIntegrationTests.java +++ b/src/test/java/io/lettuce/core/RedisClientConnectIntegrationTests.java @@ -32,6 +32,7 @@ import javax.inject.Inject; +import io.lettuce.core.protocol.ProtocolVersion; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; @@ -219,6 +220,23 @@ void connectPubSubCodecSentinelMissingHostAndSocketUri() { assertThatThrownBy(() -> client.connectPubSub(UTF8, invalidSentinel())).isInstanceOf(IllegalArgumentException.class); } + @Test + void connectPubSubAsyncReauthNotSupportedWithRESP2() { + ClientOptions.ReauthenticateBehavior reauth = client.getOptions().getReauthenticateBehaviour(); + ProtocolVersion protocolVersion = client.getOptions().getConfiguredProtocolVersion(); + try { + client.setOptions(client.getOptions().mutate().protocolVersion(ProtocolVersion.RESP2) + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); + + RedisURI redisURI = redis(host, port).build(); + assertThatThrownBy(() -> client.connectPubSubAsync(UTF8, redisURI)).isInstanceOf(RedisConnectionException.class); + + } finally { + client.setOptions( + client.getOptions().mutate().protocolVersion(protocolVersion).reauthenticateBehavior(reauth).build()); + } + } + /* * Sentinel Stateful */ diff --git a/src/test/java/io/lettuce/core/cluster/ClusterClientOptionsIntegrationTests.java b/src/test/java/io/lettuce/core/cluster/ClusterClientOptionsIntegrationTests.java index 6eddfa2e0..94cbbe76b 100644 --- a/src/test/java/io/lettuce/core/cluster/ClusterClientOptionsIntegrationTests.java +++ b/src/test/java/io/lettuce/core/cluster/ClusterClientOptionsIntegrationTests.java @@ -1,6 +1,7 @@ package io.lettuce.core.cluster; import static io.lettuce.TestTags.INTEGRATION_TEST; +import static io.lettuce.core.codec.StringCodec.UTF8; import static org.assertj.core.api.Assertions.*; import java.time.Duration; @@ -8,6 +9,9 @@ import javax.inject.Inject; +import io.lettuce.core.ClientOptions; +import io.lettuce.core.RedisConnectionException; +import io.lettuce.core.protocol.ProtocolVersion; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -80,4 +84,23 @@ void shouldApplyTimeoutOptionsToPubSubClusterConnection() throws InterruptedExce Thread.sleep(300); } + @Test + void connectPubSubAsyncReauthNotSupportedWithRESP2() { + + ClientOptions.ReauthenticateBehavior reauth = clusterClient.getClusterClientOptions().getReauthenticateBehaviour(); + ProtocolVersion protocolVersion = clusterClient.getClusterClientOptions().getConfiguredProtocolVersion(); + + try { + clusterClient.setOptions(clusterClient.getClusterClientOptions().mutate().protocolVersion(ProtocolVersion.RESP2) + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); + assertThatThrownBy(() -> clusterClient.connectPubSub(UTF8)).isInstanceOf(RedisConnectionException.class); + + } finally { + + clusterClient.setOptions(clusterClient.getClusterClientOptions().mutate().protocolVersion(protocolVersion) + .reauthenticateBehavior(reauth).build()); + } + + } + } From 77c5058d382da0b4a0d07c34d5221adf415eb324 Mon Sep 17 00:00:00 2001 From: ggivo Date: Mon, 2 Dec 2024 18:13:11 +0200 Subject: [PATCH 20/28] Token based auth integration with core extension Provide a way for lettuce clients to use token-based authentication. TOKENs come with a TTL. After a Redis client authenticates with a TOKEN, if they didn't renew their authentication we need to evict (close) them. The suggested approach is to leverage the existing CredentialsProvider and add support for streaming credentials to handle token refresh scenarios. Each time a new token is received connection is reauthenticated. --- pom.xml | 35 ++++- .../TokenBasedRedisCredentialsProvider.java | 102 ++++++++++++ ...okenBasedRedisCredentialsProviderTest.java | 146 ++++++++++++++++++ .../core/AuthenticationIntegrationTests.java | 46 ++++++ .../io/lettuce/core/TestTokenManager.java | 50 ++++++ ...gCredentialsProviderlIntegrationTests.java | 1 - .../examples/TokenBasedAuthExample.java | 136 ++++++++++++++++ 7 files changed, 513 insertions(+), 3 deletions(-) create mode 100644 src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java create mode 100644 src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java create mode 100644 src/test/java/io/lettuce/core/TestTokenManager.java create mode 100644 src/test/java/io/lettuce/examples/TokenBasedAuthExample.java diff --git a/pom.xml b/pom.xml index e92ed704e..e90411a01 100644 --- a/pom.xml +++ b/pom.xml @@ -89,6 +89,19 @@ HEAD + + + sonatype-snapshots + https://oss.sonatype.org/content/repositories/snapshots/ + + false + + + true + + + + ossrh @@ -173,12 +186,30 @@ pom import - + + redis.clients.authentication + redis-authx-core + 0.1.0-SNAPSHOT + + + redis.clients.authentication + redis-authx-entraid + 0.1.0-SNAPSHOT + test + - + + redis.clients.authentication + redis-authx-core + + + redis.clients.authentication + redis-authx-entraid + test + diff --git a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java new file mode 100644 index 000000000..ec8a36372 --- /dev/null +++ b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java @@ -0,0 +1,102 @@ +package io.lettuce.authx; + +import io.lettuce.core.RedisCredentials; +import io.lettuce.core.StreamingCredentialsProvider; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.core.publisher.Sinks; +import redis.clients.authentication.core.Token; +import redis.clients.authentication.core.TokenAuthConfig; +import redis.clients.authentication.core.TokenListener; +import redis.clients.authentication.core.TokenManager; + +public class TokenBasedRedisCredentialsProvider implements StreamingCredentialsProvider { + + private final TokenManager tokenManager; + + private final Sinks.Many credentialsSink = Sinks.many().replay().latest(); + + public TokenBasedRedisCredentialsProvider(TokenAuthConfig tokenAuthConfig) { + this(new TokenManager(tokenAuthConfig.getIdentityProviderConfig().getProvider(), + tokenAuthConfig.getTokenManagerConfig())); + + } + + public TokenBasedRedisCredentialsProvider(TokenManager tokenManager) { + this.tokenManager = tokenManager; + initializeTokenManager(); + } + + /** + * Initialize the TokenManager and subscribe to token renewal events. + */ + private void initializeTokenManager() { + TokenListener listener = new TokenListener() { + + @Override + public void onTokenRenewed(Token token) { + try { + String username = token.tryGet("oid"); + char[] pass = token.getValue().toCharArray(); + RedisCredentials credentials = RedisCredentials.just(username, pass); + credentialsSink.tryEmitNext(credentials); + } catch (Exception e) { + credentialsSink.emitError(e, Sinks.EmitFailureHandler.FAIL_FAST); + } + } + + @Override + public void onError(Exception exception) { + credentialsSink.tryEmitError(exception); + } + + }; + + try { + tokenManager.start(listener, false); + } catch (Exception e) { + credentialsSink.tryEmitError(e); + } + } + + /** + * Resolve the latest available credentials as a Mono. + *

+ * This method returns a Mono that emits the most recent set of Redis credentials. The Mono will complete once the + * credentials are emitted. If no credentials are available at the time of subscription, the Mono will wait until + * credentials are available. + * + * @return a Mono that emits the latest Redis credentials + */ + @Override + public Mono resolveCredentials() { + + return credentialsSink.asFlux().next(); + } + + /** + * Expose the Flux for all credential updates. + *

+ * This method returns a Flux that emits all updates to the Redis credentials. Subscribers will receive the latest + * credentials whenever they are updated. The Flux will continue to emit updates until the provider is shut down. + * + * @return a Flux that emits all updates to the Redis credentials + */ + @Override + public Flux credentials() { + + return credentialsSink.asFlux().onBackpressureLatest(); // Provide a continuous stream of credentials + } + + /** + * Stop the credentials provider and clean up resources. + *

+ * This method stops the TokenManager and completes the credentials sink, ensuring that all resources are properly released. + * It should be called when the credentials provider is no longer needed. + */ + public void shutdown() { + credentialsSink.tryEmitComplete(); + tokenManager.stop(); + } + +} diff --git a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java new file mode 100644 index 000000000..2d43bb9b1 --- /dev/null +++ b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java @@ -0,0 +1,146 @@ +package io.lettuce.authx; + +import io.lettuce.core.RedisCredentials; +import io.lettuce.core.TestTokenManager; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import reactor.core.Disposable; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.test.StepVerifier; +import redis.clients.authentication.core.SimpleToken; + +import java.time.Duration; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; + +public class TokenBasedRedisCredentialsProviderTest { + + private TestTokenManager tokenManager; + + private TokenBasedRedisCredentialsProvider credentialsProvider; + + @BeforeEach + public void setUp() { + // Use TestToken manager to emit tokens/errors on request + tokenManager = new TestTokenManager(null, null); + credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenManager); + } + + @Test + public void shouldReturnPreviouslyEmittedTokenWhenResolved() { + tokenManager.emitToken(testToken("test-user", "token-1")); + + Mono credentials = credentialsProvider.resolveCredentials(); + + StepVerifier.create(credentials).assertNext(actual -> { + assertThat(actual.getUsername()).isEqualTo("test-user"); + assertThat(new String(actual.getPassword())).isEqualTo("token-1"); + }).verifyComplete(); + } + + @Test + public void shouldReturnLatestEmittedTokenWhenResolved() { + tokenManager.emitToken(testToken("test-user", "token-2")); + tokenManager.emitToken(testToken("test-user", "token-3")); // Latest token + + Mono credentials = credentialsProvider.resolveCredentials(); + + StepVerifier.create(credentials).assertNext(actual -> { + assertThat(actual.getUsername()).isEqualTo("test-user"); + assertThat(new String(actual.getPassword())).isEqualTo("token-3"); + }).verifyComplete(); + } + + @Test + public void shouldReturnTokenEmittedBeforeSubscription() { + + tokenManager.emitToken(testToken("test-user", "token-1")); + + // Test resolveCredentials + Mono credentials1 = credentialsProvider.resolveCredentials(); + + StepVerifier.create(credentials1).assertNext(actual -> { + assertThat(actual.getUsername()).isEqualTo("test-user"); + assertThat(new String(actual.getPassword())).isEqualTo("token-1"); + }).verifyComplete(); + + // Emit second token and subscribe another + tokenManager.emitToken(testToken("test-user", "token-2")); + tokenManager.emitToken(testToken("test-user", "token-3")); + Mono credentials2 = credentialsProvider.resolveCredentials(); + StepVerifier.create(credentials2).assertNext(actual -> { + assertThat(actual.getUsername()).isEqualTo("test-user"); + assertThat(new String(actual.getPassword())).isEqualTo("token-3"); + }).verifyComplete(); + } + + @Test + public void shouldWaitForAndReturnTokenWhenEmittedLater() { + Mono result = credentialsProvider.resolveCredentials(); + + tokenManager.emitTokenWithDelay(testToken("test-user", "delayed-token"), 100); // Emit token after 100ms + StepVerifier.create(result) + .assertNext(credentials -> assertThat(String.valueOf(credentials.getPassword())).isEqualTo("delayed-token")) + .verifyComplete(); + } + + @Test + public void shouldCompleteAllSubscribersOnStop() { + Flux credentialsFlux1 = credentialsProvider.credentials(); + Flux credentialsFlux2 = credentialsProvider.credentials(); + + Disposable subscription1 = credentialsFlux1.subscribe(); + Disposable subscription2 = credentialsFlux2.subscribe(); + + tokenManager.emitToken(testToken("test-user", "token-1")); + + new Thread(() -> { + try { + Thread.sleep(100); // Delay of 100 milliseconds + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + credentialsProvider.shutdown(); + }).start(); + + StepVerifier.create(credentialsFlux1) + .assertNext(credentials -> assertThat(String.valueOf(credentials.getPassword())).isEqualTo("token-1")) + .verifyComplete(); + + StepVerifier.create(credentialsFlux2) + .assertNext(credentials -> assertThat(String.valueOf(credentials.getPassword())).isEqualTo("token-1")) + .verifyComplete(); + } + + @Test + public void shouldPropagateMultipleTokensOnStream() { + + Flux result = credentialsProvider.credentials(); + StepVerifier.create(result).then(() -> tokenManager.emitToken(testToken("test-user", "token1"))) + .then(() -> tokenManager.emitToken(testToken("test-user", "token2"))) + .assertNext(credentials -> assertThat(String.valueOf(credentials.getPassword())).isEqualTo("token1")) + .assertNext(credentials -> assertThat(String.valueOf(credentials.getPassword())).isEqualTo("token2")) + .thenCancel().verify(Duration.ofMillis(100)); + } + + @Test + public void shouldHandleTokenRequestErrorGracefully() { + Exception simulatedError = new RuntimeException("Token request failed"); + tokenManager.emitError(simulatedError); + + Flux result = credentialsProvider.credentials(); + + StepVerifier.create(result).expectErrorMatches( + throwable -> throwable instanceof RuntimeException && "Token request failed".equals(throwable.getMessage())) + .verify(); + } + + private SimpleToken testToken(String username, String value) { + return new SimpleToken(value, System.currentTimeMillis() + 5000, // expires in 5 seconds + System.currentTimeMillis(), Collections.singletonMap("oid", username)); + + } + +} diff --git a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java index 1890113f7..e54d935d0 100644 --- a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java +++ b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java @@ -5,6 +5,7 @@ import javax.inject.Inject; +import io.lettuce.authx.TokenBasedRedisCredentialsProvider; import io.lettuce.core.event.command.CommandListener; import io.lettuce.core.event.command.CommandSucceededEvent; import io.lettuce.core.protocol.RedisCommand; @@ -24,9 +25,12 @@ import io.lettuce.test.condition.EnabledOnCommand; import io.lettuce.test.settings.TestSettings; import reactor.core.publisher.Mono; +import redis.clients.authentication.core.SimpleToken; import java.time.Duration; +import java.time.Instant; import java.util.ArrayList; +import java.util.Collections; import java.util.List; /** @@ -121,6 +125,43 @@ void streamingCredentialProvider(RedisClient client) { client.getOptions().mutate().reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.DEFAULT).build()); } + @Test + @Inject + void tokenBasedCredentialProvider(RedisClient client) { + + TestCommandListener listener = new TestCommandListener(); + client.addListener(listener); + client.setOptions(client.getOptions().mutate() + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); + + TestTokenManager tokenManager = new TestTokenManager(null, null); + TokenBasedRedisCredentialsProvider credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenManager); + + // Build RedisURI with streaming credentials provider + RedisURI uri = RedisURI.builder().withHost(TestSettings.host()).withPort(TestSettings.port()) + .withClientName("streaming_cred_test").withAuthentication(credentialsProvider) + .withTimeout(Duration.ofSeconds(5)).build(); + tokenManager.emitToken(testToken(TestSettings.username(), TestSettings.password().toString().toCharArray())); + + StatefulRedisConnection connection = client.connect(StringCodec.UTF8, uri); + assertThat(connection.sync().aclWhoami()).isEqualTo(TestSettings.username()); + + // rotate the credentials + tokenManager.emitToken(testToken("steave", "foobared".toCharArray())); + + Awaitility.await().atMost(Duration.ofSeconds(1)).until(() -> listener.succeeded.stream() + .anyMatch(command -> isAuthCommandWithCredentials(command, "steave", "foobared".toCharArray()))); + + // verify that the connection is re-authenticated with the new user credentials + assertThat(connection.sync().aclWhoami()).isEqualTo("steave"); + + credentialsProvider.shutdown(); + connection.close(); + client.removeListener(listener); + client.setOptions( + client.getOptions().mutate().reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.DEFAULT).build()); + } + static class TestCommandListener implements CommandListener { final List> succeeded = new ArrayList<>(); @@ -142,4 +183,9 @@ private boolean isAuthCommandWithCredentials(RedisCommand command, Stri return false; } + private SimpleToken testToken(String username, char[] password) { + return new SimpleToken(String.valueOf(password), Instant.now().plusMillis(500).toEpochMilli(), + Instant.now().toEpochMilli(), Collections.singletonMap("oid", username)); + } + } diff --git a/src/test/java/io/lettuce/core/TestTokenManager.java b/src/test/java/io/lettuce/core/TestTokenManager.java new file mode 100644 index 000000000..391b6302b --- /dev/null +++ b/src/test/java/io/lettuce/core/TestTokenManager.java @@ -0,0 +1,50 @@ +package io.lettuce.core; + +import redis.clients.authentication.core.IdentityProvider; +import redis.clients.authentication.core.SimpleToken; +import redis.clients.authentication.core.TokenListener; +import redis.clients.authentication.core.TokenManager; +import redis.clients.authentication.core.TokenManagerConfig; + +public class TestTokenManager extends TokenManager { + + private TokenListener listener; + + public TestTokenManager(IdentityProvider identityProvider, TokenManagerConfig tokenManagerConfig) { + super(identityProvider, tokenManagerConfig); + } + + @Override + public void start(TokenListener listener, boolean waitForToken) { + this.listener = listener; + } + + @Override + public void stop() { + // Cleanup logic if needed + } + + public void emitToken(SimpleToken token) { + if (listener != null) { + listener.onTokenRenewed(token); + } + } + + public void emitError(Exception exception) { + if (listener != null) { + listener.onError(exception); + } + } + + public void emitTokenWithDelay(SimpleToken token, long delayMillis) { + new Thread(() -> { + try { + Thread.sleep(delayMillis); + emitToken(token); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }).start(); + } + +} diff --git a/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java b/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java index 3c8f20a96..908ec7583 100644 --- a/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java +++ b/src/test/java/io/lettuce/core/cluster/RedisClusterStreamingCredentialsProviderlIntegrationTests.java @@ -140,7 +140,6 @@ void nodeSelectionApiShouldWork() { @Test void shouldPerformNodeConnectionReauth() { ClusterClientOptions origClientOptions = redisClient.getClusterClientOptions(); - origClientOptions.mutate().reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); redisClient.setOptions(origClientOptions.mutate() .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); diff --git a/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java b/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java new file mode 100644 index 000000000..26b60caa8 --- /dev/null +++ b/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java @@ -0,0 +1,136 @@ +package io.lettuce.examples; + +import io.lettuce.authx.TokenBasedRedisCredentialsProvider; +import io.lettuce.core.ClientOptions; +import io.lettuce.core.RedisClient; +import io.lettuce.core.RedisURI; +import io.lettuce.core.SocketOptions; +import io.lettuce.core.TimeoutOptions; +import io.lettuce.core.api.StatefulRedisConnection; +import io.lettuce.core.cluster.ClusterClientOptions; +import io.lettuce.core.cluster.RedisClusterClient; +import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; +import io.lettuce.core.cluster.api.sync.NodeSelection; +import io.lettuce.core.cluster.api.sync.RedisAdvancedClusterCommands; +import io.lettuce.core.codec.StringCodec; +import redis.clients.authentication.core.IdentityProviderConfig; +import redis.clients.authentication.core.TokenAuthConfig; +import redis.clients.authentication.entraid.EntraIDTokenAuthConfigBuilder; + +import java.time.Duration; +import java.util.Collections; +import java.util.Set; + +public class TokenBasedAuthExample { + + public static final String REDIS_URI = "redis://108.143.40.70:12002"; + + public static void main(String[] args) throws Exception { + // Configure TokenManager + String authority = "https://login.microsoftonline.com/562f7bf2-f594-47bf-8ac3-a06514b5d434"; + Set scopes = Collections.singleton("https://redis.azure.com/.default"); + + String User1_clientId = System.getenv("USER1_CLIENT_ID"); + String User1_secret = System.getenv("USER1_SECRET"); + + String User2_clientId = System.getenv("USER2_CLIENT_ID"); + String User2_secret = System.getenv("USER2_SECRET"); + // User 1 + // from redis-authx-entraind + IdentityProviderConfig config1; + try (EntraIDTokenAuthConfigBuilder builder = EntraIDTokenAuthConfigBuilder.builder()) { + config1 = builder.authority(authority).clientId(User1_clientId).secret(User1_secret).scopes(scopes) + .tokenRequestExecTimeoutInMs(10000).build().getIdentityProviderConfig(); + } + + // from redis-authx-core + TokenAuthConfig tokenAuthConfigUser1 = TokenAuthConfig.builder().tokenRequestExecTimeoutInMs(10000) + .expirationRefreshRatio(0.1f).identityProviderConfig(config1).build(); + // Create credentials provider user1 + TokenBasedRedisCredentialsProvider credentialsUser1 = new TokenBasedRedisCredentialsProvider(tokenAuthConfigUser1); + + // User2 + // from redis-authx-entraind + IdentityProviderConfig config2 = EntraIDTokenAuthConfigBuilder.builder().authority(authority).clientId(User2_clientId) + .secret(User2_secret).scopes(scopes).tokenRequestExecTimeoutInMs(10000).build().getIdentityProviderConfig(); + // from redis-authx-core + TokenAuthConfig tokenAuthConfigUser2 = TokenAuthConfig.builder().tokenRequestExecTimeoutInMs(10000) + .expirationRefreshRatio(0.1f).identityProviderConfig(config2).build(); + // Create credentials provider user2 + // TODO: lettuce-autx-tba ( TokenBasedRedisCredentialsProvider & Example there) + TokenBasedRedisCredentialsProvider credentialsUser2 = new TokenBasedRedisCredentialsProvider(tokenAuthConfigUser2); + + // lettuce-core + RedisURI redisURI1 = RedisURI.create(REDIS_URI); + redisURI1.setCredentialsProvider(credentialsUser1); + + RedisURI redisURI2 = RedisURI.create(REDIS_URI); + redisURI2.setCredentialsProvider(credentialsUser2); + + // Create RedisClient + ClientOptions clientOptions = ClientOptions.builder() + .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(5)).build()) + .disconnectedBehavior(ClientOptions.DisconnectedBehavior.REJECT_COMMANDS) + .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); + try { + + // RedisClient using user1 credentials by default + RedisClient redisClient = RedisClient.create(redisURI1); + redisClient.setOptions(clientOptions); + + // create connection using default URI (authorised as user1) + try (StatefulRedisConnection user1 = redisClient.connect(StringCodec.UTF8)) { + + user1.reactive().aclWhoami().doOnNext(System.out::println).block(); + } + + // another connection using different authorizations (user2 credentials provider) + try (StatefulRedisConnection user2 = redisClient.connect(StringCodec.UTF8, redisURI2);) { + user2.reactive().aclWhoami().doOnNext(System.out::println).block(); + } + + // Shutdown Redis client and close connections + redisClient.shutdown(); + + ClusterClientOptions clusterClientOptions = ClusterClientOptions.builder() + .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(5)).build()) + .disconnectedBehavior(ClientOptions.DisconnectedBehavior.REJECT_COMMANDS) + .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); + + // RedisClient using user1 credentials by default + RedisClusterClient redisClusterClient = RedisClusterClient.create(redisURI1); + redisClusterClient.setOptions(clusterClientOptions); + + // create connection using default URI (authorised as user1) + try (StatefulRedisClusterConnection clusterConnection = redisClusterClient.connect(StringCodec.UTF8)) { + + String info = clusterConnection.sync().clusterInfo(); + System.out.println("Cluster Info :" + info); + + String nodes = clusterConnection.sync().clusterNodes(); + System.out.println("Cluster Nodes :" + nodes); + + clusterConnection.sync().set("cluster-key", "cluster-value"); + System.out.println("set " + clusterConnection.sync().get("cluster-key")); + + RedisAdvancedClusterCommands sync = clusterConnection.sync(); + NodeSelection upstream = sync.upstream(); + + upstream.commands().clientId().forEach((v) -> { System.out.println("Client Id : " + v);}); + + System.out.println(" whoami :" + clusterConnection.getConnection(clusterConnection.getPartitions().getPartition(0).getNodeId()).sync() + .aclWhoami()); + } + // Shutdown Redis client and close connections + redisClusterClient.shutdown(); + } finally { + credentialsUser1.shutdown(); + credentialsUser2.shutdown(); + + } + + } + +} From 3c6dbc9c8922e33cccada81c068d63e31145c517 Mon Sep 17 00:00:00 2001 From: ggivo Date: Fri, 13 Dec 2024 16:52:22 +0200 Subject: [PATCH 21/28] rebase to address "oid" core-autx lib change formating --- .../TokenBasedRedisCredentialsProvider.java | 2 +- ...okenBasedRedisCredentialsProviderTest.java | 5 +- .../core/AuthenticationIntegrationTests.java | 4 +- .../examples/TokenBasedAuthExample.java | 163 +++++++++--------- 4 files changed, 88 insertions(+), 86 deletions(-) diff --git a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java index ec8a36372..ca86891c1 100644 --- a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java +++ b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java @@ -36,7 +36,7 @@ private void initializeTokenManager() { @Override public void onTokenRenewed(Token token) { try { - String username = token.tryGet("oid"); + String username = token.getUser(); char[] pass = token.getValue().toCharArray(); RedisCredentials credentials = RedisCredentials.just(username, pass); credentialsSink.tryEmitNext(credentials); diff --git a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java index 2d43bb9b1..d6ff84648 100644 --- a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java +++ b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java @@ -138,9 +138,8 @@ public void shouldHandleTokenRequestErrorGracefully() { } private SimpleToken testToken(String username, String value) { - return new SimpleToken(value, System.currentTimeMillis() + 5000, // expires in 5 seconds - System.currentTimeMillis(), Collections.singletonMap("oid", username)); - + return new SimpleToken(username, value, System.currentTimeMillis() + 5000, // expires in 5 seconds + System.currentTimeMillis(), Collections.emptyMap()); } } diff --git a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java index e54d935d0..618bb1a14 100644 --- a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java +++ b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java @@ -184,8 +184,8 @@ private boolean isAuthCommandWithCredentials(RedisCommand command, Stri } private SimpleToken testToken(String username, char[] password) { - return new SimpleToken(String.valueOf(password), Instant.now().plusMillis(500).toEpochMilli(), - Instant.now().toEpochMilli(), Collections.singletonMap("oid", username)); + return new SimpleToken(username, String.valueOf(password), Instant.now().plusMillis(500).toEpochMilli(), + Instant.now().toEpochMilli(), Collections.emptyMap()); } } diff --git a/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java b/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java index 26b60caa8..ef1b51419 100644 --- a/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java +++ b/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java @@ -35,101 +35,104 @@ public static void main(String[] args) throws Exception { String User2_clientId = System.getenv("USER2_CLIENT_ID"); String User2_secret = System.getenv("USER2_SECRET"); - // User 1 - // from redis-authx-entraind - IdentityProviderConfig config1; - try (EntraIDTokenAuthConfigBuilder builder = EntraIDTokenAuthConfigBuilder.builder()) { - config1 = builder.authority(authority).clientId(User1_clientId).secret(User1_secret).scopes(scopes) - .tokenRequestExecTimeoutInMs(10000).build().getIdentityProviderConfig(); - } - - // from redis-authx-core - TokenAuthConfig tokenAuthConfigUser1 = TokenAuthConfig.builder().tokenRequestExecTimeoutInMs(10000) - .expirationRefreshRatio(0.1f).identityProviderConfig(config1).build(); - // Create credentials provider user1 - TokenBasedRedisCredentialsProvider credentialsUser1 = new TokenBasedRedisCredentialsProvider(tokenAuthConfigUser1); - - // User2 - // from redis-authx-entraind - IdentityProviderConfig config2 = EntraIDTokenAuthConfigBuilder.builder().authority(authority).clientId(User2_clientId) - .secret(User2_secret).scopes(scopes).tokenRequestExecTimeoutInMs(10000).build().getIdentityProviderConfig(); - // from redis-authx-core - TokenAuthConfig tokenAuthConfigUser2 = TokenAuthConfig.builder().tokenRequestExecTimeoutInMs(10000) - .expirationRefreshRatio(0.1f).identityProviderConfig(config2).build(); - // Create credentials provider user2 - // TODO: lettuce-autx-tba ( TokenBasedRedisCredentialsProvider & Example there) - TokenBasedRedisCredentialsProvider credentialsUser2 = new TokenBasedRedisCredentialsProvider(tokenAuthConfigUser2); - - // lettuce-core - RedisURI redisURI1 = RedisURI.create(REDIS_URI); - redisURI1.setCredentialsProvider(credentialsUser1); - - RedisURI redisURI2 = RedisURI.create(REDIS_URI); - redisURI2.setCredentialsProvider(credentialsUser2); - - // Create RedisClient - ClientOptions clientOptions = ClientOptions.builder() - .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(5)).build()) - .disconnectedBehavior(ClientOptions.DisconnectedBehavior.REJECT_COMMANDS) - .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) - .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); - try { - - // RedisClient using user1 credentials by default - RedisClient redisClient = RedisClient.create(redisURI1); - redisClient.setOptions(clientOptions); - - // create connection using default URI (authorised as user1) - try (StatefulRedisConnection user1 = redisClient.connect(StringCodec.UTF8)) { - - user1.reactive().aclWhoami().doOnNext(System.out::println).block(); - } - - // another connection using different authorizations (user2 credentials provider) - try (StatefulRedisConnection user2 = redisClient.connect(StringCodec.UTF8, redisURI2);) { - user2.reactive().aclWhoami().doOnNext(System.out::println).block(); + // User 1 + // from redis-authx-entraind + IdentityProviderConfig config1; + try (EntraIDTokenAuthConfigBuilder builder = EntraIDTokenAuthConfigBuilder.builder()) { + config1 = builder.authority(authority).clientId(User1_clientId).secret(User1_secret).scopes(scopes) + .tokenRequestExecTimeoutInMs(10000).build().getIdentityProviderConfig(); } - // Shutdown Redis client and close connections - redisClient.shutdown(); - - ClusterClientOptions clusterClientOptions = ClusterClientOptions.builder() + // from redis-authx-core + TokenAuthConfig tokenAuthConfigUser1 = TokenAuthConfig.builder().tokenRequestExecTimeoutInMs(10000) + .expirationRefreshRatio(0.1f).identityProviderConfig(config1).build(); + // Create credentials provider user1 + TokenBasedRedisCredentialsProvider credentialsUser1 = new TokenBasedRedisCredentialsProvider(tokenAuthConfigUser1); + + // User2 + // from redis-authx-entraind + IdentityProviderConfig config2 = EntraIDTokenAuthConfigBuilder.builder().authority(authority).clientId(User2_clientId) + .secret(User2_secret).scopes(scopes).tokenRequestExecTimeoutInMs(10000).build().getIdentityProviderConfig(); + // from redis-authx-core + TokenAuthConfig tokenAuthConfigUser2 = TokenAuthConfig.builder().tokenRequestExecTimeoutInMs(10000) + .expirationRefreshRatio(0.1f).identityProviderConfig(config2).build(); + // Create credentials provider user2 + // TODO: lettuce-autx-tba ( TokenBasedRedisCredentialsProvider & Example there) + TokenBasedRedisCredentialsProvider credentialsUser2 = new TokenBasedRedisCredentialsProvider(tokenAuthConfigUser2); + + // lettuce-core + RedisURI redisURI1 = RedisURI.create(REDIS_URI); + redisURI1.setCredentialsProvider(credentialsUser1); + + RedisURI redisURI2 = RedisURI.create(REDIS_URI); + redisURI2.setCredentialsProvider(credentialsUser2); + + // Create RedisClient + ClientOptions clientOptions = ClientOptions.builder() .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(5)).build()) .disconnectedBehavior(ClientOptions.DisconnectedBehavior.REJECT_COMMANDS) .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); + try { - // RedisClient using user1 credentials by default - RedisClusterClient redisClusterClient = RedisClusterClient.create(redisURI1); - redisClusterClient.setOptions(clusterClientOptions); + // RedisClient using user1 credentials by default + RedisClient redisClient = RedisClient.create(redisURI1); + redisClient.setOptions(clientOptions); - // create connection using default URI (authorised as user1) - try (StatefulRedisClusterConnection clusterConnection = redisClusterClient.connect(StringCodec.UTF8)) { + // create connection using default URI (authorised as user1) + try (StatefulRedisConnection user1 = redisClient.connect(StringCodec.UTF8)) { - String info = clusterConnection.sync().clusterInfo(); - System.out.println("Cluster Info :" + info); + user1.reactive().aclWhoami().doOnNext(System.out::println).block(); + } - String nodes = clusterConnection.sync().clusterNodes(); - System.out.println("Cluster Nodes :" + nodes); + // another connection using different authorizations (user2 credentials provider) + try (StatefulRedisConnection user2 = redisClient.connect(StringCodec.UTF8, redisURI2);) { + user2.reactive().aclWhoami().doOnNext(System.out::println).block(); + } - clusterConnection.sync().set("cluster-key", "cluster-value"); - System.out.println("set " + clusterConnection.sync().get("cluster-key")); + // Shutdown Redis client and close connections + redisClient.shutdown(); - RedisAdvancedClusterCommands sync = clusterConnection.sync(); - NodeSelection upstream = sync.upstream(); + ClusterClientOptions clusterClientOptions = ClusterClientOptions.builder() + .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(5)).build()) + .disconnectedBehavior(ClientOptions.DisconnectedBehavior.REJECT_COMMANDS) + .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); - upstream.commands().clientId().forEach((v) -> { System.out.println("Client Id : " + v);}); + // RedisClient using user1 credentials by default + RedisClusterClient redisClusterClient = RedisClusterClient.create(redisURI1); + redisClusterClient.setOptions(clusterClientOptions); - System.out.println(" whoami :" + clusterConnection.getConnection(clusterConnection.getPartitions().getPartition(0).getNodeId()).sync() - .aclWhoami()); - } - // Shutdown Redis client and close connections - redisClusterClient.shutdown(); - } finally { - credentialsUser1.shutdown(); - credentialsUser2.shutdown(); + // create connection using default URI (authorised as user1) + try (StatefulRedisClusterConnection clusterConnection = redisClusterClient + .connect(StringCodec.UTF8)) { - } + String info = clusterConnection.sync().clusterInfo(); + System.out.println("Cluster Info :" + info); + + String nodes = clusterConnection.sync().clusterNodes(); + System.out.println("Cluster Nodes :" + nodes); + + clusterConnection.sync().set("cluster-key", "cluster-value"); + System.out.println("set " + clusterConnection.sync().get("cluster-key")); + + RedisAdvancedClusterCommands sync = clusterConnection.sync(); + NodeSelection upstream = sync.upstream(); + + upstream.commands().clientId().forEach((v) -> { + System.out.println("Client Id : " + v); + }); + + System.out.println(" whoami :" + clusterConnection + .getConnection(clusterConnection.getPartitions().getPartition(0).getNodeId()).sync().aclWhoami()); + } + // Shutdown Redis client and close connections + redisClusterClient.shutdown(); + } finally { + credentialsUser1.shutdown(); + credentialsUser2.shutdown(); + + } } From 00de1ceb628e690fee451f6aeb0ee3ff60b300e5 Mon Sep 17 00:00:00 2001 From: ggivo Date: Mon, 16 Dec 2024 13:44:42 +0200 Subject: [PATCH 22/28] Add EntraId integration tests Verify authentication using Azure AD with service principals --- pom.xml | 11 ++ .../TokenBasedRedisCredentialsProvider.java | 5 +- .../authx/EntraIdIntegrationTests.java | 177 ++++++++++++++++++ .../io/lettuce/authx/EntraIdTestContext.java | 111 +++++++++++ ...okenBasedRedisCredentialsProviderTest.java | 2 +- .../core/AuthenticationIntegrationTests.java | 2 +- .../examples/TokenBasedAuthExample.java | 5 +- src/test/resources/.env.entraid | 11 ++ 8 files changed, 317 insertions(+), 7 deletions(-) create mode 100644 src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java create mode 100644 src/test/java/io/lettuce/authx/EntraIdTestContext.java create mode 100644 src/test/resources/.env.entraid diff --git a/pom.xml b/pom.xml index e90411a01..06954ae37 100644 --- a/pom.xml +++ b/pom.xml @@ -197,6 +197,12 @@ 0.1.0-SNAPSHOT test + + io.github.cdimascio + dotenv-java + 2.2.0 + test + @@ -210,6 +216,11 @@ redis-authx-entraid test + + io.github.cdimascio + dotenv-java + test + diff --git a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java index ca86891c1..4753010ca 100644 --- a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java +++ b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java @@ -10,7 +10,7 @@ import redis.clients.authentication.core.TokenListener; import redis.clients.authentication.core.TokenManager; -public class TokenBasedRedisCredentialsProvider implements StreamingCredentialsProvider { +public class TokenBasedRedisCredentialsProvider implements StreamingCredentialsProvider, AutoCloseable { private final TokenManager tokenManager; @@ -94,7 +94,8 @@ public Flux credentials() { * This method stops the TokenManager and completes the credentials sink, ensuring that all resources are properly released. * It should be called when the credentials provider is no longer needed. */ - public void shutdown() { + @Override + public void close() { credentialsSink.tryEmitComplete(); tokenManager.stop(); } diff --git a/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java b/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java new file mode 100644 index 000000000..84100f577 --- /dev/null +++ b/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java @@ -0,0 +1,177 @@ +package io.lettuce.authx; + +import io.lettuce.core.ClientOptions; +import io.lettuce.core.RedisClient; +import io.lettuce.core.RedisFuture; +import io.lettuce.core.RedisURI; +import io.lettuce.core.SocketOptions; +import io.lettuce.core.TimeoutOptions; +import io.lettuce.core.TransactionResult; +import io.lettuce.core.api.StatefulRedisConnection; +import io.lettuce.core.api.async.RedisAsyncCommands; +import io.lettuce.core.cluster.ClusterClientOptions; +import io.lettuce.core.cluster.RedisClusterClient; +import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import redis.clients.authentication.core.TokenAuthConfig; +import redis.clients.authentication.entraid.EntraIDTokenAuthConfigBuilder; + +import java.time.Duration; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +public class EntraIdIntegrationTests { + + private static EntraIdTestContext testCtx = EntraIdTestContext.DEFAULT;; + + @BeforeAll + public static void setup() { + Assumptions.assumeTrue(testCtx.host() != null && !testCtx.host().isEmpty(), + "Skipping EntraID tests. Redis host with enabled EntraId not provided!"); + } + + // T.1.1 + // Verify authentication using Azure AD with service principals using Redis Standalone client + @Test + public void standaloneWithSecret_azureServicePrincipalIntegrationTest() throws ExecutionException, InterruptedException { + TokenAuthConfig tokenAuthConfig = EntraIDTokenAuthConfigBuilder.builder().clientId(testCtx.getClientId()) + .secret(testCtx.getClientSecret()).authority(testCtx.getAuthority()).scopes(testCtx.getRedisScopes()).build(); + + // Configure timeout options to assure fast test failover + ClientOptions clientOptions = ClientOptions.builder() + .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(1)).build()) + .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); + + try (TokenBasedRedisCredentialsProvider credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenAuthConfig)) { + RedisURI uri = RedisURI.builder().withHost(testCtx.host()).withPort(testCtx.port()) + .withAuthentication(credentialsProvider).build(); + + try (RedisClient client = RedisClient.create(uri)) { + client.setOptions(clientOptions); + + try (StatefulRedisConnection connection = client.connect()) { + assertThat(connection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); + assertThat(connection.async().aclWhoami().get()).isEqualTo(testCtx.getSpOID()); + assertThat(connection.reactive().aclWhoami().block()).isEqualTo(testCtx.getSpOID()); + } + } + } + } + + // T.1.1 + // Verify authentication using Azure AD with service principals using Redis Cluster Client + @Test + public void clusterWithSecret_azureServicePrincipalIntegrationTest() throws ExecutionException, InterruptedException { + TokenAuthConfig tokenAuthConfig = EntraIDTokenAuthConfigBuilder.builder().clientId(testCtx.getClientId()) + .secret(testCtx.getClientSecret()).authority(testCtx.getAuthority()).scopes(testCtx.getRedisScopes()).build(); + + // Configure timeout options to assure fast test failover + ClusterClientOptions clientOptions = ClusterClientOptions.builder() + .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(1)).build()) + .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); + + try (TokenBasedRedisCredentialsProvider credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenAuthConfig)) { + RedisURI uri = RedisURI.builder().withHost(testCtx.clusterHost().get(0)).withPort(testCtx.clusterPort()) + .withAuthentication(credentialsProvider).build(); + + try (RedisClusterClient client = RedisClusterClient.create(uri)) { + client.setOptions(clientOptions); + + try (StatefulRedisClusterConnection connection = client.connect()) { + assertThat(connection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); + assertThat(connection.async().aclWhoami().get()).isEqualTo(testCtx.getSpOID()); + assertThat(connection.reactive().aclWhoami().block()).isEqualTo(testCtx.getSpOID()); + + connection.getPartitions().forEach((partition) -> { + try (StatefulRedisConnection nodeConnection = connection.getConnection(partition.getNodeId())) { + assertThat(nodeConnection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); + } + }); + } + } + } + } + + // T.2.2 + // Test that the Redis client is not blocked/interrupted during token renewal. + @Test + public void renewalDuringOperationsTest() throws InterruptedException, ExecutionException { + TokenAuthConfig tokenAuthConfig = EntraIDTokenAuthConfigBuilder.builder().clientId(testCtx.getClientId()) + .secret(testCtx.getClientSecret()).authority(testCtx.getAuthority()).scopes(testCtx.getRedisScopes()) + .expirationRefreshRatio(0.000001F).build(); + + // Configure timeout options to assure fast test failover + ClientOptions clientOptions = ClientOptions.builder() + .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(1)).build()) + .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) + .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); + + try (TokenBasedRedisCredentialsProvider credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenAuthConfig)) { + RedisURI uri = RedisURI.builder().withHost(testCtx.host()).withPort(testCtx.port()) + .withAuthentication(credentialsProvider).build(); + + try (RedisClient client = RedisClient.create(uri)) { + client.setOptions(clientOptions); + + try (StatefulRedisConnection connection = client.connect()) { + + // Counter to track the number of command cycles + AtomicInteger commandCycleCount = new AtomicInteger(0); + + // Start a thread to continuously send Redis commands + Thread commandThread = new Thread(() -> { + try { + RedisAsyncCommands async = client.connect().async(); + for (int i = 1; i <= 10; i++) { + // Start a transaction with SET and INCRBY commands + RedisFuture multi = async.multi(); + RedisFuture set = async.set("key", "1"); + RedisFuture incrby = async.incrby("key", 1); + RedisFuture exec = async.exec(); + TransactionResult results = exec.get(1, TimeUnit.SECONDS); + + // Increment the command cycle count after each execution + commandCycleCount.incrementAndGet(); + + // Verify the results from EXEC + assertThat(results).hasSize(2); // We expect 2 responses: SET and INCRBY + + // Check the response from each command in the transaction + assertThat((String) results.get(0)).isEqualTo("OK"); // SET "key" = "1" + assertThat((Long) results.get(1)).isEqualTo(2L); // INCRBY "key" by 1, expected result is 2 + } + } catch (Exception e) { + fail("Command execution failed during token refresh", e); + } + }); + + commandThread.start(); + + // Count token renewals directly within the main thread + AtomicInteger renewalCount = new AtomicInteger(0); + CountDownLatch latch = new CountDownLatch(10); // Wait for at least 10 token renewals + + credentialsProvider.credentials().subscribe(cred -> { + latch.countDown(); // Signal each renewal as it's received + }); + + latch.await(1, TimeUnit.SECONDS); // Wait to reach 10 renewals + commandThread.join(); // Wait for the command thread to finish + + // Verify that at least 10 command cycles were executed during the test + assertThat(commandCycleCount.get()).isGreaterThanOrEqualTo(10); + } + } + } + } + +} diff --git a/src/test/java/io/lettuce/authx/EntraIdTestContext.java b/src/test/java/io/lettuce/authx/EntraIdTestContext.java new file mode 100644 index 000000000..7abfac0fe --- /dev/null +++ b/src/test/java/io/lettuce/authx/EntraIdTestContext.java @@ -0,0 +1,111 @@ +package io.lettuce.authx; + +import io.github.cdimascio.dotenv.Dotenv; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class EntraIdTestContext { + + private static final String AZURE_CLIENT_ID = "AZURE_CLIENT_ID"; + + private static final String AZURE_CLIENT_SECRET = "AZURE_CLIENT_SECRET"; + + private static final String AZURE_SP_OID = "AZURE_SP_OID"; + + private static final String AZURE_AUTHORITY = "AZURE_AUTHORITY"; + + private static final String AZURE_REDIS_SCOPES = "AZURE_REDIS_SCOPES"; + + private static final String REDIS_AZURE_HOST = "REDIS_AZURE_HOST"; + + private static final String REDIS_AZURE_PORT = "REDIS_AZURE_PORT"; + + private static final String REDIS_AZURE_CLUSTER_HOST = "REDIS_AZURE_CLUSTER_HOST"; + + private static final String REDIS_AZURE_CLUSTER_PORT = "REDIS_AZURE_CLUSTER_PORT"; + + private static final String REDIS_AZURE_DB = "REDIS_AZURE_DB"; + + private final String clientId; + + private final String authority; + + private final String clientSecret; + + private final String spOID; + + private final Set redisScopes; + + private final String redisHost; + + private final int redisPort; + + private final List redisClusterHost; + + private final int redisClusterPort; + + private static Dotenv dotenv; + static { + dotenv = Dotenv.configure().directory("src/test/resources").filename(".env.entraid").load(); + } + + public static final EntraIdTestContext DEFAULT = new EntraIdTestContext(); + + private EntraIdTestContext() { + // Using Dotenv directly here + clientId = dotenv.get(AZURE_CLIENT_ID, ""); + clientSecret = dotenv.get(AZURE_CLIENT_SECRET, ""); + spOID = dotenv.get(AZURE_SP_OID, ""); + authority = dotenv.get(AZURE_AUTHORITY, "https://login.microsoftonline.com/your-tenant-id"); + redisHost = dotenv.get(REDIS_AZURE_HOST); + redisPort = Integer.parseInt(dotenv.get(REDIS_AZURE_PORT, "6379")); + redisClusterHost = Arrays.asList(dotenv.get(REDIS_AZURE_CLUSTER_HOST, "").split(",")); + redisClusterPort = Integer.parseInt(dotenv.get(REDIS_AZURE_CLUSTER_PORT, "6379")); + String redisScopesEnv = dotenv.get(AZURE_REDIS_SCOPES, "https://redis.azure.com/.default"); + if (redisScopesEnv != null && !redisScopesEnv.isEmpty()) { + this.redisScopes = new HashSet<>(Arrays.asList(redisScopesEnv.split(";"))); + } else { + this.redisScopes = new HashSet<>(); + } + } + + public String host() { + return redisHost; + } + + public int port() { + return redisPort; + } + + public List clusterHost() { + return redisClusterHost; + } + + public int clusterPort() { + return redisClusterPort; + } + + public String getClientId() { + return clientId; + } + + public String getSpOID() { + return spOID; + } + + public String getAuthority() { + return authority; + } + + public String getClientSecret() { + return clientSecret; + } + + public Set getRedisScopes() { + return redisScopes; + } + +} diff --git a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java index d6ff84648..da78dcfdd 100644 --- a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java +++ b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java @@ -102,7 +102,7 @@ public void shouldCompleteAllSubscribersOnStop() { } catch (InterruptedException e) { Thread.currentThread().interrupt(); } - credentialsProvider.shutdown(); + credentialsProvider.close(); }).start(); StepVerifier.create(credentialsFlux1) diff --git a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java index 618bb1a14..97e88218c 100644 --- a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java +++ b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java @@ -155,7 +155,7 @@ void tokenBasedCredentialProvider(RedisClient client) { // verify that the connection is re-authenticated with the new user credentials assertThat(connection.sync().aclWhoami()).isEqualTo("steave"); - credentialsProvider.shutdown(); + credentialsProvider.close(); connection.close(); client.removeListener(listener); client.setOptions( diff --git a/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java b/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java index ef1b51419..1c317b97a 100644 --- a/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java +++ b/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java @@ -129,9 +129,8 @@ public static void main(String[] args) throws Exception { // Shutdown Redis client and close connections redisClusterClient.shutdown(); } finally { - credentialsUser1.shutdown(); - credentialsUser2.shutdown(); - + credentialsUser1.close(); + credentialsUser2.close(); } } diff --git a/src/test/resources/.env.entraid b/src/test/resources/.env.entraid new file mode 100644 index 000000000..016449e92 --- /dev/null +++ b/src/test/resources/.env.entraid @@ -0,0 +1,11 @@ +AZURE_SP_OID= +AZURE_CLIENT_ID= +AZURE_CLIENT_SECRET= +AZURE_REDIS_SCOPES=https://redis.azure.com/.default +AZURE_AUTHORITY=https://login.microsoftonline.com/ +# Redis standalone db with Azure enabled authentication +REDIS_AZURE_HOST= +REDIS_AZURE_PORT=6379 +# Redis cluster db with Azure enabled authentication & osscluster API enabled +REDIS_AZURE_CLUSTER_HOST= +REDIS_AZURE_CLUSTER_PORT=6379 From 0e0f502402648afee2adb8c10e3205d95ad0d017 Mon Sep 17 00:00:00 2001 From: ggivo Date: Wed, 18 Dec 2024 11:31:27 +0200 Subject: [PATCH 23/28] StreamingCredentialsProvider replaced with RedisCredentialsProvider.supportsStreaming() --- .../authx/TokenBasedRedisCredentialsProvider.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java index 4753010ca..e75abbf23 100644 --- a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java +++ b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java @@ -1,7 +1,7 @@ package io.lettuce.authx; import io.lettuce.core.RedisCredentials; -import io.lettuce.core.StreamingCredentialsProvider; +import io.lettuce.core.RedisCredentialsProvider; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.publisher.Sinks; @@ -10,7 +10,7 @@ import redis.clients.authentication.core.TokenListener; import redis.clients.authentication.core.TokenManager; -public class TokenBasedRedisCredentialsProvider implements StreamingCredentialsProvider, AutoCloseable { +public class TokenBasedRedisCredentialsProvider implements RedisCredentialsProvider, AutoCloseable { private final TokenManager tokenManager; @@ -88,6 +88,11 @@ public Flux credentials() { return credentialsSink.asFlux().onBackpressureLatest(); // Provide a continuous stream of credentials } + @Override + public boolean supportsStreaming() { + return true; + } + /** * Stop the credentials provider and clean up resources. *

From 2acdb6d00844b59a7a74d35242459d2fc22f40f4 Mon Sep 17 00:00:00 2001 From: ggivo Date: Wed, 18 Dec 2024 17:59:10 +0200 Subject: [PATCH 24/28] pub/sub test basic functionality with entraid auth --- .../authx/EntraIdIntegrationTests.java | 224 ++++++++++-------- 1 file changed, 121 insertions(+), 103 deletions(-) diff --git a/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java b/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java index 84100f577..6584030f1 100644 --- a/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java +++ b/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java @@ -12,6 +12,10 @@ import io.lettuce.core.cluster.ClusterClientOptions; import io.lettuce.core.cluster.RedisClusterClient; import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; +import io.lettuce.core.pubsub.StatefulRedisPubSubConnection; +import io.lettuce.core.support.PubSubTestListener; +import io.lettuce.test.Wait; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -29,40 +33,60 @@ public class EntraIdIntegrationTests { - private static EntraIdTestContext testCtx = EntraIdTestContext.DEFAULT;; + private static final EntraIdTestContext testCtx = EntraIdTestContext.DEFAULT;; + + private static ClusterClientOptions clientOptions; + + private static TokenBasedRedisCredentialsProvider credentialsProvider; + + private static RedisClient client; + + private static RedisClusterClient clusterClient; @BeforeAll public static void setup() { Assumptions.assumeTrue(testCtx.host() != null && !testCtx.host().isEmpty(), "Skipping EntraID tests. Redis host with enabled EntraId not provided!"); - } - - // T.1.1 - // Verify authentication using Azure AD with service principals using Redis Standalone client - @Test - public void standaloneWithSecret_azureServicePrincipalIntegrationTest() throws ExecutionException, InterruptedException { - TokenAuthConfig tokenAuthConfig = EntraIDTokenAuthConfigBuilder.builder().clientId(testCtx.getClientId()) - .secret(testCtx.getClientSecret()).authority(testCtx.getAuthority()).scopes(testCtx.getRedisScopes()).build(); // Configure timeout options to assure fast test failover - ClientOptions clientOptions = ClientOptions.builder() + clientOptions = ClusterClientOptions.builder() .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(1)).build()) .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); - try (TokenBasedRedisCredentialsProvider credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenAuthConfig)) { - RedisURI uri = RedisURI.builder().withHost(testCtx.host()).withPort(testCtx.port()) - .withAuthentication(credentialsProvider).build(); + TokenAuthConfig tokenAuthConfig = EntraIDTokenAuthConfigBuilder.builder().clientId(testCtx.getClientId()) + .secret(testCtx.getClientSecret()).authority(testCtx.getAuthority()).scopes(testCtx.getRedisScopes()) + .expirationRefreshRatio(0.0000001F).build(); - try (RedisClient client = RedisClient.create(uri)) { - client.setOptions(clientOptions); + credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenAuthConfig); - try (StatefulRedisConnection connection = client.connect()) { - assertThat(connection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); - assertThat(connection.async().aclWhoami().get()).isEqualTo(testCtx.getSpOID()); - assertThat(connection.reactive().aclWhoami().block()).isEqualTo(testCtx.getSpOID()); - } - } + RedisURI uri = RedisURI.builder().withHost(testCtx.host()).withPort(testCtx.port()) + .withAuthentication(credentialsProvider).build(); + + client = RedisClient.create(uri); + client.setOptions(clientOptions); + + RedisURI clusterUri = RedisURI.builder().withHost(testCtx.clusterHost().get(0)).withPort(testCtx.clusterPort()) + .withAuthentication(credentialsProvider).build(); + clusterClient = RedisClusterClient.create(clusterUri); + clusterClient.setOptions(clientOptions); + } + + @AfterAll + public static void cleanup() { + if (credentialsProvider != null) { + credentialsProvider.close(); + } + } + + // T.1.1 + // Verify authentication using Azure AD with service principals using Redis Standalone client + @Test + public void standaloneWithSecret_azureServicePrincipalIntegrationTest() throws ExecutionException, InterruptedException { + try (StatefulRedisConnection connection = client.connect()) { + assertThat(connection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); + assertThat(connection.async().aclWhoami().get()).isEqualTo(testCtx.getSpOID()); + assertThat(connection.reactive().aclWhoami().block()).isEqualTo(testCtx.getSpOID()); } } @@ -70,107 +94,101 @@ public void standaloneWithSecret_azureServicePrincipalIntegrationTest() throws E // Verify authentication using Azure AD with service principals using Redis Cluster Client @Test public void clusterWithSecret_azureServicePrincipalIntegrationTest() throws ExecutionException, InterruptedException { - TokenAuthConfig tokenAuthConfig = EntraIDTokenAuthConfigBuilder.builder().clientId(testCtx.getClientId()) - .secret(testCtx.getClientSecret()).authority(testCtx.getAuthority()).scopes(testCtx.getRedisScopes()).build(); - // Configure timeout options to assure fast test failover - ClusterClientOptions clientOptions = ClusterClientOptions.builder() - .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(1)).build()) - .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) - .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); - - try (TokenBasedRedisCredentialsProvider credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenAuthConfig)) { - RedisURI uri = RedisURI.builder().withHost(testCtx.clusterHost().get(0)).withPort(testCtx.clusterPort()) - .withAuthentication(credentialsProvider).build(); - - try (RedisClusterClient client = RedisClusterClient.create(uri)) { - client.setOptions(clientOptions); - - try (StatefulRedisClusterConnection connection = client.connect()) { - assertThat(connection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); - assertThat(connection.async().aclWhoami().get()).isEqualTo(testCtx.getSpOID()); - assertThat(connection.reactive().aclWhoami().block()).isEqualTo(testCtx.getSpOID()); + try (StatefulRedisClusterConnection connection = clusterClient.connect()) { + assertThat(connection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); + assertThat(connection.async().aclWhoami().get()).isEqualTo(testCtx.getSpOID()); + assertThat(connection.reactive().aclWhoami().block()).isEqualTo(testCtx.getSpOID()); - connection.getPartitions().forEach((partition) -> { - try (StatefulRedisConnection nodeConnection = connection.getConnection(partition.getNodeId())) { - assertThat(nodeConnection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); - } - }); + connection.getPartitions().forEach((partition) -> { + try (StatefulRedisConnection nodeConnection = connection.getConnection(partition.getNodeId())) { + assertThat(nodeConnection.sync().aclWhoami()).isEqualTo(testCtx.getSpOID()); } - } + }); } } // T.2.2 // Test that the Redis client is not blocked/interrupted during token renewal. @Test - public void renewalDuringOperationsTest() throws InterruptedException, ExecutionException { - TokenAuthConfig tokenAuthConfig = EntraIDTokenAuthConfigBuilder.builder().clientId(testCtx.getClientId()) - .secret(testCtx.getClientSecret()).authority(testCtx.getAuthority()).scopes(testCtx.getRedisScopes()) - .expirationRefreshRatio(0.000001F).build(); - - // Configure timeout options to assure fast test failover - ClientOptions clientOptions = ClientOptions.builder() - .socketOptions(SocketOptions.builder().connectTimeout(Duration.ofSeconds(1)).build()) - .timeoutOptions(TimeoutOptions.enabled(Duration.ofSeconds(1))) - .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build(); - - try (TokenBasedRedisCredentialsProvider credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenAuthConfig)) { - RedisURI uri = RedisURI.builder().withHost(testCtx.host()).withPort(testCtx.port()) - .withAuthentication(credentialsProvider).build(); - - try (RedisClient client = RedisClient.create(uri)) { - client.setOptions(clientOptions); - - try (StatefulRedisConnection connection = client.connect()) { + public void renewalDuringOperationsTest() throws InterruptedException { + + // Counter to track the number of command cycles + AtomicInteger commandCycleCount = new AtomicInteger(0); + + // Start a thread to continuously send Redis commands + Thread commandThread = new Thread(() -> { + try (StatefulRedisConnection connection = client.connect()) { + RedisAsyncCommands async = connection.async(); + for (int i = 1; i <= 10; i++) { + // Start a transaction with SET and INCRBY commands + RedisFuture multi = async.multi(); + RedisFuture set = async.set("key", "1"); + RedisFuture incrby = async.incrby("key", 1); + RedisFuture exec = async.exec(); + TransactionResult results = exec.get(1, TimeUnit.SECONDS); + + // Increment the command cycle count after each execution + commandCycleCount.incrementAndGet(); + + // Verify the results from EXEC + assertThat(results).hasSize(2); // We expect 2 responses: SET and INCRBY + + // Check the response from each command in the transaction + assertThat((String) results.get(0)).isEqualTo("OK"); // SET "key" = "1" + assertThat((Long) results.get(1)).isEqualTo(2L); // INCRBY "key" by 1, expected result is 2 + } + } catch (Exception e) { + fail("Command execution failed during token refresh", e); + } + }); - // Counter to track the number of command cycles - AtomicInteger commandCycleCount = new AtomicInteger(0); + commandThread.start(); - // Start a thread to continuously send Redis commands - Thread commandThread = new Thread(() -> { - try { - RedisAsyncCommands async = client.connect().async(); - for (int i = 1; i <= 10; i++) { - // Start a transaction with SET and INCRBY commands - RedisFuture multi = async.multi(); - RedisFuture set = async.set("key", "1"); - RedisFuture incrby = async.incrby("key", 1); - RedisFuture exec = async.exec(); - TransactionResult results = exec.get(1, TimeUnit.SECONDS); + CountDownLatch latch = new CountDownLatch(10); // Wait for at least 10 token renewals - // Increment the command cycle count after each execution - commandCycleCount.incrementAndGet(); + credentialsProvider.credentials().subscribe(cred -> { + latch.countDown(); // Signal each renewal as it's received + }); - // Verify the results from EXEC - assertThat(results).hasSize(2); // We expect 2 responses: SET and INCRBY + assertThat(latch.await(1, TimeUnit.SECONDS)).isTrue(); // Wait to reach 10 renewals + commandThread.join(); // Wait for the command thread to finish - // Check the response from each command in the transaction - assertThat((String) results.get(0)).isEqualTo("OK"); // SET "key" = "1" - assertThat((Long) results.get(1)).isEqualTo(2L); // INCRBY "key" by 1, expected result is 2 - } - } catch (Exception e) { - fail("Command execution failed during token refresh", e); - } - }); + // Verify that at least 10 command cycles were executed during the test + assertThat(commandCycleCount.get()).isGreaterThanOrEqualTo(10); + } - commandThread.start(); + // T.2.2 + // Test basic Pub/Sub functionality is not blocked/interrupted during token renewal. + @Test + public void renewalDuringPubSubOperationsTest() throws InterruptedException { + try (StatefulRedisPubSubConnection connectionPubSub = client.connectPubSub(); + StatefulRedisPubSubConnection connectionPubSub1 = client.connectPubSub()) { + + PubSubTestListener listener = new PubSubTestListener(); + connectionPubSub.addListener(listener); + connectionPubSub.sync().subscribe("channel"); + + // Start a thread to continuously send Redis commands + Thread pubsubThread = new Thread(() -> { + for (int i = 1; i <= 100; i++) { + connectionPubSub1.sync().publish("channel", "message"); + } + }); - // Count token renewals directly within the main thread - AtomicInteger renewalCount = new AtomicInteger(0); - CountDownLatch latch = new CountDownLatch(10); // Wait for at least 10 token renewals + pubsubThread.start(); - credentialsProvider.credentials().subscribe(cred -> { - latch.countDown(); // Signal each renewal as it's received - }); + CountDownLatch latch = new CountDownLatch(10); + credentialsProvider.credentials().subscribe(cred -> { + latch.countDown(); + }); - latch.await(1, TimeUnit.SECONDS); // Wait to reach 10 renewals - commandThread.join(); // Wait for the command thread to finish + assertThat(latch.await(1, TimeUnit.SECONDS)).isTrue(); // Wait for at least 10 token renewals + pubsubThread.join(); // Wait for the pub/sub thread to finish - // Verify that at least 10 command cycles were executed during the test - assertThat(commandCycleCount.get()).isGreaterThanOrEqualTo(10); - } - } + // Verify that all messages were received + Wait.untilEquals(100, () -> listener.getMessages().size()).waitOrTimeout(); + assertThat(listener.getMessages()).allMatch(msg -> msg.equals("message")); } } From 8c760f1695cc0f8e27c84a28efcd0f4fe5673251 Mon Sep 17 00:00:00 2001 From: ggivo Date: Thu, 19 Dec 2024 14:20:59 +0200 Subject: [PATCH 25/28] Update src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java Co-authored-by: Tihomir Krasimirov Mateev --- .../lettuce/authx/TokenBasedRedisCredentialsProvider.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java index e75abbf23..d7098c9ae 100644 --- a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java +++ b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java @@ -1,3 +1,9 @@ +/* + * Copyright 2024, Redis Ltd. and Contributors + * All rights reserved. + * + * Licensed under the MIT License. + */ package io.lettuce.authx; import io.lettuce.core.RedisCredentials; From a2a8e10bb90758697c2dcdafa5a05efd42fa9093 Mon Sep 17 00:00:00 2001 From: ggivo Date: Thu, 19 Dec 2024 14:39:20 +0200 Subject: [PATCH 26/28] Addressing review comments from @tishun --- .../TokenBasedRedisCredentialsProvider.java | 46 ++++++++++--------- .../authx/EntraIdIntegrationTests.java | 2 +- ...okenBasedRedisCredentialsProviderTest.java | 14 ++++-- .../core/AuthenticationIntegrationTests.java | 2 +- .../examples/TokenBasedAuthExample.java | 4 +- 5 files changed, 38 insertions(+), 30 deletions(-) diff --git a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java index d7098c9ae..f7b82924a 100644 --- a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java +++ b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java @@ -8,6 +8,8 @@ import io.lettuce.core.RedisCredentials; import io.lettuce.core.RedisCredentialsProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.publisher.Sinks; @@ -18,42 +20,31 @@ public class TokenBasedRedisCredentialsProvider implements RedisCredentialsProvider, AutoCloseable { + private static final Logger log = LoggerFactory.getLogger(TokenBasedRedisCredentialsProvider.class); + private final TokenManager tokenManager; private final Sinks.Many credentialsSink = Sinks.many().replay().latest(); - public TokenBasedRedisCredentialsProvider(TokenAuthConfig tokenAuthConfig) { - this(new TokenManager(tokenAuthConfig.getIdentityProviderConfig().getProvider(), - tokenAuthConfig.getTokenManagerConfig())); - - } - - public TokenBasedRedisCredentialsProvider(TokenManager tokenManager) { + private TokenBasedRedisCredentialsProvider(TokenManager tokenManager) { this.tokenManager = tokenManager; - initializeTokenManager(); } - /** - * Initialize the TokenManager and subscribe to token renewal events. - */ - private void initializeTokenManager() { + private void init() { + TokenListener listener = new TokenListener() { @Override public void onTokenRenewed(Token token) { - try { - String username = token.getUser(); - char[] pass = token.getValue().toCharArray(); - RedisCredentials credentials = RedisCredentials.just(username, pass); - credentialsSink.tryEmitNext(credentials); - } catch (Exception e) { - credentialsSink.emitError(e, Sinks.EmitFailureHandler.FAIL_FAST); - } + String username = token.getUser(); + char[] pass = token.getValue().toCharArray(); + RedisCredentials credentials = RedisCredentials.just(username, pass); + credentialsSink.tryEmitNext(credentials); } @Override public void onError(Exception exception) { - credentialsSink.tryEmitError(exception); + log.error("Token renew failed!", exception); } }; @@ -62,6 +53,8 @@ public void onError(Exception exception) { tokenManager.start(listener, false); } catch (Exception e) { credentialsSink.tryEmitError(e); + tokenManager.stop(); + throw new RuntimeException("Failed to start TokenManager", e); } } @@ -111,4 +104,15 @@ public void close() { tokenManager.stop(); } + public static TokenBasedRedisCredentialsProvider create(TokenAuthConfig tokenAuthConfig) { + return create(new TokenManager(tokenAuthConfig.getIdentityProviderConfig().getProvider(), + tokenAuthConfig.getTokenManagerConfig())); + } + + public static TokenBasedRedisCredentialsProvider create(TokenManager tokenManager) { + TokenBasedRedisCredentialsProvider credentialManager = new TokenBasedRedisCredentialsProvider(tokenManager); + credentialManager.init(); + return credentialManager; + } + } diff --git a/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java b/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java index 6584030f1..a4eba6704 100644 --- a/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java +++ b/src/test/java/io/lettuce/authx/EntraIdIntegrationTests.java @@ -58,7 +58,7 @@ public static void setup() { .secret(testCtx.getClientSecret()).authority(testCtx.getAuthority()).scopes(testCtx.getRedisScopes()) .expirationRefreshRatio(0.0000001F).build(); - credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenAuthConfig); + credentialsProvider = TokenBasedRedisCredentialsProvider.create(tokenAuthConfig); RedisURI uri = RedisURI.builder().withHost(testCtx.host()).withPort(testCtx.port()) .withAuthentication(credentialsProvider).build(); diff --git a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java index da78dcfdd..68436ec36 100644 --- a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java +++ b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java @@ -25,7 +25,7 @@ public class TokenBasedRedisCredentialsProviderTest { public void setUp() { // Use TestToken manager to emit tokens/errors on request tokenManager = new TestTokenManager(null, null); - credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenManager); + credentialsProvider = TokenBasedRedisCredentialsProvider.create(tokenManager); } @Test @@ -128,13 +128,17 @@ public void shouldPropagateMultipleTokensOnStream() { @Test public void shouldHandleTokenRequestErrorGracefully() { Exception simulatedError = new RuntimeException("Token request failed"); - tokenManager.emitError(simulatedError); Flux result = credentialsProvider.credentials(); - StepVerifier.create(result).expectErrorMatches( - throwable -> throwable instanceof RuntimeException && "Token request failed".equals(throwable.getMessage())) - .verify(); + StepVerifier.create(result).then(() -> { + tokenManager.emitToken(testToken("test-user", "token1")); + tokenManager.emitError(simulatedError); + tokenManager.emitToken(testToken("test-user", "token2")); + }).assertNext(credentials -> assertThat(String.valueOf(credentials.getPassword())).isEqualTo("token1")) + .assertNext(credentials -> assertThat(String.valueOf(credentials.getPassword())).isEqualTo("token2")) + .thenCancel().verify(Duration.ofMillis(100)); + } private SimpleToken testToken(String username, String value) { diff --git a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java index 97e88218c..7edab1567 100644 --- a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java +++ b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java @@ -135,7 +135,7 @@ void tokenBasedCredentialProvider(RedisClient client) { .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); TestTokenManager tokenManager = new TestTokenManager(null, null); - TokenBasedRedisCredentialsProvider credentialsProvider = new TokenBasedRedisCredentialsProvider(tokenManager); + TokenBasedRedisCredentialsProvider credentialsProvider = TokenBasedRedisCredentialsProvider.create(tokenManager); // Build RedisURI with streaming credentials provider RedisURI uri = RedisURI.builder().withHost(TestSettings.host()).withPort(TestSettings.port()) diff --git a/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java b/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java index 1c317b97a..e8bd9f9a3 100644 --- a/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java +++ b/src/test/java/io/lettuce/examples/TokenBasedAuthExample.java @@ -47,7 +47,7 @@ public static void main(String[] args) throws Exception { TokenAuthConfig tokenAuthConfigUser1 = TokenAuthConfig.builder().tokenRequestExecTimeoutInMs(10000) .expirationRefreshRatio(0.1f).identityProviderConfig(config1).build(); // Create credentials provider user1 - TokenBasedRedisCredentialsProvider credentialsUser1 = new TokenBasedRedisCredentialsProvider(tokenAuthConfigUser1); + TokenBasedRedisCredentialsProvider credentialsUser1 = TokenBasedRedisCredentialsProvider.create(tokenAuthConfigUser1); // User2 // from redis-authx-entraind @@ -58,7 +58,7 @@ public static void main(String[] args) throws Exception { .expirationRefreshRatio(0.1f).identityProviderConfig(config2).build(); // Create credentials provider user2 // TODO: lettuce-autx-tba ( TokenBasedRedisCredentialsProvider & Example there) - TokenBasedRedisCredentialsProvider credentialsUser2 = new TokenBasedRedisCredentialsProvider(tokenAuthConfigUser2); + TokenBasedRedisCredentialsProvider credentialsUser2 = TokenBasedRedisCredentialsProvider.create(tokenAuthConfigUser2); // lettuce-core RedisURI redisURI1 = RedisURI.create(REDIS_URI); From 668b8500b4fd31eda2ecaa594827012426c9a5c2 Mon Sep 17 00:00:00 2001 From: ggivo Date: Fri, 20 Dec 2024 14:09:02 +0200 Subject: [PATCH 27/28] Bump redis-authx-core & redis-authx-entraid from 0.1.0-SNAPSHOT to 0.1.1-beta1 --- pom.xml | 34 +++---------------- ...okenBasedRedisCredentialsProviderTest.java | 11 +++++- .../core/AuthenticationIntegrationTests.java | 7 +++- 3 files changed, 20 insertions(+), 32 deletions(-) diff --git a/pom.xml b/pom.xml index 06954ae37..a7334e25a 100644 --- a/pom.xml +++ b/pom.xml @@ -89,19 +89,6 @@ HEAD - - - sonatype-snapshots - https://oss.sonatype.org/content/repositories/snapshots/ - - false - - - true - - - - ossrh @@ -186,23 +173,7 @@ pom import - - redis.clients.authentication - redis-authx-core - 0.1.0-SNAPSHOT - - - redis.clients.authentication - redis-authx-entraid - 0.1.0-SNAPSHOT - test - - - io.github.cdimascio - dotenv-java - 2.2.0 - test - + @@ -210,15 +181,18 @@ redis.clients.authentication redis-authx-core + 0.1.1-beta1 redis.clients.authentication redis-authx-entraid + 0.1.1-beta1 test io.github.cdimascio dotenv-java + 2.2.0 test diff --git a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java index 68436ec36..7ce58d0a6 100644 --- a/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java +++ b/src/test/java/io/lettuce/authx/TokenBasedRedisCredentialsProviderTest.java @@ -9,11 +9,14 @@ import reactor.core.publisher.Mono; import reactor.test.StepVerifier; import redis.clients.authentication.core.SimpleToken; +import redis.clients.authentication.core.TokenManagerConfig; import java.time.Duration; import java.util.Collections; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TokenBasedRedisCredentialsProviderTest { @@ -24,7 +27,9 @@ public class TokenBasedRedisCredentialsProviderTest { @BeforeEach public void setUp() { // Use TestToken manager to emit tokens/errors on request - tokenManager = new TestTokenManager(null, null); + TokenManagerConfig tokenManagerConfig = mock(TokenManagerConfig.class); + when(tokenManagerConfig.getRetryPolicy()).thenReturn(mock(TokenManagerConfig.RetryPolicy.class)); + tokenManager = new TestTokenManager(null, tokenManagerConfig); credentialsProvider = TokenBasedRedisCredentialsProvider.create(tokenManager); } @@ -112,6 +117,10 @@ public void shouldCompleteAllSubscribersOnStop() { StepVerifier.create(credentialsFlux2) .assertNext(credentials -> assertThat(String.valueOf(credentials.getPassword())).isEqualTo("token-1")) .verifyComplete(); + + assertThat(subscription1.isDisposed()).isTrue(); + assertThat(subscription2.isDisposed()).isTrue(); + } @Test diff --git a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java index 7edab1567..9914d2189 100644 --- a/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java +++ b/src/test/java/io/lettuce/core/AuthenticationIntegrationTests.java @@ -2,6 +2,8 @@ import static io.lettuce.TestTags.INTEGRATION_TEST; import static org.assertj.core.api.Assertions.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import javax.inject.Inject; @@ -26,6 +28,7 @@ import io.lettuce.test.settings.TestSettings; import reactor.core.publisher.Mono; import redis.clients.authentication.core.SimpleToken; +import redis.clients.authentication.core.TokenManagerConfig; import java.time.Duration; import java.time.Instant; @@ -134,7 +137,9 @@ void tokenBasedCredentialProvider(RedisClient client) { client.setOptions(client.getOptions().mutate() .reauthenticateBehavior(ClientOptions.ReauthenticateBehavior.ON_NEW_CREDENTIALS).build()); - TestTokenManager tokenManager = new TestTokenManager(null, null); + TokenManagerConfig tokenManagerConfig = mock(TokenManagerConfig.class); + when(tokenManagerConfig.getRetryPolicy()).thenReturn(mock(TokenManagerConfig.RetryPolicy.class)); + TestTokenManager tokenManager = new TestTokenManager(null, tokenManagerConfig); TokenBasedRedisCredentialsProvider credentialsProvider = TokenBasedRedisCredentialsProvider.create(tokenManager); // Build RedisURI with streaming credentials provider From 369341c2b44670bb54e07fa498fdda36847e35a4 Mon Sep 17 00:00:00 2001 From: ggivo Date: Fri, 20 Dec 2024 16:25:56 +0200 Subject: [PATCH 28/28] add java doc for TokenBasedRedisCredentialsProvider --- .../TokenBasedRedisCredentialsProvider.java | 19 ++++++++++++ .../core/RedisAuthenticationHandler.java | 30 +++++++++++++++++-- 2 files changed, 46 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java index f7b82924a..8eb12e7bc 100644 --- a/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java +++ b/src/main/java/io/lettuce/authx/TokenBasedRedisCredentialsProvider.java @@ -18,6 +18,25 @@ import redis.clients.authentication.core.TokenListener; import redis.clients.authentication.core.TokenManager; +/** + * A {@link RedisCredentialsProvider} implementation that supports token-based authentication for Redis. + *

+ * This provider uses a {@link TokenManager} to manage and renew tokens, ensuring that the Redis client can authenticate with + * Redis using a dynamically updated token. This is particularly useful in scenarios where Redis access is controlled via + * token-based authentication, such as when Redis is integrated with an identity provider like EntraID. + *

+ *

+ * The provider supports streaming of credentials and automatically emits new credentials whenever a token is renewed. It must + * be used with {@link io.lettuce.core.ClientOptions.ReauthenticateBehavior#ON_NEW_CREDENTIALS} to automatically re-authenticate + * connections whenever new tokens are emitted by the provider. + *

+ *

+ * The lifecycle of this provider is externally managed. It should be closed when there are no longer any connections using it, + * to stop the token management process and release resources. + *

+ * + * @since 6.6 + */ public class TokenBasedRedisCredentialsProvider implements RedisCredentialsProvider, AutoCloseable { private static final Logger log = LoggerFactory.getLogger(TokenBasedRedisCredentialsProvider.class); diff --git a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java index 5a3733381..81b0bc9dd 100644 --- a/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java +++ b/src/main/java/io/lettuce/core/RedisAuthenticationHandler.java @@ -190,14 +190,33 @@ private static boolean isSupported(ClientOptions clientOptions) { } } - public void postProcess(RedisCommand toSend) { + /** + * Post-processes the command after it is sent to the server. + *

+ * If the command type is either {@link RedisCommand.Type#EXEC} or {@link RedisCommand.Type#DISCARD}, the transaction state + * is cleared and a check for deferred credentials is initiated. + *

+ * + * @param toSend the command to post-process + */ + protected void postProcess(RedisCommand toSend) { if (toSend.getType() == EXEC || toSend.getType() == DISCARD) { inTransaction.set(false); setCredentials(credentialsRef.getAndSet(null)); } } - public void postProcess(Collection> dispatched) { + /** + * Post-processes a collection of dispatched commands after they are sent to the server. + *

+ * This method checks if any of the dispatched commands indicate the completion of a transaction (via + * {@link RedisCommand.Type#EXEC} or {@link RedisCommand.Type#DISCARD}). If the transaction is complete, it clears the + * transaction state and initiates a check for deferred credentials. + *

+ * + * @param dispatched the collection of dispatched commands to post-process + */ + protected void postProcess(Collection> dispatched) { Boolean transactionComplete = null; for (RedisCommand command : dispatched) { if (command.getType() == EXEC || command.getType() == DISCARD) { @@ -348,7 +367,12 @@ public DisabledAuthenticationHandler() { } @Override - public void postProcess(RedisCommand toSend) { + protected void postProcess(RedisCommand toSend) { + // No-op + } + + @Override + protected void postProcess(Collection> dispatched) { // No-op }