From 8a9f876ec26d7b6c8d75508c250f4bb41ad50028 Mon Sep 17 00:00:00 2001 From: atakavci Date: Fri, 27 Jun 2025 19:12:02 +0300 Subject: [PATCH 01/23] - weighted cluster seleciton - Healtstatus manager with initial listener and registration logic - pluggable health checker strategy introduced, these are draft NoOpStrategy, EchoStrategy, LagAwareStrategy, - fix failing tests impacted from weighted clusters --- .../java/redis/clients/jedis/HostAndPort.java | 4 +- .../jedis/MultiClusterClientConfig.java | 23 +- .../redis/clients/jedis/UnifiedJedis.java | 14 +- .../mcf/CircuitBreakerCommandExecutor.java | 12 +- .../jedis/mcf/CircuitBreakerFailoverBase.java | 26 +- .../redis/clients/jedis/mcf/EchoStrategy.java | 32 ++ .../redis/clients/jedis/mcf/Endpoint.java | 9 + .../clients/jedis/mcf/FailoverOptions.java | 49 +++ .../redis/clients/jedis/mcf/HealthCheck.java | 107 +++++++ .../jedis/mcf/HealthCheckCollection.java | 46 +++ .../jedis/mcf/HealthCheckStrategy.java | 11 + .../redis/clients/jedis/mcf/HealthStatus.java | 19 ++ .../jedis/mcf/HealthStatusChangeEvent.java | 28 ++ .../jedis/mcf/HealthStatusListener.java | 7 + .../jedis/mcf/HealthStatusManager.java | 79 +++++ .../clients/jedis/mcf/LagAwareStrategy.java | 43 +++ .../redis/clients/jedis/mcf/NoOpStrategy.java | 20 ++ .../clients/jedis/mcf/RedisRestAPIHelper.java | 95 ++++++ .../MultiClusterPooledConnectionProvider.java | 281 +++++++++++++----- .../failover/FailoverIntegrationTest.java | 63 ++-- .../jedis/misc/AutomaticFailoverTest.java | 4 +- ...tiClusterPooledConnectionProviderTest.java | 84 ++++-- .../scenario/ActiveActiveFailoverTest.java | 4 +- 23 files changed, 878 insertions(+), 182 deletions(-) create mode 100644 src/main/java/redis/clients/jedis/mcf/EchoStrategy.java create mode 100644 src/main/java/redis/clients/jedis/mcf/Endpoint.java create mode 100644 src/main/java/redis/clients/jedis/mcf/HealthCheck.java create mode 100644 src/main/java/redis/clients/jedis/mcf/HealthCheckCollection.java create mode 100644 src/main/java/redis/clients/jedis/mcf/HealthCheckStrategy.java create mode 100644 src/main/java/redis/clients/jedis/mcf/HealthStatus.java create mode 100644 src/main/java/redis/clients/jedis/mcf/HealthStatusChangeEvent.java create mode 100644 src/main/java/redis/clients/jedis/mcf/HealthStatusListener.java create mode 100644 src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java create mode 100644 src/main/java/redis/clients/jedis/mcf/LagAwareStrategy.java create mode 100644 src/main/java/redis/clients/jedis/mcf/NoOpStrategy.java create mode 100644 src/main/java/redis/clients/jedis/mcf/RedisRestAPIHelper.java diff --git a/src/main/java/redis/clients/jedis/HostAndPort.java b/src/main/java/redis/clients/jedis/HostAndPort.java index ca51f6bad1..9548071976 100644 --- a/src/main/java/redis/clients/jedis/HostAndPort.java +++ b/src/main/java/redis/clients/jedis/HostAndPort.java @@ -2,7 +2,9 @@ import java.io.Serializable; -public class HostAndPort implements Serializable { +import redis.clients.jedis.mcf.Endpoint; + +public class HostAndPort implements Serializable, Endpoint { private static final long serialVersionUID = -519876229978427751L; diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 50f80c1c0e..b3f7e450ba 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -12,6 +12,7 @@ import redis.clients.jedis.exceptions.JedisConnectionException; import redis.clients.jedis.exceptions.JedisValidationException; import redis.clients.jedis.mcf.ConnectionFailoverException; +import redis.clients.jedis.mcf.FailoverOptions; /** * @author Allen Terleto (aterleto) @@ -195,10 +196,10 @@ public List> getFallbackExceptionList() { public static class ClusterConfig { - private int priority; private HostAndPort hostAndPort; private JedisClientConfig clientConfig; private GenericObjectPoolConfig connectionPoolConfig; + private FailoverOptions faileoverOptions = FailoverOptions.builder().build(); public ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig) { this.hostAndPort = hostAndPort; @@ -212,12 +213,12 @@ public ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig, this.connectionPoolConfig = connectionPoolConfig; } - public int getPriority() { - return priority; - } - - private void setPriority(int priority) { - this.priority = priority; + public ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig, + GenericObjectPoolConfig connectionPoolConfig, FailoverOptions failoverOptions) { + this.hostAndPort = hostAndPort; + this.clientConfig = clientConfig; + this.connectionPoolConfig = connectionPoolConfig; + this.faileoverOptions = failoverOptions; } public HostAndPort getHostAndPort() { @@ -231,6 +232,10 @@ public JedisClientConfig getJedisClientConfig() { public GenericObjectPoolConfig getConnectionPoolConfig() { return connectionPoolConfig; } + + public FailoverOptions getFailoverOptions() { + return faileoverOptions; + } } public static class Builder { @@ -258,10 +263,8 @@ public Builder(ClusterConfig[] clusterConfigs) { if (clusterConfigs == null || clusterConfigs.length < 1) throw new JedisValidationException( "ClusterClientConfigs are required for MultiClusterPooledConnectionProvider"); - for (int i = 0; i < clusterConfigs.length; i++) - clusterConfigs[i].setPriority(i + 1); - this.clusterConfigs = clusterConfigs; + this.clusterConfigs = clusterConfigs; } public Builder(List clusterConfigs) { diff --git a/src/main/java/redis/clients/jedis/UnifiedJedis.java b/src/main/java/redis/clients/jedis/UnifiedJedis.java index 21a126eaad..a217cf4adf 100644 --- a/src/main/java/redis/clients/jedis/UnifiedJedis.java +++ b/src/main/java/redis/clients/jedis/UnifiedJedis.java @@ -238,19 +238,7 @@ public UnifiedJedis(ConnectionProvider provider, int maxAttempts, Duration maxTo */ @Experimental public UnifiedJedis(MultiClusterPooledConnectionProvider provider) { - this(new CircuitBreakerCommandExecutor(provider, FailoverOptions.builder().build()), provider); - } - - /** - * Constructor which supports multiple cluster/database endpoints each with their own isolated connection pool. - *

- * With this Constructor users can seamlessly failover to Disaster Recovery (DR), Backup, and Active-Active cluster(s) - * by using simple configuration which is passed through from Resilience4j - https://resilience4j.readme.io/docs - *

- */ - @Experimental - public UnifiedJedis(MultiClusterPooledConnectionProvider provider, FailoverOptions failoverOptions) { - this(new CircuitBreakerCommandExecutor(provider, failoverOptions), provider); + this(new CircuitBreakerCommandExecutor(provider), provider); } /** diff --git a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerCommandExecutor.java b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerCommandExecutor.java index cfde64ad94..e68d15bcac 100644 --- a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerCommandExecutor.java +++ b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerCommandExecutor.java @@ -22,11 +22,8 @@ @Experimental public class CircuitBreakerCommandExecutor extends CircuitBreakerFailoverBase implements CommandExecutor { - private final FailoverOptions options; - - public CircuitBreakerCommandExecutor(MultiClusterPooledConnectionProvider provider, FailoverOptions options) { + public CircuitBreakerCommandExecutor(MultiClusterPooledConnectionProvider provider) { super(provider); - this.options = options != null ? options : FailoverOptions.builder().build(); } @Override @@ -50,8 +47,7 @@ private T handleExecuteCommand(CommandObject commandObject, Cluster clust try (Connection connection = cluster.getConnection()) { return connection.executeCommand(commandObject); } catch (Exception e) { - - if (retryOnFailover() && !isActiveCluster(cluster) + if (cluster.retryOnFailover() && !isActiveCluster(cluster) && isCircuitBreakerTrackedException(e, cluster.getCircuitBreaker())) { throw new ConnectionFailoverException( "Command failed during failover: " + cluster.getCircuitBreaker().getName(), e); @@ -65,10 +61,6 @@ private boolean isCircuitBreakerTrackedException(Exception e, CircuitBreaker cb) return cb.getCircuitBreakerConfig().getRecordExceptionPredicate().test(e); } - private boolean retryOnFailover() { - return options.isRetryOnFailover(); - } - private boolean isActiveCluster(Cluster cluster) { Cluster activeCluster = provider.getCluster(); return activeCluster != null && activeCluster.equals(cluster); diff --git a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java index ec15a5ae98..2dae7b5443 100644 --- a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java +++ b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java @@ -39,7 +39,7 @@ protected void clusterFailover(CircuitBreaker circuitBreaker) { lock.lock(); try { - // Check state to handle race conditions since incrementActiveMultiClusterIndex() is + // Check state to handle race conditions since iterateActiveCluster() is // non-idempotent if (!CircuitBreaker.State.FORCED_OPEN.equals(circuitBreaker.getState())) { @@ -48,25 +48,29 @@ protected void clusterFailover(CircuitBreaker circuitBreaker) { // To recover/transition from this forced state the user will need to manually failback circuitBreaker.transitionToForcedOpenState(); - // Incrementing the activeMultiClusterIndex will allow subsequent calls to the - // executeCommand() - // to use the next cluster's connection pool - according to the configuration's - // prioritization/order - int activeMultiClusterIndex = provider.incrementActiveMultiClusterIndex(); + // Iterating the active cluster will allow subsequent calls to the executeCommand() to use the next + // cluster's connection pool - according to the configuration's prioritization/order/weight + // int activeMultiClusterIndex = provider.incrementActiveMultiClusterIndex1(); + provider.iterateActiveCluster(); // Implementation is optionally provided during configuration. Typically, used for // activeMultiClusterIndex persistence or custom logging - provider.runClusterFailoverPostProcessor(activeMultiClusterIndex); + provider.runClusterFailoverPostProcessor(provider.getCluster()); } - - // Once the priority list is exhausted only a manual failback can open the circuit breaker so - // all subsequent operations will fail - else if (provider.isLastClusterCircuitBreakerForcedOpen()) { + // this check relies on the fact that many failover attempts can hit with the same CB, + // only the first one will trigger a failover, and make the CB FORCED_OPEN. + // when the rest reaches here, the active cluster is already the next one, and should be different than active CB. + // if its the same one and there are no more clusters to failover to, + // then throw an exception + else if (circuitBreaker == provider.getCluster().getCircuitBreaker() && !provider.canIterateOnceMore()) { throw new JedisConnectionException( "Cluster/database endpoint could not failover since the MultiClusterClientConfig was not " + "provided with an additional cluster/database endpoint according to its prioritized sequence. " + "If applicable, consider failing back OR restarting with an available cluster/database endpoint"); } + // Ignore exceptions since we are already in a failure state + } catch (RuntimeException e) { + throw e; } finally { lock.unlock(); } diff --git a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java new file mode 100644 index 0000000000..a778679be2 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java @@ -0,0 +1,32 @@ +package redis.clients.jedis.mcf; + +import java.util.function.Function; + +public class EchoStrategy implements HealthCheckStrategy { + + private int interval; + private int timeout; + private Function echo; + + public EchoStrategy(int healthCheckInterval, int healthCheckTimeout, Function echo) { + this.interval = healthCheckInterval; + this.timeout = healthCheckTimeout; + this.echo = echo; + } + + @Override + public int getInterval() { + return interval; + } + + @Override + public int getTimeout() { + return timeout; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + return "OK".equals(echo.apply(endpoint)) ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; + } + +} diff --git a/src/main/java/redis/clients/jedis/mcf/Endpoint.java b/src/main/java/redis/clients/jedis/mcf/Endpoint.java new file mode 100644 index 0000000000..e50d8b21f2 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/Endpoint.java @@ -0,0 +1,9 @@ +package redis.clients.jedis.mcf; + +public interface Endpoint { + + String getHost(); + + int getPort(); + +} diff --git a/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java b/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java index 9b919fd134..16b369b2ce 100644 --- a/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java +++ b/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java @@ -1,5 +1,6 @@ package redis.clients.jedis.mcf; +import redis.clients.jedis.HostAndPort; import redis.clients.jedis.annots.Experimental; /** @@ -8,9 +9,34 @@ @Experimental public class FailoverOptions { private final boolean retryOnFailover; + private final StrategySupplier healthCheckStrategySupplier; + private final float weight; + private boolean failback; + + private static StrategySupplier defaultStrategySupplier = (endpoint) -> new NoOpStrategy(); private FailoverOptions(Builder builder) { this.retryOnFailover = builder.retryOnFailover; + this.healthCheckStrategySupplier = builder.healthCheckStrategySupplier == null ? defaultStrategySupplier + : builder.healthCheckStrategySupplier; + this.weight = builder.weight; + this.failback = builder.failback; + } + + public static interface StrategySupplier { + HealthCheckStrategy get(HostAndPort hostAndPort); + } + + public HealthCheckStrategy getFailoverHealthCheckStrategy(HostAndPort hostAndPort) { + return healthCheckStrategySupplier.get(hostAndPort); + } + + public float getWeight() { + return weight; + } + + public boolean isFailbackEnabled() { + return failback; } /** @@ -34,6 +60,9 @@ public static Builder builder() { */ public static class Builder { private boolean retryOnFailover = false; + private StrategySupplier healthCheckStrategySupplier; + private float weight = 1.0f; + private boolean failback; private Builder() { } @@ -48,6 +77,26 @@ public Builder retryOnFailover(boolean retry) { return this; } + public Builder healthCheckStrategySupplier(StrategySupplier healthCheckStrategySupplier) { + this.healthCheckStrategySupplier = healthCheckStrategySupplier; + return this; + } + + public Builder weight(float weight) { + this.weight = weight; + return this; + } + + public Builder failback(boolean failbackEnabled) { + this.failback = failbackEnabled; + return this; + } + + public Builder healthCheckStrategy(HealthCheckStrategy healthCheckStrategy) { + this.healthCheckStrategySupplier = (hostAndPort) -> healthCheckStrategy; + return this; + } + /** * Builds a new FailoverOptions instance with the configured options * @return a new FailoverOptions instance diff --git a/src/main/java/redis/clients/jedis/mcf/HealthCheck.java b/src/main/java/redis/clients/jedis/mcf/HealthCheck.java new file mode 100644 index 0000000000..e23e0e5810 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/HealthCheck.java @@ -0,0 +1,107 @@ + +package redis.clients.jedis.mcf; + +import java.util.AbstractMap.SimpleEntry; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +public class HealthCheck { + + private Endpoint endpoint; + private HealthCheckStrategy strategy; + private AtomicReference> statusRef = new AtomicReference>(); + private Consumer statusChangeCallback; + + private ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor(); + private ExecutorService executor = Executors.newCachedThreadPool(); + + HealthCheck(Endpoint endpoint, HealthCheckStrategy strategy, + Consumer statusChangeCallback) { + this.endpoint = endpoint; + this.strategy = strategy; + this.statusChangeCallback = statusChangeCallback; + statusRef.set(new SimpleEntry<>(0L, HealthStatus.HEALTHY)); + } + + public Endpoint getEndpoint() { + return endpoint; + } + + public HealthStatus getStatus() { + return statusRef.get().getValue(); + } + + public void start() { + scheduler.scheduleAtFixedRate(this::healthCheck, 0, strategy.getInterval(), TimeUnit.MILLISECONDS); + } + + public void stop() { + this.statusChangeCallback = null; + scheduler.shutdown(); + executor.shutdown(); + try { + // Wait for graceful shutdown then force if required + if (!scheduler.awaitTermination(1, TimeUnit.SECONDS)) { + scheduler.shutdownNow(); + } + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException e) { + // Force shutdown immediately + scheduler.shutdownNow(); + executor.shutdownNow(); + Thread.currentThread().interrupt(); + } + } + + private void healthCheck() { + long me = System.currentTimeMillis(); + Future future = executor.submit(() -> { + HealthStatus newStatus = strategy.doHealthCheck(endpoint); + safeUpdate(me, newStatus); + }); + + try { + future.get(strategy.getTimeout(), TimeUnit.MILLISECONDS); + } catch (TimeoutException | ExecutionException e) { + // Cancel immediately on timeout or exec exception + future.cancel(true); + safeUpdate(me, HealthStatus.UNHEALTHY); + } catch (InterruptedException e) { + // Health check thread was interrupted + future.cancel(true); + safeUpdate(me, HealthStatus.UNHEALTHY); + Thread.currentThread().interrupt(); // Restore interrupted status + } + } + + // just to avoid to replace status with an outdated result from another healthCheck + private void safeUpdate(long owner, HealthStatus status) { + SimpleEntry newStatus = new SimpleEntry<>(owner, status); + SimpleEntry oldStatus = statusRef.getAndUpdate(current -> { + if (current.getKey() < owner) { + return newStatus; + } + return current; + }); + if (oldStatus.getKey() != owner || oldStatus.getValue() != status) { + // notify listeners + notifyListeners(oldStatus.getValue(), newStatus.getValue()); + } + } + + private void notifyListeners(HealthStatus oldStatus, HealthStatus newStatus) { + if (statusChangeCallback != null) { + statusChangeCallback.accept(new HealthStatusChangeEvent(endpoint, oldStatus, newStatus)); + } + } + +} diff --git a/src/main/java/redis/clients/jedis/mcf/HealthCheckCollection.java b/src/main/java/redis/clients/jedis/mcf/HealthCheckCollection.java new file mode 100644 index 0000000000..9584a01c6a --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/HealthCheckCollection.java @@ -0,0 +1,46 @@ +package redis.clients.jedis.mcf; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class HealthCheckCollection { + + private Map healthChecks = new ConcurrentHashMap(); + + public HealthCheck add(HealthCheck healthCheck) { + return healthChecks.put(healthCheck.getEndpoint(), healthCheck); + } + + public HealthCheck[] addAll(HealthCheck[] healthChecks) { + HealthCheck[] old = new HealthCheck[healthChecks.length]; + for (int i = 0; i < healthChecks.length; i++) { + old[i] = add(healthChecks[i]); + } + return old; + } + + public HealthCheck remove(Endpoint endpoint) { + HealthCheck old = healthChecks.remove(endpoint); + if (old != null) { + old.stop(); + } + return old; + } + + public HealthCheck remove(HealthCheck healthCheck) { + HealthCheck[] temp = new HealthCheck[1]; + healthChecks.computeIfPresent(healthCheck.getEndpoint(), (key, existing) -> { + if (existing == healthCheck) { + temp[0] = existing; + return null; + } + return existing; + }); + return temp[0]; + } + + public HealthCheck get(Endpoint endpoint) { + return healthChecks.get(endpoint); + } + +} diff --git a/src/main/java/redis/clients/jedis/mcf/HealthCheckStrategy.java b/src/main/java/redis/clients/jedis/mcf/HealthCheckStrategy.java new file mode 100644 index 0000000000..366bbb1fb8 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/HealthCheckStrategy.java @@ -0,0 +1,11 @@ +package redis.clients.jedis.mcf; + +public interface HealthCheckStrategy { + + int getInterval(); + + int getTimeout(); + + HealthStatus doHealthCheck(Endpoint endpoint); + +} diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatus.java b/src/main/java/redis/clients/jedis/mcf/HealthStatus.java new file mode 100644 index 0000000000..f28761d0b9 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatus.java @@ -0,0 +1,19 @@ +package redis.clients.jedis.mcf; + +public enum HealthStatus { + HEALTHY(0x01), UNHEALTHY(0x02), FORCED_UNHEALTHY(0x04); + + private final int value; + + HealthStatus(int val) { + this.value = val; + } + + public boolean isHealthy() { + return (this.value & HEALTHY.value) != 0; + } + + public boolean isForcedUnhealthy() { + return (this.value & FORCED_UNHEALTHY.value) != 0; + } +} diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatusChangeEvent.java b/src/main/java/redis/clients/jedis/mcf/HealthStatusChangeEvent.java new file mode 100644 index 0000000000..20e1a1fe65 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatusChangeEvent.java @@ -0,0 +1,28 @@ + +package redis.clients.jedis.mcf; + + +public class HealthStatusChangeEvent { + + private final Endpoint endpoint; + private final HealthStatus oldStatus; + private final HealthStatus newStatus; + + public HealthStatusChangeEvent(Endpoint endpoint, HealthStatus oldStatus, HealthStatus newStatus) { + this.endpoint = endpoint; + this.oldStatus = oldStatus; + this.newStatus = newStatus; + } + + public Endpoint getEndpoint() { + return endpoint; + } + + public HealthStatus getOldStatus() { + return oldStatus; + } + + public HealthStatus getNewStatus() { + return newStatus; + } +} diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatusListener.java b/src/main/java/redis/clients/jedis/mcf/HealthStatusListener.java new file mode 100644 index 0000000000..f01c2da563 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatusListener.java @@ -0,0 +1,7 @@ +package redis.clients.jedis.mcf; + +public interface HealthStatusListener { + + void onStatusChange(HealthStatusChangeEvent event); + +} diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java b/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java new file mode 100644 index 0000000000..7bb4d315f1 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java @@ -0,0 +1,79 @@ +package redis.clients.jedis.mcf; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; + +public class HealthStatusManager { + + private HealthCheckCollection healthChecks = new HealthCheckCollection(); + private final List listeners = new CopyOnWriteArrayList<>(); + private final Map> endpointListeners = new ConcurrentHashMap>(); + + public void registerListener(HealthStatusListener listener) { + listeners.add(listener); + } + + public void unregisterListener(HealthStatusListener listener) { + listeners.remove(listener); + } + + public void registerListener(Endpoint endpoint, HealthStatusListener listener) { + endpointListeners.computeIfAbsent(endpoint, k -> new CopyOnWriteArrayList<>()).add(listener); + listeners.add(listener); + } + + public void unregisterListener(Endpoint endpoint, HealthStatusListener listener) { + endpointListeners.computeIfPresent(endpoint, (k, v) -> { + v.remove(listener); + return v; + }); + listeners.remove(listener); + } + + public void notifyListeners(HealthStatusChangeEvent eventArgs) { + endpointListeners.computeIfPresent(eventArgs.getEndpoint(), (k, v) -> { + for (HealthStatusListener listener : v) { + listener.onStatusChange(eventArgs); + } + return v; + }); + for (HealthStatusListener listener : listeners) { + listener.onStatusChange(eventArgs); + } + } + + public void add(Endpoint endpoint, HealthCheckStrategy strategy) { + HealthCheck hc = new HealthCheck(endpoint, strategy, this::notifyListeners); + HealthCheck old = healthChecks.add(hc); + hc.start(); + if (old != null) { + old.stop(); + } + } + + public void addAll(Endpoint[] endpoints, HealthCheckStrategy strategy) { + for (Endpoint endpoint : endpoints) { + add(endpoint, strategy); + } + } + + public void remove(Endpoint endpoint) { + HealthCheck old = healthChecks.remove(endpoint); + if (old != null) { + old.stop(); + } + } + + public void removeAll(Endpoint[] endpoints) { + for (Endpoint endpoint : endpoints) { + remove(endpoint); + } + } + + public HealthStatus getHealthStatus(Endpoint endpoint) { + HealthCheck healthCheck = healthChecks.get(endpoint); + return healthCheck != null ? healthCheck.getStatus() : HealthStatus.UNHEALTHY; + } +} diff --git a/src/main/java/redis/clients/jedis/mcf/LagAwareStrategy.java b/src/main/java/redis/clients/jedis/mcf/LagAwareStrategy.java new file mode 100644 index 0000000000..60124f0854 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/LagAwareStrategy.java @@ -0,0 +1,43 @@ +package redis.clients.jedis.mcf; + +import java.io.IOException; +import java.util.List; + +public class LagAwareStrategy implements HealthCheckStrategy { + + private int interval; + private int timeout; + + public LagAwareStrategy(int healthCheckInterval, int healthCheckTimeout) { + this.interval = healthCheckInterval; + this.timeout = healthCheckTimeout; + } + + @Override + public int getInterval() { + return interval; + } + + @Override + public int getTimeout() { + return timeout; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + RedisRestAPIHelper helper = new RedisRestAPIHelper(endpoint.getHost(), String.valueOf(endpoint.getPort()), + "admin", "admin"); + try { + List bdbs = helper.getBdbs(); + if (bdbs.size() > 0) { + if ("available".equals(helper.checkBdbAvailability(bdbs.get(0)))) { + return HealthStatus.HEALTHY; + } + } + } catch (IOException e) { + // log error + return HealthStatus.UNHEALTHY; + } + return HealthStatus.UNHEALTHY; + } +} diff --git a/src/main/java/redis/clients/jedis/mcf/NoOpStrategy.java b/src/main/java/redis/clients/jedis/mcf/NoOpStrategy.java new file mode 100644 index 0000000000..e2f76cbee5 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/NoOpStrategy.java @@ -0,0 +1,20 @@ +package redis.clients.jedis.mcf; + +class NoOpStrategy implements HealthCheckStrategy { + + @Override + public int getInterval() { + return 1000; + } + + @Override + public int getTimeout() { + return 1000; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + return HealthStatus.HEALTHY; + } + +} diff --git a/src/main/java/redis/clients/jedis/mcf/RedisRestAPIHelper.java b/src/main/java/redis/clients/jedis/mcf/RedisRestAPIHelper.java new file mode 100644 index 0000000000..8863e49d46 --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/RedisRestAPIHelper.java @@ -0,0 +1,95 @@ +package redis.clients.jedis.mcf; + +import java.io.IOException; +import java.io.InputStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.List; +import java.util.stream.Collectors; + +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; + +/** + * Helper class to check the availability of a Redis database + */ +class RedisRestAPIHelper { + // Required connection information - replace with your host, port, username, and + // password + private final String host; + private final String port; + private final String userName; + private final String password; + + RedisRestAPIHelper(String host, String port, String userName, String password) { + this.host = host; + this.port = port; + this.userName = userName; + this.password = password; + } + + private static final String BDBS_URL = "https://%s:%s/v1/bdbs"; + private static final String AVAILABILITY_URL = "https://%s:%s/v1/bdbs/%d/availability"; + + public List getBdbs() throws IOException { + String bdbsUri = String.format(BDBS_URL, host, port); + HttpURLConnection getConnection = createConnection(bdbsUri, "GET"); + getConnection.setRequestProperty("Accept", "application/json"); + + String responseBody = readResponse(getConnection); + return JsonParser.parseString(responseBody).getAsJsonArray().asList().stream().map(e -> e.getAsString()) + .collect(Collectors.toList()); + } + + public String checkBdbAvailability(String uid) throws IOException { + String availabilityUri = String.format(AVAILABILITY_URL, host, port, uid); + HttpURLConnection availConnection = createConnection(availabilityUri, "GET"); + availConnection.setRequestProperty("Accept", "application/json"); + + String availResponse = readResponse(availConnection); + JsonObject availJson = JsonParser.parseString(availResponse).getAsJsonObject(); + + return availJson.get("status").getAsString(); + + } + + private HttpURLConnection createConnection(String urlString, String method) throws IOException { + URL url = new URL(urlString); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod(method); + + // Set basic authentication + String auth = userName + ":" + password; + String encodedAuth = Base64.getEncoder().encodeToString(auth.getBytes(StandardCharsets.UTF_8)); + connection.setRequestProperty("Authorization", "Basic " + encodedAuth); + + return connection; + } + + private String readResponse(HttpURLConnection connection) throws IOException { + InputStream inputStream; + try { + inputStream = connection.getInputStream(); + } catch (IOException e) { + // If there's an error, try to read from error stream + inputStream = connection.getErrorStream(); + if (inputStream == null) { + return ""; + } + } + + StringBuilder response = new StringBuilder(); + byte[] buffer = new byte[1024]; + int bytesRead; + + while ((bytesRead = inputStream.read(buffer)) != -1) { + response.append(new String(buffer, 0, bytesRead, StandardCharsets.UTF_8)); + } + + inputStream.close(); + return response.toString(); + } + +} \ No newline at end of file diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index afa9cf6c37..e0dd55221a 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -10,12 +10,14 @@ import io.github.resilience4j.retry.RetryRegistry; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; +import java.util.function.Predicate; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.slf4j.Logger; @@ -27,7 +29,12 @@ import redis.clients.jedis.annots.VisibleForTesting; import redis.clients.jedis.exceptions.JedisConnectionException; import redis.clients.jedis.exceptions.JedisValidationException; +import redis.clients.jedis.mcf.HealthStatus; +import redis.clients.jedis.mcf.HealthStatusChangeEvent; +import redis.clients.jedis.mcf.HealthStatusManager; import redis.clients.jedis.util.Pool; +import redis.clients.jedis.mcf.Endpoint; +import redis.clients.jedis.mcf.FailoverOptions; /** * @author Allen Terleto (aterleto) @@ -50,24 +57,16 @@ public class MultiClusterPooledConnectionProvider implements ConnectionProvider * Ordered map of cluster/database endpoints which were provided at startup via the MultiClusterClientConfig. Users * can move down (failover) or (up) failback the map depending on their availability and order. */ - private final Map multiClusterMap = new ConcurrentHashMap<>(); + private final Map multiClusterMap = new ConcurrentHashMap<>(); /** * Indicates the actively used cluster/database endpoint (connection pool) amongst the pre-configured list which - * were provided at startup via the MultiClusterClientConfig. All traffic will be routed according to this index. + * were provided at startup via the MultiClusterClientConfig. All traffic will be routed with this cluster/database */ - private volatile Integer activeMultiClusterIndex = 1; + private volatile Cluster activeCluster; private final Lock activeClusterIndexLock = new ReentrantLock(true); - /** - * Indicates the final cluster/database endpoint (connection pool), according to the pre-configured list provided at - * startup via the MultiClusterClientConfig, is unavailable and therefore no further failover is possible. Users can - * manually failback to an available cluster which would reset this flag via - * {@link #setActiveMultiClusterIndex(int)} - */ - private volatile boolean lastClusterCircuitBreakerForcedOpen = false; - /** * Functional interface typically used for activeMultiClusterIndex persistence or custom logging after a successful * failover of a cluster/database endpoint (connection pool). Cluster/database endpoint info is passed as the sole @@ -77,6 +76,8 @@ public class MultiClusterPooledConnectionProvider implements ConnectionProvider private List> fallbackExceptionList; + private HealthStatusManager healthStatusManager = new HealthStatusManager(); + public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiClusterClientConfig) { if (multiClusterClientConfig == null) throw new JedisValidationException( @@ -130,7 +131,7 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste for (ClusterConfig config : clusterConfigs) { GenericObjectPoolConfig poolConfig = config.getConnectionPoolConfig(); - String clusterId = "cluster:" + config.getPriority() + ":" + config.getHostAndPort(); + String clusterId = "cluster:" + config.getFailoverOptions().getWeight() + ":" + config.getHostAndPort(); Retry retry = RetryRegistry.of(retryConfig).retry(clusterId); @@ -147,61 +148,86 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste circuitBreakerEventPublisher.onSlowCallRateExceeded(event -> log.error(String.valueOf(event))); circuitBreakerEventPublisher.onStateTransition(event -> log.warn(String.valueOf(event))); + ConnectionPool pool; if (poolConfig != null) { - multiClusterMap.put(config.getPriority(), - new Cluster(new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig(), poolConfig), - retry, circuitBreaker)); + pool = new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig(), poolConfig); } else { - multiClusterMap.put(config.getPriority(), new Cluster( - new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig()), retry, circuitBreaker)); + pool = new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig()); } + Cluster cluster = new Cluster(pool, retry, circuitBreaker, config.getFailoverOptions()); + multiClusterMap.put(config.getHostAndPort(), cluster); + + healthStatusManager.add(config.getHostAndPort(), + config.getFailoverOptions().getFailoverHealthCheckStrategy(config.getHostAndPort())); } + // selecting activeCluster with configuration values. + // all health status would be HEALTHY at this point + activeCluster = findWeightedHealthyCluster().getValue(); + + for (Endpoint endpoint : multiClusterMap.keySet()) { + healthStatusManager.registerListener(endpoint, this::handleStatusChange); + } /// --- /// this.fallbackExceptionList = multiClusterClientConfig.getFallbackExceptionList(); } - /** - * Increments the actively used cluster/database endpoint (connection pool) amongst the pre-configured list which - * were provided at startup via the MultiClusterClientConfig. All traffic will be routed according to this index. - * Only indexes within the pre-configured range (static) are supported otherwise an exception will be thrown. In the - * event that the next prioritized connection has a forced open state, the method will recursively increment the - * index in order to avoid a failed command. - */ - public int incrementActiveMultiClusterIndex() { + private void handleStatusChange(HealthStatusChangeEvent eventArgs) { - // Field-level synchronization is used to avoid the edge case in which - // setActiveMultiClusterIndex(int multiClusterIndex) is called at the same time - activeClusterIndexLock.lock(); + Endpoint endpoint = eventArgs.getEndpoint(); + HealthStatus newStatus = eventArgs.getNewStatus(); + log.info("Health status changed for {} from {} to {}", endpoint, eventArgs.getOldStatus(), newStatus); - try { - String originalClusterName = getClusterCircuitBreaker().getName(); + Cluster clusterWithHealthChange = multiClusterMap.get(endpoint); - // Only increment if it can pass this validation otherwise we will need to check for NULL in the data path - if (activeMultiClusterIndex + 1 > multiClusterMap.size()) { + if (clusterWithHealthChange == null) return; + if (clusterWithHealthChange.isForcedUnhealthy()) return; - lastClusterCircuitBreakerForcedOpen = true; + clusterWithHealthChange.setHealthStatus(newStatus); - throw new JedisConnectionException( - "Cluster/database endpoint could not failover since the MultiClusterClientConfig was not " - + "provided with an additional cluster/database endpoint according to its prioritized sequence. " - + "If applicable, consider failing back OR restarting with an available cluster/database endpoint."); - } else activeMultiClusterIndex++; + if (newStatus.isHealthy()) { + if (clusterWithHealthChange.isFailbackEnabled() && activeCluster != clusterWithHealthChange) { + // lets check if weighted switching is possible + Map.Entry failbackCluster = findWeightedFailbackCluster(); + if (failbackCluster == clusterWithHealthChange + && clusterWithHealthChange.getWeight() > activeCluster.getWeight()) { + setActiveCluster(clusterWithHealthChange, false); + } + } + } else if (clusterWithHealthChange == activeCluster) { + iterateActiveCluster(); + } + } - CircuitBreaker circuitBreaker = getClusterCircuitBreaker(); + public Endpoint iterateActiveCluster() { + Map.Entry clusterToIterate = findWeightedHealthyCluster(); + if (clusterToIterate == null) { + throw new JedisConnectionException( + "Cluster/database endpoint could not failover since the MultiClusterClientConfig was not " + + "provided with an additional cluster/database endpoint according to its prioritized sequence. " + + "If applicable, consider failing back OR restarting with an available cluster/database endpoint"); + } + setActiveCluster(clusterToIterate.getValue(), false); + return clusterToIterate.getKey(); + } - // Handles edge-case in which the user resets the activeMultiClusterIndex to a higher priority prematurely - // which forces a failover to the next prioritized cluster that has potentially not yet recovered - if (CircuitBreaker.State.FORCED_OPEN.equals(circuitBreaker.getState())) incrementActiveMultiClusterIndex(); + private static Comparator> maxByWeight = Map.Entry + . comparingByValue(Comparator.comparing(Cluster::getWeight)); - else log.warn("Cluster/database endpoint successfully updated from '{}' to '{}'", originalClusterName, - circuitBreaker.getName()); - } finally { - activeClusterIndexLock.unlock(); - } + private static Predicate> filterByHealth = c -> c.getValue().isHealthy(); + private static Predicate> filterByFailback = c -> c.getValue().isHealthy(); - return activeMultiClusterIndex; + private Map.Entry findWeightedHealthyCluster() { + Cluster current = activeCluster; + return multiClusterMap.entrySet().stream().filter(filterByHealth).filter(entry -> entry.getValue() != current) + .max(maxByWeight).orElse(null); + } + + private Map.Entry findWeightedFailbackCluster() { + Cluster current = activeCluster; + return multiClusterMap.entrySet().stream().filter(filterByHealth).filter(filterByFailback) + .filter(entry -> entry.getValue() != current).max(maxByWeight).orElse(null); } /** @@ -209,9 +235,13 @@ public int incrementActiveMultiClusterIndex() { * there was discussion to handle cross-cluster replication validation by setting a key/value pair per hashslot in * the active connection (with a TTL) and subsequently reading it from the target connection. */ - public void validateTargetConnection(int multiClusterIndex) { + public void validateTargetConnection(Endpoint endpoint) { + Cluster cluster = multiClusterMap.get(endpoint); + validateTargetConnection(cluster); + } - CircuitBreaker circuitBreaker = getClusterCircuitBreaker(multiClusterIndex); + private void validateTargetConnection(Cluster cluster) { + CircuitBreaker circuitBreaker = cluster.getCircuitBreaker(); State originalState = circuitBreaker.getState(); try { @@ -220,14 +250,14 @@ public void validateTargetConnection(int multiClusterIndex) { // yet circuitBreaker.transitionToClosedState(); - try (Connection targetConnection = getConnection(multiClusterIndex)) { + try (Connection targetConnection = cluster.getConnection()) { targetConnection.ping(); } } catch (Exception e) { // If the original state was FORCED_OPEN, then transition it back which stops state transition, metrics and // event publishing - if (CircuitBreaker.State.FORCED_OPEN.equals(originalState)) circuitBreaker.transitionToForcedOpenState(); + if (State.FORCED_OPEN.equals(originalState)) circuitBreaker.transitionToForcedOpenState(); throw new JedisValidationException( circuitBreaker.getName() + " failed to connect. Please check configuration and try again.", e); @@ -240,8 +270,54 @@ public void validateTargetConnection(int multiClusterIndex) { * provided new index. Special care should be taken to confirm cluster/database availability AND potentially * cross-cluster replication BEFORE using this capability. */ - public void setActiveMultiClusterIndex(int multiClusterIndex) { + // public void setActiveMultiClusterIndex(int multiClusterIndex) { + + // // Field-level synchronization is used to avoid the edge case in which + // // incrementActiveMultiClusterIndex() is called at the same time + // activeClusterIndexLock.lock(); + + // try { + + // // Allows an attempt to reset the current cluster from a FORCED_OPEN to CLOSED state in the event that no + // // failover is possible + // if (activeMultiClusterIndex == multiClusterIndex + // && !CircuitBreaker.State.FORCED_OPEN.equals(getClusterCircuitBreaker(multiClusterIndex).getState())) + // return; + + // if (multiClusterIndex < 1 || multiClusterIndex > multiClusterMap.size()) + // throw new JedisValidationException("MultiClusterIndex: " + multiClusterIndex + " is not within " + // + "the configured range. Please choose an index between 1 and " + multiClusterMap.size()); + // validateTargetConnection(multiClusterIndex); + + // String originalClusterName = getClusterCircuitBreaker().getName(); + + // if (activeMultiClusterIndex == multiClusterIndex) + // log.warn("Cluster/database endpoint '{}' successfully closed its circuit breaker", originalClusterName); + // else log.warn("Cluster/database endpoint successfully updated from '{}' to '{}'", originalClusterName, + // getClusterCircuitBreaker(multiClusterIndex).getName()); + + // activeMultiClusterIndex = multiClusterIndex; + // // lastClusterCircuitBreakerForcedOpen = false; + // } finally { + // activeClusterIndexLock.unlock(); + // } + // } + + public void setActiveCluster(Endpoint endpoint) { + if (endpoint == null) { + throw new JedisValidationException("Provided endpoint is null. Please use one from the configuration"); + } + Cluster cluster = multiClusterMap.get(endpoint); + if (cluster == null) { + throw new JedisValidationException("Provided endpoint: " + endpoint + " is not within " + + "the configured endpoints. Please use one from the configuration"); + } + setActiveCluster(cluster, true); + } + + private void setActiveCluster(Cluster cluster, boolean validateConnection) { + // Cluster cluster = clusterEntry.getValue(); // Field-level synchronization is used to avoid the edge case in which // incrementActiveMultiClusterIndex() is called at the same time activeClusterIndexLock.lock(); @@ -250,25 +326,20 @@ public void setActiveMultiClusterIndex(int multiClusterIndex) { // Allows an attempt to reset the current cluster from a FORCED_OPEN to CLOSED state in the event that no // failover is possible - if (activeMultiClusterIndex == multiClusterIndex - && !CircuitBreaker.State.FORCED_OPEN.equals(getClusterCircuitBreaker(multiClusterIndex).getState())) - return; + if (activeCluster == cluster && !cluster.isCBForcedOpen()) return; - if (multiClusterIndex < 1 || multiClusterIndex > multiClusterMap.size()) - throw new JedisValidationException("MultiClusterIndex: " + multiClusterIndex + " is not within " - + "the configured range. Please choose an index between 1 and " + multiClusterMap.size()); - - validateTargetConnection(multiClusterIndex); + if (validateConnection) validateTargetConnection(cluster); String originalClusterName = getClusterCircuitBreaker().getName(); - if (activeMultiClusterIndex == multiClusterIndex) + if (activeCluster == cluster) log.warn("Cluster/database endpoint '{}' successfully closed its circuit breaker", originalClusterName); else log.warn("Cluster/database endpoint successfully updated from '{}' to '{}'", originalClusterName, - getClusterCircuitBreaker(multiClusterIndex).getName()); - - activeMultiClusterIndex = multiClusterIndex; - lastClusterCircuitBreakerForcedOpen = false; + cluster.circuitBreaker.getName()); + if (!activeCluster.isFailbackEnabled()) { + activeCluster.setHealthStatus(HealthStatus.FORCED_UNHEALTHY); + } + activeCluster = cluster; } finally { activeClusterIndexLock.unlock(); } @@ -276,53 +347,59 @@ public void setActiveMultiClusterIndex(int multiClusterIndex) { @Override public void close() { - multiClusterMap.get(activeMultiClusterIndex).getConnectionPool().close(); + activeCluster.getConnectionPool().close(); } @Override public Connection getConnection() { - return multiClusterMap.get(activeMultiClusterIndex).getConnection(); + return activeCluster.getConnection(); } - public Connection getConnection(int multiClusterIndex) { - return multiClusterMap.get(multiClusterIndex).getConnection(); + public Connection getConnection(Endpoint endpoint) { + return multiClusterMap.get(endpoint).getConnection(); } @Override public Connection getConnection(CommandArguments args) { - return multiClusterMap.get(activeMultiClusterIndex).getConnection(); + return activeCluster.getConnection(); } @Override public Map> getConnectionMap() { - ConnectionPool connectionPool = multiClusterMap.get(activeMultiClusterIndex).getConnectionPool(); + ConnectionPool connectionPool = activeCluster.getConnectionPool(); return Collections.singletonMap(connectionPool.getFactory(), connectionPool); } public Cluster getCluster() { - return multiClusterMap.get(activeMultiClusterIndex); + return activeCluster; } @VisibleForTesting - public Cluster getCluster(int multiClusterIndex) { + public Cluster getCluster(Endpoint multiClusterIndex) { return multiClusterMap.get(multiClusterIndex); } public CircuitBreaker getClusterCircuitBreaker() { - return multiClusterMap.get(activeMultiClusterIndex).getCircuitBreaker(); + return activeCluster.getCircuitBreaker(); } public CircuitBreaker getClusterCircuitBreaker(int multiClusterIndex) { - return multiClusterMap.get(multiClusterIndex).getCircuitBreaker(); + return activeCluster.getCircuitBreaker(); } - public boolean isLastClusterCircuitBreakerForcedOpen() { - return lastClusterCircuitBreakerForcedOpen; + /** + * Indicates the final cluster/database endpoint (connection pool), according to the pre-configured list provided at + * startup via the MultiClusterClientConfig, is unavailable and therefore no further failover is possible. Users can + * manually failback to an available cluster + */ + public boolean canIterateOnceMore() { + Map.Entry e = findWeightedHealthyCluster(); + return e != null; } - public void runClusterFailoverPostProcessor(Integer multiClusterIndex) { + public void runClusterFailoverPostProcessor(Cluster cluster) { if (clusterFailoverPostProcessor != null) - clusterFailoverPostProcessor.accept(getClusterCircuitBreaker(multiClusterIndex).getName()); + clusterFailoverPostProcessor.accept(cluster.getCircuitBreaker().getName()); } public void setClusterFailoverPostProcessor(Consumer clusterFailoverPostProcessor) { @@ -338,11 +415,18 @@ public static class Cluster { private final ConnectionPool connectionPool; private final Retry retry; private final CircuitBreaker circuitBreaker; + private final float weight; + // it starts its life with the assumption of being healthy + private HealthStatus healthStatus = HealthStatus.HEALTHY; + private FailoverOptions failoverOptions; - public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circuitBreaker) { + public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circuitBreaker, + FailoverOptions failoverOptions) { this.connectionPool = connectionPool; this.retry = retry; this.circuitBreaker = circuitBreaker; + this.failoverOptions = failoverOptions; + this.weight = failoverOptions.getWeight(); } public Connection getConnection() { @@ -360,6 +444,41 @@ public Retry getRetry() { public CircuitBreaker getCircuitBreaker() { return circuitBreaker; } + + public HealthStatus getHealthStatus() { + return healthStatus; + } + + public void setHealthStatus(HealthStatus healthStatus) { + this.healthStatus = healthStatus; + } + + /** + * Assigned weight for this cluster + */ + public float getWeight() { + return weight; + } + + public boolean isCBForcedOpen() { + return circuitBreaker.getState() == CircuitBreaker.State.FORCED_OPEN; + } + + public boolean isHealthy() { + return healthStatus.isHealthy() && !isCBForcedOpen(); + } + + public boolean isFailbackEnabled() { + return failoverOptions.isFailbackEnabled(); + } + + public boolean isForcedUnhealthy() { + return healthStatus.isForcedUnhealthy(); + } + + public boolean retryOnFailover() { + return failoverOptions.isRetryOnFailover(); + } } } \ No newline at end of file diff --git a/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java b/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java index dcd5e681a9..b7a9b4a4d7 100644 --- a/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java @@ -109,7 +109,7 @@ public void setup() throws IOException { // Create default provider and client for most tests provider = createProvider(); - failoverClient = createClient(provider, null); + failoverClient = new UnifiedJedis(provider); } @AfterEach @@ -143,7 +143,8 @@ public void testAutomaticFailoverWhenServerBecomesUnavailable() throws Exception // 3. Subsequent calls should be routed to Endpoint 2 assertThrows(JedisConnectionException.class, () -> failoverClient.info("server")); - assertThat(provider.getCluster(1).getCircuitBreaker().getState(), equalTo(CircuitBreaker.State.OPEN)); + assertThat(provider.getCluster(endpoint1.getHostAndPort()).getCircuitBreaker().getState(), + equalTo(CircuitBreaker.State.OPEN)); // Check that the failoverClient is now using Endpoint 2 assertThat(getNodeId(failoverClient.info("server")), equalTo(JEDIS2_ID)); @@ -153,7 +154,8 @@ public void testAutomaticFailoverWhenServerBecomesUnavailable() throws Exception // Endpoint1 and Endpoint2 are NOT available, assertThrows(JedisConnectionException.class, () -> failoverClient.info("server")); - assertThat(provider.getCluster(2).getCircuitBreaker().getState(), equalTo(CircuitBreaker.State.OPEN)); + assertThat(provider.getCluster(endpoint2.getHostAndPort()).getCircuitBreaker().getState(), + equalTo(CircuitBreaker.State.OPEN)); // and since no other nodes are available, it should propagate the errors to the caller // subsequent calls @@ -164,7 +166,7 @@ public void testAutomaticFailoverWhenServerBecomesUnavailable() throws Exception public void testManualFailoverNewCommandsAreSentToActiveCluster() throws InterruptedException { assertThat(getNodeId(failoverClient.info("server")), equalTo(JEDIS1_ID)); - provider.setActiveMultiClusterIndex(2); + provider.setActiveCluster(endpoint2.getHostAndPort()); assertThat(getNodeId(failoverClient.info("server")), equalTo(JEDIS2_ID)); } @@ -177,6 +179,14 @@ private List getClusterConfigs(JedisClie .collect(Collectors.toList()); } + private List getClusterConfigs(JedisClientConfig clientConfig, + FailoverOptions failoverOptions, EndpointConfig... endpoints) { + + return Arrays.stream(endpoints).map( + e -> new MultiClusterClientConfig.ClusterConfig(e.getHostAndPort(), clientConfig, null, failoverOptions)) + .collect(Collectors.toList()); + } + @Test @Timeout(5) public void testManualFailoverInflightCommandsCompleteGracefully() throws ExecutionException, InterruptedException { @@ -186,14 +196,15 @@ public void testManualFailoverInflightCommandsCompleteGracefully() throws Execut // We will trigger failover while this command is in-flight Future> blpop = executor.submit(() -> failoverClient.blpop(1000, "test-list")); - provider.setActiveMultiClusterIndex(2); + provider.setActiveCluster(endpoint2.getHostAndPort()); // After the manual failover, commands should be executed against Endpoint 2 assertThat(getNodeId(failoverClient.info("server")), equalTo(JEDIS2_ID)); // Failover was manually triggered, and there were no errors // previous endpoint CB should still be in CLOSED state - assertThat(provider.getCluster(1).getCircuitBreaker().getState(), equalTo(CircuitBreaker.State.CLOSED)); + assertThat(provider.getCluster(endpoint1.getHostAndPort()).getCircuitBreaker().getState(), + equalTo(CircuitBreaker.State.CLOSED)); jedis1.rpush("test-list", "somevalue"); @@ -211,7 +222,7 @@ public void testManualFailoverInflightCommandsWithErrorsPropagateError() throws Future> blpop = executor.submit(() -> failoverClient.blpop(10000, "test-list-1")); // trigger failover manually - provider.setActiveMultiClusterIndex(2); + provider.setActiveCluster(endpoint2.getHostAndPort()); Future infoCmd = executor.submit(() -> failoverClient.info("server")); // After the manual failover, commands should be executed against Endpoint 2 @@ -225,7 +236,8 @@ public void testManualFailoverInflightCommandsWithErrorsPropagateError() throws assertThat(exception.getCause(), instanceOf(JedisConnectionException.class)); // Check that the circuit breaker for Endpoint 1 is open after the error - assertThat(provider.getCluster(1).getCircuitBreaker().getState(), equalTo(CircuitBreaker.State.OPEN)); + assertThat(provider.getCluster(endpoint1.getHostAndPort()).getCircuitBreaker().getState(), + equalTo(CircuitBreaker.State.OPEN)); // Ensure that the active cluster is still Endpoint 2 assertThat(getNodeId(failoverClient.info("server")), equalTo(JEDIS2_ID)); @@ -272,7 +284,8 @@ public void testCircuitBreakerCountsEachConnectionErrorSeparately() throws IOExc assertThrows(JedisConnectionException.class, () -> client.info("server")); // Circuit breaker should be open after just one command with retries - assertThat(provider.getCluster(1).getCircuitBreaker().getState(), equalTo(CircuitBreaker.State.OPEN)); + assertThat(provider.getCluster(endpoint1.getHostAndPort()).getCircuitBreaker().getState(), + equalTo(CircuitBreaker.State.OPEN)); // Next command should be routed to the second endpoint // Command 2 @@ -291,12 +304,13 @@ public void testCircuitBreakerCountsEachConnectionErrorSeparately() throws IOExc @Test public void testInflightCommandsAreRetriedAfterFailover() throws Exception { - MultiClusterPooledConnectionProvider customProvider = createProvider(); + MultiClusterPooledConnectionProvider customProvider = createProvider(builder -> builder.retryOnFailover(true)); // Create a custom client with retryOnFailover enabled for this specific test - try (UnifiedJedis customClient = createClient(customProvider, builder -> builder.retryOnFailover(true))) { + try (UnifiedJedis customClient = new UnifiedJedis(customProvider)) { assertThat(getNodeId(customClient.info("server")), equalTo(JEDIS1_ID)); + Thread.sleep(1000); // We will trigger failover while this command is in-flight Future> blpop = executor.submit(() -> customClient.blpop(10000, "test-list-1")); @@ -311,7 +325,7 @@ public void testInflightCommandsAreRetriedAfterFailover() throws Exception { // immediately when CB state change to OPEN/FORCED_OPENs assertThat(getNodeId(customClient.info("server")), equalTo(JEDIS2_ID)); // Check that the circuit breaker for Endpoint 1 is open - assertThat(customProvider.getCluster(1).getCircuitBreaker().getState(), + assertThat(customProvider.getCluster(endpoint1.getHostAndPort()).getCircuitBreaker().getState(), equalTo(CircuitBreaker.State.FORCED_OPEN)); // Disable redisProxy1 to enforce connection drop for the in-flight (blpop) command @@ -329,9 +343,9 @@ public void testInflightCommandsAreRetriedAfterFailover() throws Exception { @Test public void testInflightCommandsAreNotRetriedAfterFailover() throws Exception { // Create a custom provider and client with retry disabled for this specific test - MultiClusterPooledConnectionProvider customProvider = createProvider(); + MultiClusterPooledConnectionProvider customProvider = createProvider(builder -> builder.retryOnFailover(false)); - try (UnifiedJedis customClient = createClient(customProvider, builder -> builder.retryOnFailover(false))) { + try (UnifiedJedis customClient = new UnifiedJedis(customProvider)) { assertThat(getNodeId(customClient.info("server")), equalTo(JEDIS1_ID)); Future> blpop = executor.submit(() -> customClient.blpop(500, "test-list-2")); @@ -342,7 +356,8 @@ public void testInflightCommandsAreNotRetriedAfterFailover() throws Exception { assertThrows(JedisConnectionException.class, () -> customClient.set("test-key", generateTestValue(150))); // Check that the circuit breaker for Endpoint 1 is open - assertThat(customProvider.getCluster(1).getCircuitBreaker().getState(), equalTo(CircuitBreaker.State.OPEN)); + assertThat(customProvider.getCluster(endpoint1.getHostAndPort()).getCircuitBreaker().getState(), + equalTo(CircuitBreaker.State.OPEN)); // Disable redisProxy1 to enforce the current blpop command failure redisProxy1.disable(); @@ -398,18 +413,24 @@ private MultiClusterPooledConnectionProvider createProvider() { } /** - * Creates a UnifiedJedis client with customizable failover options - * @param provider The connection provider to use - * @param optionsCustomizer A function that customizes the failover options (can be null for defaults) - * @return A configured failover client + * Creates a MultiClusterPooledConnectionProvider with standard configuration + * @return A configured provider */ - private UnifiedJedis createClient(MultiClusterPooledConnectionProvider provider, + private MultiClusterPooledConnectionProvider createProvider( Function optionsCustomizer) { FailoverOptions.Builder builder = FailoverOptions.builder(); if (optionsCustomizer != null) { builder = optionsCustomizer.apply(builder); } + JedisClientConfig clientConfig = DefaultJedisClientConfig.builder() + .socketTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS) + .connectionTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS).build(); - return new UnifiedJedis(provider, builder.build()); + MultiClusterClientConfig failoverConfig = new MultiClusterClientConfig.Builder( + getClusterConfigs(clientConfig, builder.build(), endpoint1, endpoint2)).retryMaxAttempts(1) + .retryWaitDuration(1).circuitBreakerSlidingWindowType(COUNT_BASED).circuitBreakerSlidingWindowSize(1) + .circuitBreakerFailureRateThreshold(100).circuitBreakerSlidingWindowMinCalls(1).build(); + + return new MultiClusterPooledConnectionProvider(failoverConfig); } } diff --git a/src/test/java/redis/clients/jedis/misc/AutomaticFailoverTest.java b/src/test/java/redis/clients/jedis/misc/AutomaticFailoverTest.java index c6ccbc6636..b9ff226303 100644 --- a/src/test/java/redis/clients/jedis/misc/AutomaticFailoverTest.java +++ b/src/test/java/redis/clients/jedis/misc/AutomaticFailoverTest.java @@ -65,7 +65,7 @@ public void pipelineWithSwitch() { AbstractPipeline pipe = client.pipelined(); pipe.set("pstr", "foobar"); pipe.hset("phash", "foo", "bar"); - provider.incrementActiveMultiClusterIndex(); + provider.iterateActiveCluster(); pipe.sync(); } @@ -83,7 +83,7 @@ public void transactionWithSwitch() { AbstractTransaction tx = client.multi(); tx.set("tstr", "foobar"); tx.hset("thash", "foo", "bar"); - provider.incrementActiveMultiClusterIndex(); + provider.iterateActiveCluster(); assertEquals(Arrays.asList("OK", 1L), tx.exec()); } diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java index 8996f0e285..9e84d5d01f 100644 --- a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java @@ -7,6 +7,8 @@ import redis.clients.jedis.MultiClusterClientConfig.ClusterConfig; import redis.clients.jedis.exceptions.JedisConnectionException; import redis.clients.jedis.exceptions.JedisValidationException; +import redis.clients.jedis.mcf.Endpoint; +import redis.clients.jedis.mcf.FailoverOptions; import java.util.concurrent.atomic.AtomicBoolean; @@ -26,10 +28,13 @@ public class MultiClusterPooledConnectionProviderTest { public void setUp() { ClusterConfig[] clusterConfigs = new ClusterConfig[2]; - clusterConfigs[0] = new ClusterConfig(endpointStandalone0.getHostAndPort(), endpointStandalone0.getClientConfigBuilder().build()); - clusterConfigs[1] = new ClusterConfig(endpointStandalone1.getHostAndPort(), endpointStandalone0.getClientConfigBuilder().build()); + clusterConfigs[0] = new ClusterConfig(endpointStandalone0.getHostAndPort(), + endpointStandalone0.getClientConfigBuilder().build(), null, FailoverOptions.builder().weight(0.5f).build()); + clusterConfigs[1] = new ClusterConfig(endpointStandalone1.getHostAndPort(), + endpointStandalone0.getClientConfigBuilder().build(), null, FailoverOptions.builder().weight(0.3f).build()); - provider = new MultiClusterPooledConnectionProvider(new MultiClusterClientConfig.Builder(clusterConfigs).build()); + provider = new MultiClusterPooledConnectionProvider( + new MultiClusterClientConfig.Builder(clusterConfigs).build()); } @Test @@ -50,44 +55,46 @@ public void testCircuitBreakerForcedTransitions() { @Test public void testIncrementActiveMultiClusterIndex() { - int index = provider.incrementActiveMultiClusterIndex(); - assertEquals(2, index); + Endpoint e2 = provider.iterateActiveCluster(); + assertEquals(endpointStandalone1.getHostAndPort(), e2); } @Test public void testIncrementActiveMultiClusterIndexOutOfRange() { - provider.setActiveMultiClusterIndex(1); + provider.setActiveCluster(endpointStandalone0.getHostAndPort()); - int index = provider.incrementActiveMultiClusterIndex(); - assertEquals(2, index); + Endpoint e2 = provider.iterateActiveCluster(); + assertEquals(endpointStandalone1.getHostAndPort(), e2); - assertThrows(JedisConnectionException.class, - () -> provider.incrementActiveMultiClusterIndex()); // Should throw an exception + assertThrows(JedisConnectionException.class, () -> provider.iterateActiveCluster()); // Should throw an + // exception } @Test public void testIsLastClusterCircuitBreakerForcedOpen() { - provider.setActiveMultiClusterIndex(1); + provider.setActiveCluster(endpointStandalone0.getHostAndPort()); try { - provider.incrementActiveMultiClusterIndex(); - } catch (Exception e) {} + provider.iterateActiveCluster(); + } catch (Exception e) { + } // This should set the isLastClusterCircuitBreakerForcedOpen to true try { - provider.incrementActiveMultiClusterIndex(); - } catch (Exception e) {} + provider.iterateActiveCluster(); + } catch (Exception e) { + } - assertTrue(provider.isLastClusterCircuitBreakerForcedOpen()); + assertFalse(provider.canIterateOnceMore()); } @Test public void testRunClusterFailoverPostProcessor() { ClusterConfig[] clusterConfigs = new ClusterConfig[2]; clusterConfigs[0] = new ClusterConfig(new HostAndPort("purposefully-incorrect", 0000), - DefaultJedisClientConfig.builder().build()); + DefaultJedisClientConfig.builder().build()); clusterConfigs[1] = new ClusterConfig(new HostAndPort("purposefully-incorrect", 0001), - DefaultJedisClientConfig.builder().build()); + DefaultJedisClientConfig.builder().build()); MultiClusterClientConfig.Builder builder = new MultiClusterClientConfig.Builder(clusterConfigs); @@ -98,41 +105,54 @@ public void testRunClusterFailoverPostProcessor() { AtomicBoolean isValidTest = new AtomicBoolean(false); MultiClusterPooledConnectionProvider localProvider = new MultiClusterPooledConnectionProvider(builder.build()); - localProvider.setClusterFailoverPostProcessor(a -> { isValidTest.set(true); }); + localProvider.setClusterFailoverPostProcessor(a -> { + isValidTest.set(true); + }); try (UnifiedJedis jedis = new UnifiedJedis(localProvider)) { // This should fail after 3 retries and meet the requirements to open the circuit on the next iteration try { jedis.get("foo"); - } catch (Exception e) {} + } catch (Exception e) { + } // This should fail after 3 retries and open the circuit which will trigger the post processor try { jedis.get("foo"); - } catch (Exception e) {} + } catch (Exception e) { + } } - assertTrue(isValidTest.get()); + assertTrue(isValidTest.get()); } @Test public void testSetActiveMultiClusterIndexEqualsZero() { - assertThrows(JedisValidationException.class, - () -> provider.setActiveMultiClusterIndex(0)); // Should throw an exception + assertThrows(JedisValidationException.class, () -> provider.setActiveCluster(null)); // Should throw an + // exception } @Test public void testSetActiveMultiClusterIndexLessThanZero() { - assertThrows(JedisValidationException.class, - () -> provider.setActiveMultiClusterIndex(-1)); // Should throw an exception + assertThrows(JedisValidationException.class, () -> provider.setActiveCluster(null)); // Should throw an + // exception } @Test public void testSetActiveMultiClusterIndexOutOfRange() { - assertThrows(JedisValidationException.class, - () -> provider.setActiveMultiClusterIndex(3)); // Should throw an exception + assertThrows(JedisValidationException.class, () -> provider.setActiveCluster(new Endpoint() { + @Override + public String getHost() { + return "purposefully-incorrect"; + } + + @Override + public int getPort() { + return 0000; + } + })); // Should throw an exception } @Test @@ -142,10 +162,12 @@ public void testConnectionPoolConfigApplied() { poolConfig.setMaxIdle(4); poolConfig.setMinIdle(1); ClusterConfig[] clusterConfigs = new ClusterConfig[2]; - clusterConfigs[0] = new ClusterConfig(endpointStandalone0.getHostAndPort(), endpointStandalone0.getClientConfigBuilder().build(), poolConfig); - clusterConfigs[1] = new ClusterConfig(endpointStandalone1.getHostAndPort(), endpointStandalone0.getClientConfigBuilder().build(), poolConfig); + clusterConfigs[0] = new ClusterConfig(endpointStandalone0.getHostAndPort(), + endpointStandalone0.getClientConfigBuilder().build(), poolConfig); + clusterConfigs[1] = new ClusterConfig(endpointStandalone1.getHostAndPort(), + endpointStandalone0.getClientConfigBuilder().build(), poolConfig); try (MultiClusterPooledConnectionProvider customProvider = new MultiClusterPooledConnectionProvider( - new MultiClusterClientConfig.Builder(clusterConfigs).build())) { + new MultiClusterClientConfig.Builder(clusterConfigs).build())) { MultiClusterPooledConnectionProvider.Cluster activeCluster = customProvider.getCluster(); ConnectionPool connectionPool = activeCluster.getConnectionPool(); assertEquals(8, connectionPool.getMaxTotal()); diff --git a/src/test/java/redis/clients/jedis/scenario/ActiveActiveFailoverTest.java b/src/test/java/redis/clients/jedis/scenario/ActiveActiveFailoverTest.java index 7ec4edb14e..f72923d0fc 100644 --- a/src/test/java/redis/clients/jedis/scenario/ActiveActiveFailoverTest.java +++ b/src/test/java/redis/clients/jedis/scenario/ActiveActiveFailoverTest.java @@ -99,7 +99,7 @@ public void accept(String clusterName) { builder.build()); FailoverReporter reporter = new FailoverReporter(); provider.setClusterFailoverPostProcessor(reporter); - provider.setActiveMultiClusterIndex(1); + provider.setActiveCluster(endpoint.getHostAndPort(0)); UnifiedJedis client = new UnifiedJedis(provider); @@ -180,7 +180,7 @@ public void accept(String clusterName) { throw new RuntimeException(e); } - ConnectionPool pool = provider.getCluster(1).getConnectionPool(); + ConnectionPool pool = provider.getCluster(endpoint.getHostAndPort(0)).getConnectionPool(); log.info("First connection pool state: active: {}, idle: {}", pool.getNumActive(), pool.getNumIdle()); From d514ecf45ed636ba0f130e5ef7db8f0de1c05255 Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 9 Jul 2025 12:40:21 +0300 Subject: [PATCH 02/23] - add builder for ClusterConfig - add echo ot CommandObjects and UnifiedJEdis - improve StrategySupplier by accepting jedisclientconfig - adapt EchoStrategy to StrategySupplier. Now it handles the creation of connection by accepting endpoint and JedisClientConfig - make healthchecks disabled by default - drop noOpStrategy - add unit&integration tests for health check --- .../redis/clients/jedis/CommandObjects.java | 4 + .../jedis/MultiClusterClientConfig.java | 32 +- .../redis/clients/jedis/UnifiedJedis.java | 4 + .../redis/clients/jedis/mcf/EchoStrategy.java | 32 +- .../clients/jedis/mcf/FailoverOptions.java | 40 +- .../redis/clients/jedis/mcf/NoOpStrategy.java | 20 - .../MultiClusterPooledConnectionProvider.java | 14 +- .../failover/FailoverIntegrationTest.java | 6 +- .../jedis/mcf/HealthCheckIntegrationTest.java | 102 +++++ .../clients/jedis/mcf/HealthCheckTest.java | 400 ++++++++++++++++++ ...tiClusterPooledConnectionProviderTest.java | 20 +- 11 files changed, 625 insertions(+), 49 deletions(-) delete mode 100644 src/main/java/redis/clients/jedis/mcf/NoOpStrategy.java create mode 100644 src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java create mode 100644 src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java diff --git a/src/main/java/redis/clients/jedis/CommandObjects.java b/src/main/java/redis/clients/jedis/CommandObjects.java index ea4930f894..0bdf5adb7a 100644 --- a/src/main/java/redis/clients/jedis/CommandObjects.java +++ b/src/main/java/redis/clients/jedis/CommandObjects.java @@ -74,6 +74,10 @@ public final CommandObject ping() { return PING_COMMAND_OBJECT; } + public final CommandObject echo(String msg) { + return new CommandObject<>(commandArguments(ECHO).add(msg), BuilderFactory.STRING); + } + private final CommandObject FLUSHALL_COMMAND_OBJECT = new CommandObject<>(commandArguments(FLUSHALL), BuilderFactory.STRING); public final CommandObject flushAll() { diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index b3f7e450ba..3aadbb69ef 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -213,7 +213,7 @@ public ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig, this.connectionPoolConfig = connectionPoolConfig; } - public ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig, + private ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig, GenericObjectPoolConfig connectionPoolConfig, FailoverOptions failoverOptions) { this.hostAndPort = hostAndPort; this.clientConfig = clientConfig; @@ -225,6 +225,10 @@ public HostAndPort getHostAndPort() { return hostAndPort; } + public static Builder builder(HostAndPort hostAndPort, JedisClientConfig clientConfig){ + return new Builder( hostAndPort, clientConfig); + } + public JedisClientConfig getJedisClientConfig() { return clientConfig; } @@ -236,6 +240,32 @@ public GenericObjectPoolConfig getConnectionPoolConfig() { public FailoverOptions getFailoverOptions() { return faileoverOptions; } + + public static class Builder{ + private HostAndPort hostAndPort; + private JedisClientConfig clientConfig; + private GenericObjectPoolConfig connectionPoolConfig; + private FailoverOptions failoverOptions = FailoverOptions.builder().build(); + + public Builder(HostAndPort hostAndPort, JedisClientConfig clientConfig) { + this.hostAndPort = hostAndPort; + this.clientConfig = clientConfig; + } + + public Builder connectionPoolConfig(GenericObjectPoolConfig connectionPoolConfig) { + this.connectionPoolConfig = connectionPoolConfig; + return this; + } + + public Builder failoverOptions(FailoverOptions failoverOptions) { + this.failoverOptions = failoverOptions; + return this; + } + + public ClusterConfig build() { + return new ClusterConfig(hostAndPort, clientConfig, connectionPoolConfig, failoverOptions); + } + } } public static class Builder { diff --git a/src/main/java/redis/clients/jedis/UnifiedJedis.java b/src/main/java/redis/clients/jedis/UnifiedJedis.java index a217cf4adf..7456452ea3 100644 --- a/src/main/java/redis/clients/jedis/UnifiedJedis.java +++ b/src/main/java/redis/clients/jedis/UnifiedJedis.java @@ -342,6 +342,10 @@ public String ping() { return checkAndBroadcastCommand(commandObjects.ping()); } + public String echo(String string) { + return executeCommand(commandObjects.echo(string)); + } + public String flushDB() { return checkAndBroadcastCommand(commandObjects.flushDB()); } diff --git a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java index a778679be2..c9fbecc53e 100644 --- a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java +++ b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java @@ -1,17 +1,31 @@ package redis.clients.jedis.mcf; -import java.util.function.Function; +import redis.clients.jedis.ConnectionFactory; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.UnifiedJedis; +import redis.clients.jedis.exceptions.JedisConnectionException; +import redis.clients.jedis.mcf.FailoverOptions.StrategySupplier; public class EchoStrategy implements HealthCheckStrategy { private int interval; private int timeout; - private Function echo; + private UnifiedJedis jedis; - public EchoStrategy(int healthCheckInterval, int healthCheckTimeout, Function echo) { - this.interval = healthCheckInterval; - this.timeout = healthCheckTimeout; - this.echo = echo; + public EchoStrategy(HostAndPort hostAndPort, JedisClientConfig jedisClientConfig) { + this(hostAndPort, jedisClientConfig, 1000, 1000); + } + + public EchoStrategy(HostAndPort hostAndPort, JedisClientConfig jedisClientConfig, int interval, int timeout) { + this.interval = interval; + this.timeout = timeout; + ConnectionFactory connFactory = new ConnectionFactory(hostAndPort, jedisClientConfig); + try { + this.jedis = new UnifiedJedis(connFactory.makeObject().getObject()); + } catch (Exception e) { + new JedisConnectionException("HealtCheck connection Failed!", e); + } } @Override @@ -26,7 +40,11 @@ public int getTimeout() { @Override public HealthStatus doHealthCheck(Endpoint endpoint) { - return "OK".equals(echo.apply(endpoint)) ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; + return "HealthCheck".equals(jedis.echo("HealthCheck")) ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; } + public static final StrategySupplier DEFAULT = (hostAndPort, jedisClientConfig) -> { + return new EchoStrategy(hostAndPort, jedisClientConfig); + }; + } diff --git a/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java b/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java index 16b369b2ce..a39674d635 100644 --- a/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java +++ b/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java @@ -1,6 +1,7 @@ package redis.clients.jedis.mcf; import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; import redis.clients.jedis.annots.Experimental; /** @@ -13,22 +14,30 @@ public class FailoverOptions { private final float weight; private boolean failback; - private static StrategySupplier defaultStrategySupplier = (endpoint) -> new NoOpStrategy(); - private FailoverOptions(Builder builder) { this.retryOnFailover = builder.retryOnFailover; - this.healthCheckStrategySupplier = builder.healthCheckStrategySupplier == null ? defaultStrategySupplier - : builder.healthCheckStrategySupplier; + if (builder.healthCheckStrategySupplier != null) { + this.healthCheckStrategySupplier = builder.healthCheckStrategySupplier; + } else { + this.healthCheckStrategySupplier = builder.enableHealthCheck ? EchoStrategy.DEFAULT : null; + } this.weight = builder.weight; this.failback = builder.failback; + } public static interface StrategySupplier { - HealthCheckStrategy get(HostAndPort hostAndPort); + /** + * Creates a HealthCheckStrategy for the given endpoint. + * @param hostAndPort the endpoint to create a strategy for + * @param jedisClientConfig the client configuration, may be null for implementations that don't need it + * @return a HealthCheckStrategy instance + */ + HealthCheckStrategy get(HostAndPort hostAndPort, JedisClientConfig jedisClientConfig); } - public HealthCheckStrategy getFailoverHealthCheckStrategy(HostAndPort hostAndPort) { - return healthCheckStrategySupplier.get(hostAndPort); + public StrategySupplier getStrategySupplier() { + return healthCheckStrategySupplier; } public float getWeight() { @@ -63,6 +72,7 @@ public static class Builder { private StrategySupplier healthCheckStrategySupplier; private float weight = 1.0f; private boolean failback; + private boolean enableHealthCheck = false; private Builder() { } @@ -78,6 +88,9 @@ public Builder retryOnFailover(boolean retry) { } public Builder healthCheckStrategySupplier(StrategySupplier healthCheckStrategySupplier) { + if (healthCheckStrategySupplier == null) { + throw new IllegalArgumentException("healthCheckStrategySupplier must not be null"); + } this.healthCheckStrategySupplier = healthCheckStrategySupplier; return this; } @@ -93,7 +106,18 @@ public Builder failback(boolean failbackEnabled) { } public Builder healthCheckStrategy(HealthCheckStrategy healthCheckStrategy) { - this.healthCheckStrategySupplier = (hostAndPort) -> healthCheckStrategy; + if (healthCheckStrategy == null) { + throw new IllegalArgumentException("healthCheckStrategy must not be null"); + } + this.healthCheckStrategySupplier = (hostAndPort, jedisClientConfig) -> healthCheckStrategy; + return this; + } + + public Builder enableHealthCheck(boolean enableHealthCheck) { + this.enableHealthCheck = enableHealthCheck; + if (!enableHealthCheck) { + this.healthCheckStrategySupplier = null; + } return this; } diff --git a/src/main/java/redis/clients/jedis/mcf/NoOpStrategy.java b/src/main/java/redis/clients/jedis/mcf/NoOpStrategy.java deleted file mode 100644 index e2f76cbee5..0000000000 --- a/src/main/java/redis/clients/jedis/mcf/NoOpStrategy.java +++ /dev/null @@ -1,20 +0,0 @@ -package redis.clients.jedis.mcf; - -class NoOpStrategy implements HealthCheckStrategy { - - @Override - public int getInterval() { - return 1000; - } - - @Override - public int getTimeout() { - return 1000; - } - - @Override - public HealthStatus doHealthCheck(Endpoint endpoint) { - return HealthStatus.HEALTHY; - } - -} diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index e0dd55221a..9036e8d8fb 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -32,10 +32,15 @@ import redis.clients.jedis.mcf.HealthStatus; import redis.clients.jedis.mcf.HealthStatusChangeEvent; import redis.clients.jedis.mcf.HealthStatusManager; +import redis.clients.jedis.mcf.FailoverOptions.StrategySupplier; + import redis.clients.jedis.util.Pool; +import redis.clients.jedis.mcf.EchoStrategy; import redis.clients.jedis.mcf.Endpoint; import redis.clients.jedis.mcf.FailoverOptions; +import redis.clients.jedis.mcf.HealthCheckStrategy; + /** * @author Allen Terleto (aterleto) *

@@ -157,12 +162,15 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste Cluster cluster = new Cluster(pool, retry, circuitBreaker, config.getFailoverOptions()); multiClusterMap.put(config.getHostAndPort(), cluster); - healthStatusManager.add(config.getHostAndPort(), - config.getFailoverOptions().getFailoverHealthCheckStrategy(config.getHostAndPort())); + StrategySupplier strategySupplier = config.getFailoverOptions().getStrategySupplier(); + if (strategySupplier != null) { + HealthCheckStrategy hcs = strategySupplier.get(config.getHostAndPort(), config.getJedisClientConfig()); + healthStatusManager.add(config.getHostAndPort(), hcs); + } } // selecting activeCluster with configuration values. - // all health status would be HEALTHY at this point + // all health status would be HEALTHY at this point activeCluster = findWeightedHealthyCluster().getValue(); for (Endpoint endpoint : multiClusterMap.keySet()) { diff --git a/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java b/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java index b7a9b4a4d7..7f0d7e5e02 100644 --- a/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java @@ -19,7 +19,9 @@ import redis.clients.jedis.MultiClusterClientConfig; import redis.clients.jedis.UnifiedJedis; import redis.clients.jedis.exceptions.JedisConnectionException; +import redis.clients.jedis.mcf.EchoStrategy; import redis.clients.jedis.mcf.FailoverOptions; + import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; import redis.clients.jedis.scenario.RecommendedSettings; @@ -182,8 +184,8 @@ private List getClusterConfigs(JedisClie private List getClusterConfigs(JedisClientConfig clientConfig, FailoverOptions failoverOptions, EndpointConfig... endpoints) { - return Arrays.stream(endpoints).map( - e -> new MultiClusterClientConfig.ClusterConfig(e.getHostAndPort(), clientConfig, null, failoverOptions)) + return Arrays.stream(endpoints).map(e -> MultiClusterClientConfig.ClusterConfig + .builder(e.getHostAndPort(), clientConfig).failoverOptions(failoverOptions).build()) .collect(Collectors.toList()); } diff --git a/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java b/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java new file mode 100644 index 0000000000..bb8230e93c --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java @@ -0,0 +1,102 @@ +package redis.clients.jedis.mcf; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.jupiter.api.Test; + +import io.github.resilience4j.circuitbreaker.CircuitBreakerConfig.SlidingWindowType; +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.EndpointConfig; +import redis.clients.jedis.HostAndPorts; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.MultiClusterClientConfig; +import redis.clients.jedis.UnifiedJedis; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; +import redis.clients.jedis.scenario.RecommendedSettings; + +public class HealthCheckIntegrationTest { + + private final EndpointConfig endpoint1 = HostAndPorts.getRedisEndpoint("standalone0"); + private final EndpointConfig endpoint2 = HostAndPorts.getRedisEndpoint("standalone1"); + + private final JedisClientConfig clientConfig = endpoint1.getClientConfigBuilder() + .socketTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS) + .connectionTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS).build(); + + @Test + public void testDisableHealthCheck() { + FailoverOptions fo1 = FailoverOptions.builder().build(); + + MultiClusterPooledConnectionProvider customProvider = getMCCF(fo1); + try (UnifiedJedis customClient = new UnifiedJedis(customProvider)) { + // Verify that the client can connect and execute commands + String result = customClient.ping(); + assertEquals("PONG", result); + } + } + + @Test + public void testDefaultStrategySupplier() { + FailoverOptions fo1 = FailoverOptions.builder().enableHealthCheck(true).build(); + + MultiClusterPooledConnectionProvider customProvider = getMCCF(fo1); + try (UnifiedJedis customClient = new UnifiedJedis(customProvider)) { + // Verify that the client can connect and execute commands + String result = customClient.ping(); + assertEquals("PONG", result); + } + } + + @Test + public void testCustomStrategySupplier() { + // Create a StrategySupplier that uses the JedisClientConfig when available + FailoverOptions.StrategySupplier strategySupplier = (hostAndPort, jedisClientConfig) -> { + UnifiedJedis pinger = new UnifiedJedis(hostAndPort, jedisClientConfig); + return new HealthCheckStrategy() { + + @Override + public int getInterval() { + return 500; + } + + @Override + public int getTimeout() { + return 500; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + return "OK".equals(pinger.ping()) ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; + } + + }; + }; + + FailoverOptions fo1 = FailoverOptions.builder().healthCheckStrategySupplier(strategySupplier).build(); + + MultiClusterPooledConnectionProvider customProvider = getMCCF(fo1); + try (UnifiedJedis customClient = new UnifiedJedis(customProvider)) { + // Verify that the client can connect and execute commands + String result = customClient.ping(); + assertEquals("PONG", result); + } + } + + private MultiClusterPooledConnectionProvider getMCCF(FailoverOptions fo) { + List clusterConfigs = Arrays.stream(new EndpointConfig[] { endpoint1 }) + .map(e -> MultiClusterClientConfig.ClusterConfig.builder(e.getHostAndPort(), clientConfig) + .failoverOptions(fo).build()) + .collect(Collectors.toList()); + + MultiClusterClientConfig mccf = new MultiClusterClientConfig.Builder(clusterConfigs).retryMaxAttempts(1) + .retryWaitDuration(1).circuitBreakerSlidingWindowType(SlidingWindowType.COUNT_BASED) + .circuitBreakerSlidingWindowSize(1).circuitBreakerFailureRateThreshold(100) + .circuitBreakerSlidingWindowMinCalls(1).build(); + + return new MultiClusterPooledConnectionProvider(mccf); + } +} diff --git a/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java b/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java new file mode 100644 index 0000000000..686f9bfb9c --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java @@ -0,0 +1,400 @@ +package redis.clients.jedis.mcf; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.UnifiedJedis; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Mockito.*; + +public class HealthCheckTest { + + @Mock + private UnifiedJedis mockJedis; + + @Mock + private HealthCheckStrategy mockStrategy; + + private HealthCheckStrategy alwaysHealthyStrategy = new HealthCheckStrategy() { + @Override + public int getInterval() { + return 100; + } + + @Override + public int getTimeout() { + return 50; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + return HealthStatus.HEALTHY; + } + }; + + @Mock + private Consumer mockCallback; + + private HostAndPort testEndpoint; + private JedisClientConfig testConfig; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + testEndpoint = new HostAndPort("localhost", 6379); + testConfig = DefaultJedisClientConfig.builder().build(); + } + + // ========== HealthStatus Tests ========== + + @Test + void testHealthStatusIsHealthy() { + assertTrue(HealthStatus.HEALTHY.isHealthy()); + assertFalse(HealthStatus.UNHEALTHY.isHealthy()); + assertFalse(HealthStatus.FORCED_UNHEALTHY.isHealthy()); + } + + @Test + void testHealthStatusIsForcedUnhealthy() { + assertFalse(HealthStatus.HEALTHY.isForcedUnhealthy()); + assertFalse(HealthStatus.UNHEALTHY.isForcedUnhealthy()); + assertTrue(HealthStatus.FORCED_UNHEALTHY.isForcedUnhealthy()); + } + + // ========== HostAndPort Tests ========== + @Test + void testHostAndPortEquality() { + HostAndPort endpoint1 = new HostAndPort("localhost", 6379); + HostAndPort endpoint2 = new HostAndPort("localhost", 6379); + HostAndPort endpoint3 = new HostAndPort("localhost", 6380); + + assertEquals(endpoint1, endpoint2); + assertNotEquals(endpoint1, endpoint3); + assertEquals(endpoint1.hashCode(), endpoint2.hashCode()); + } + + // ========== HealthCheckCollection Tests ========== + + void testHealthCheckCollectionAdd() { + HealthCheckCollection collection = new HealthCheckCollection(); + HealthCheck healthCheck = new HealthCheck(testEndpoint, mockStrategy, mockCallback); + + HealthCheck previous = collection.add(healthCheck); + assertNull(previous); + + assertEquals(healthCheck, collection.get(testEndpoint)); + } + + @Test + void testHealthCheckCollectionRemoveByEndpoint() { + HealthCheckCollection collection = new HealthCheckCollection(); + HealthCheck healthCheck = new HealthCheck(testEndpoint, mockStrategy, mockCallback); + + collection.add(healthCheck); + HealthCheck removed = collection.remove(testEndpoint); + + assertEquals(healthCheck, removed); + assertNull(collection.get(testEndpoint)); + } + + @Test + void testHealthCheckCollectionAddAll() { + HealthCheckCollection collection = new HealthCheckCollection(); + HealthCheck[] healthChecks = { new HealthCheck(new HostAndPort("host1", 6379), mockStrategy, mockCallback), + new HealthCheck(new HostAndPort("host2", 6379), mockStrategy, mockCallback) }; + + HealthCheck[] previous = collection.addAll(healthChecks); + + assertNotNull(previous); + assertEquals(2, previous.length); + assertNull(previous[0]); // No previous health check for host1 + assertNull(previous[1]); // No previous health check for host2 + + assertEquals(healthChecks[0], collection.get(new HostAndPort("host1", 6379))); + assertEquals(healthChecks[1], collection.get(new HostAndPort("host2", 6379))); + } + + @Test + void testHealthCheckCollectionReplacement() { + HealthCheckCollection collection = new HealthCheckCollection(); + HealthCheck healthCheck1 = new HealthCheck(testEndpoint, mockStrategy, mockCallback); + HealthCheck healthCheck2 = new HealthCheck(testEndpoint, mockStrategy, mockCallback); + + collection.add(healthCheck1); + HealthCheck previous = collection.add(healthCheck2); + + assertEquals(healthCheck1, previous); + assertEquals(healthCheck2, collection.get(testEndpoint)); + } + + @Test + void testHealthCheckCollectionRemoveByHealthCheck() { + HealthCheckCollection collection = new HealthCheckCollection(); + HealthCheck healthCheck = new HealthCheck(testEndpoint, mockStrategy, mockCallback); + + collection.add(healthCheck); + HealthCheck removed = collection.remove(healthCheck); + + assertEquals(healthCheck, removed); + assertNull(collection.get(testEndpoint)); + } + + // ========== HealthCheck Tests ========== + + @Test + void testHealthCheckStatusUpdate() throws InterruptedException { + when(mockStrategy.getInterval()).thenReturn(100); + when(mockStrategy.getTimeout()).thenReturn(50); + when(mockStrategy.doHealthCheck(any(Endpoint.class))).thenReturn(HealthStatus.UNHEALTHY); + + CountDownLatch latch = new CountDownLatch(1); + Consumer callback = event -> { + assertEquals(HealthStatus.HEALTHY, event.getOldStatus()); + assertEquals(HealthStatus.UNHEALTHY, event.getNewStatus()); + latch.countDown(); + }; + + HealthCheck healthCheck = new HealthCheck(testEndpoint, mockStrategy, callback); + healthCheck.start(); + + assertTrue(latch.await(2, TimeUnit.SECONDS)); + healthCheck.stop(); + } + + @Test + void testHealthCheckStop() { + when(mockStrategy.getInterval()).thenReturn(1000); + when(mockStrategy.getTimeout()).thenReturn(500); + + HealthCheck healthCheck = new HealthCheck(testEndpoint, mockStrategy, mockCallback); + healthCheck.start(); + + assertDoesNotThrow(() -> healthCheck.stop()); + } + + // ========== HealthStatusManager Tests ========== + + @Test + void testHealthStatusManagerRegisterListener() { + HealthStatusManager manager = new HealthStatusManager(); + HealthStatusListener listener = mock(HealthStatusListener.class); + + manager.registerListener(listener); + + // Verify listener is registered by triggering an event + manager.add(testEndpoint, alwaysHealthyStrategy); + // Give some time for health check to run + try { + Thread.sleep(100); + } catch (InterruptedException e) { + } + + verify(listener, atLeastOnce()).onStatusChange(any(HealthStatusChangeEvent.class)); + } + + @Test + void testHealthStatusManagerUnregisterListener() { + HealthStatusManager manager = new HealthStatusManager(); + HealthStatusListener listener = mock(HealthStatusListener.class); + + manager.registerListener(listener); + manager.unregisterListener(listener); + + manager.add(testEndpoint, alwaysHealthyStrategy); + + // Give some time for potential health check + try { + Thread.sleep(100); + } catch (InterruptedException e) { + } + + verify(listener, never()).onStatusChange(any(HealthStatusChangeEvent.class)); + } + + @Test + void testHealthStatusManagerEndpointSpecificListener() { + HealthStatusManager manager = new HealthStatusManager(); + HealthStatusListener listener = mock(HealthStatusListener.class); + HostAndPort otherEndpoint = new HostAndPort("other", 6379); + + manager.registerListener(testEndpoint, listener); + manager.add(testEndpoint, alwaysHealthyStrategy); + manager.add(otherEndpoint, alwaysHealthyStrategy); + + // Give some time for health checks + try { + Thread.sleep(100); + } catch (InterruptedException e) { + } + + // Listener should only receive events for testEndpoint + verify(listener, atLeastOnce()).onStatusChange(argThat(event -> event.getEndpoint().equals(testEndpoint))); + } + + @Test + void testHealthStatusManagerLifecycle() { + HealthStatusManager manager = new HealthStatusManager(); + + // Before adding health check + assertEquals(HealthStatus.UNHEALTHY, manager.getHealthStatus(testEndpoint)); + + manager.add(testEndpoint, alwaysHealthyStrategy); + assertEquals(HealthStatus.HEALTHY, manager.getHealthStatus(testEndpoint)); + + manager.remove(testEndpoint); + assertEquals(HealthStatus.UNHEALTHY, manager.getHealthStatus(testEndpoint)); + } + + // ========== EchoStrategy Tests ========== + + @Test + void testEchoStrategyCustomIntervalTimeout() { + EchoStrategy strategy = new EchoStrategy(testEndpoint, testConfig, 2000, 1500); + + assertEquals(2000, strategy.getInterval()); + assertEquals(1500, strategy.getTimeout()); + } + + @Test + void testEchoStrategyDefaultSupplier() { + FailoverOptions.StrategySupplier supplier = EchoStrategy.DEFAULT; + HealthCheckStrategy strategy = supplier.get(testEndpoint, testConfig); + + assertInstanceOf(EchoStrategy.class, strategy); + } + + // ========== FailoverOptions Tests ========== + + @Test + void testFailoverOptionsBuilder() { + FailoverOptions options = FailoverOptions.builder().retryOnFailover(true).weight(2.5f).failback(true).build(); + + assertTrue(options.isRetryOnFailover()); + assertEquals(2.5f, options.getWeight()); + assertTrue(options.isFailbackEnabled()); + } + + @Test + void testFailoverOptionsDefaultValues() { + FailoverOptions options = FailoverOptions.builder().build(); + + assertFalse(options.isRetryOnFailover()); // Default is false + assertEquals(1.0f, options.getWeight()); // Default weight + assertFalse(options.isFailbackEnabled()); // Default is false + } + + @Test + void testFailoverOptionsWithHealthCheckStrategy() { + HealthCheckStrategy customStrategy = mock(HealthCheckStrategy.class); + + FailoverOptions options = FailoverOptions.builder().healthCheckStrategy(customStrategy).build(); + + FailoverOptions.StrategySupplier supplier = options.getStrategySupplier(); + assertNotNull(supplier); + + HealthCheckStrategy result = supplier.get(testEndpoint, testConfig); + assertEquals(customStrategy, result); + } + + @Test + void testFailoverOptionsWithStrategySupplier() { + FailoverOptions.StrategySupplier customSupplier = (hostAndPort, jedisClientConfig) -> { + return mock(HealthCheckStrategy.class); + }; + + FailoverOptions options = FailoverOptions.builder().healthCheckStrategySupplier(customSupplier).build(); + + assertEquals(customSupplier, options.getStrategySupplier()); + } + + @Test + void testFailoverOptionsWithEnabledHealthCheck() { + FailoverOptions options = FailoverOptions.builder().enableHealthCheck(true).build(); + + FailoverOptions.StrategySupplier supplier = options.getStrategySupplier(); + assertNotNull(supplier); + assertEquals(EchoStrategy.DEFAULT, supplier); + } + + @Test + void testFailoverOptionsWithDisabledHealthCheck() { + FailoverOptions options = FailoverOptions.builder().enableHealthCheck(false).build(); + + assertNull(options.getStrategySupplier()); + } + + // ========== Integration Tests ========== + + @Test + @Timeout(5) + void testHealthCheckIntegration() throws InterruptedException { + // Create a mock strategy that alternates between healthy and unhealthy + AtomicReference statusToReturn = new AtomicReference<>(HealthStatus.HEALTHY); + HealthCheckStrategy alternatingStrategy = new HealthCheckStrategy() { + @Override + public int getInterval() { + return 100; + } + + @Override + public int getTimeout() { + return 50; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + HealthStatus current = statusToReturn.get(); + statusToReturn.set(current == HealthStatus.HEALTHY ? HealthStatus.UNHEALTHY : HealthStatus.HEALTHY); + return current; + } + }; + + CountDownLatch statusChangeLatch = new CountDownLatch(2); // Wait for 2 status changes + HealthStatusListener listener = event -> statusChangeLatch.countDown(); + + HealthStatusManager manager = new HealthStatusManager(); + manager.registerListener(listener); + manager.add(testEndpoint, alternatingStrategy); + + assertTrue(statusChangeLatch.await(3, TimeUnit.SECONDS)); + + manager.remove(testEndpoint); + } + + @Test + void testFailoverOptionsStrategySupplierPolymorphism() { + // Test that the polymorphic design works correctly + FailoverOptions.StrategySupplier supplier = (hostAndPort, jedisClientConfig) -> { + if (jedisClientConfig != null) { + return new EchoStrategy(hostAndPort, jedisClientConfig, 500, 250); + } else { + return new EchoStrategy(hostAndPort, DefaultJedisClientConfig.builder().build()); + } + }; + + // Test with config + HealthCheckStrategy strategyWithConfig = supplier.get(testEndpoint, testConfig); + assertNotNull(strategyWithConfig); + assertEquals(500, strategyWithConfig.getInterval()); + assertEquals(250, strategyWithConfig.getTimeout()); + + // Test without config + HealthCheckStrategy strategyWithoutConfig = supplier.get(testEndpoint, null); + assertNotNull(strategyWithoutConfig); + assertEquals(1000, strategyWithoutConfig.getInterval()); // Default values + assertEquals(1000, strategyWithoutConfig.getTimeout()); + } +} diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java index 9e84d5d01f..78ba57a3eb 100644 --- a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java @@ -28,10 +28,12 @@ public class MultiClusterPooledConnectionProviderTest { public void setUp() { ClusterConfig[] clusterConfigs = new ClusterConfig[2]; - clusterConfigs[0] = new ClusterConfig(endpointStandalone0.getHostAndPort(), - endpointStandalone0.getClientConfigBuilder().build(), null, FailoverOptions.builder().weight(0.5f).build()); - clusterConfigs[1] = new ClusterConfig(endpointStandalone1.getHostAndPort(), - endpointStandalone0.getClientConfigBuilder().build(), null, FailoverOptions.builder().weight(0.3f).build()); + clusterConfigs[0] = ClusterConfig + .builder(endpointStandalone0.getHostAndPort(), endpointStandalone0.getClientConfigBuilder().build()) + .failoverOptions(FailoverOptions.builder().weight(0.5f).build()).build(); + clusterConfigs[1] = ClusterConfig + .builder(endpointStandalone1.getHostAndPort(), endpointStandalone1.getClientConfigBuilder().build()) + .failoverOptions(FailoverOptions.builder().weight(0.3f).build()).build(); provider = new MultiClusterPooledConnectionProvider( new MultiClusterClientConfig.Builder(clusterConfigs).build()); @@ -91,10 +93,12 @@ public void testIsLastClusterCircuitBreakerForcedOpen() { @Test public void testRunClusterFailoverPostProcessor() { ClusterConfig[] clusterConfigs = new ClusterConfig[2]; - clusterConfigs[0] = new ClusterConfig(new HostAndPort("purposefully-incorrect", 0000), - DefaultJedisClientConfig.builder().build()); - clusterConfigs[1] = new ClusterConfig(new HostAndPort("purposefully-incorrect", 0001), - DefaultJedisClientConfig.builder().build()); + clusterConfigs[0] = ClusterConfig + .builder(new HostAndPort("purposefully-incorrect", 0000), DefaultJedisClientConfig.builder().build()) + .failoverOptions(FailoverOptions.builder().weight(0.5f).build()).build(); + clusterConfigs[1] = ClusterConfig + .builder(new HostAndPort("purposefully-incorrect", 0001), DefaultJedisClientConfig.builder().build()) + .failoverOptions(FailoverOptions.builder().weight(0.4f).build()).build(); MultiClusterClientConfig.Builder builder = new MultiClusterClientConfig.Builder(clusterConfigs); From df66b1eb0ee930d65909ac072c4a98b6f3871a7b Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 9 Jul 2025 12:42:59 +0300 Subject: [PATCH 03/23] - fix naming --- .../java/redis/clients/jedis/MultiClusterClientConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 3aadbb69ef..368c2057b1 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -199,7 +199,7 @@ public static class ClusterConfig { private HostAndPort hostAndPort; private JedisClientConfig clientConfig; private GenericObjectPoolConfig connectionPoolConfig; - private FailoverOptions faileoverOptions = FailoverOptions.builder().build(); + private FailoverOptions failoverOptions = FailoverOptions.builder().build(); public ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig) { this.hostAndPort = hostAndPort; @@ -218,7 +218,7 @@ private ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig, this.hostAndPort = hostAndPort; this.clientConfig = clientConfig; this.connectionPoolConfig = connectionPoolConfig; - this.faileoverOptions = failoverOptions; + this.failoverOptions = failoverOptions; } public HostAndPort getHostAndPort() { @@ -238,7 +238,7 @@ public GenericObjectPoolConfig getConnectionPoolConfig() { } public FailoverOptions getFailoverOptions() { - return faileoverOptions; + return failoverOptions; } public static class Builder{ From 13757f51a5b2d5af40d8777251cef7c7aad0de17 Mon Sep 17 00:00:00 2001 From: atakavci Date: Thu, 10 Jul 2025 09:46:30 +0300 Subject: [PATCH 04/23] clean up and mark override methods --- .../java/redis/clients/jedis/HostAndPort.java | 2 + .../redis/clients/jedis/UnifiedJedis.java | 1 - .../MultiClusterPooledConnectionProvider.java | 41 ------------------- 3 files changed, 2 insertions(+), 42 deletions(-) diff --git a/src/main/java/redis/clients/jedis/HostAndPort.java b/src/main/java/redis/clients/jedis/HostAndPort.java index 9548071976..fbc14655e6 100644 --- a/src/main/java/redis/clients/jedis/HostAndPort.java +++ b/src/main/java/redis/clients/jedis/HostAndPort.java @@ -16,10 +16,12 @@ public HostAndPort(String host, int port) { this.port = port; } + @Override public String getHost() { return host; } + @Override public int getPort() { return port; } diff --git a/src/main/java/redis/clients/jedis/UnifiedJedis.java b/src/main/java/redis/clients/jedis/UnifiedJedis.java index 7456452ea3..73142909b9 100644 --- a/src/main/java/redis/clients/jedis/UnifiedJedis.java +++ b/src/main/java/redis/clients/jedis/UnifiedJedis.java @@ -30,7 +30,6 @@ import redis.clients.jedis.json.Path2; import redis.clients.jedis.json.JsonObjectMapper; import redis.clients.jedis.mcf.CircuitBreakerCommandExecutor; -import redis.clients.jedis.mcf.FailoverOptions; import redis.clients.jedis.mcf.MultiClusterPipeline; import redis.clients.jedis.mcf.MultiClusterTransaction; import redis.clients.jedis.params.*; diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 9036e8d8fb..5eaf5f8820 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -35,7 +35,6 @@ import redis.clients.jedis.mcf.FailoverOptions.StrategySupplier; import redis.clients.jedis.util.Pool; -import redis.clients.jedis.mcf.EchoStrategy; import redis.clients.jedis.mcf.Endpoint; import redis.clients.jedis.mcf.FailoverOptions; @@ -272,46 +271,6 @@ private void validateTargetConnection(Cluster cluster) { } } - /** - * Manually overrides the actively used cluster/database endpoint (connection pool) amongst the pre-configured list - * which were provided at startup via the MultiClusterClientConfig. All traffic will be routed according to the - * provided new index. Special care should be taken to confirm cluster/database availability AND potentially - * cross-cluster replication BEFORE using this capability. - */ - // public void setActiveMultiClusterIndex(int multiClusterIndex) { - - // // Field-level synchronization is used to avoid the edge case in which - // // incrementActiveMultiClusterIndex() is called at the same time - // activeClusterIndexLock.lock(); - - // try { - - // // Allows an attempt to reset the current cluster from a FORCED_OPEN to CLOSED state in the event that no - // // failover is possible - // if (activeMultiClusterIndex == multiClusterIndex - // && !CircuitBreaker.State.FORCED_OPEN.equals(getClusterCircuitBreaker(multiClusterIndex).getState())) - // return; - - // if (multiClusterIndex < 1 || multiClusterIndex > multiClusterMap.size()) - // throw new JedisValidationException("MultiClusterIndex: " + multiClusterIndex + " is not within " - // + "the configured range. Please choose an index between 1 and " + multiClusterMap.size()); - - // validateTargetConnection(multiClusterIndex); - - // String originalClusterName = getClusterCircuitBreaker().getName(); - - // if (activeMultiClusterIndex == multiClusterIndex) - // log.warn("Cluster/database endpoint '{}' successfully closed its circuit breaker", originalClusterName); - // else log.warn("Cluster/database endpoint successfully updated from '{}' to '{}'", originalClusterName, - // getClusterCircuitBreaker(multiClusterIndex).getName()); - - // activeMultiClusterIndex = multiClusterIndex; - // // lastClusterCircuitBreakerForcedOpen = false; - // } finally { - // activeClusterIndexLock.unlock(); - // } - // } - public void setActiveCluster(Endpoint endpoint) { if (endpoint == null) { throw new JedisValidationException("Provided endpoint is null. Please use one from the configuration"); From ef5d83ab9cfbc55c62a47a1feae759349ee7c9a6 Mon Sep 17 00:00:00 2001 From: atakavci Date: Thu, 10 Jul 2025 15:15:05 +0300 Subject: [PATCH 05/23] fix link in javadoc --- .../jedis/providers/MultiClusterPooledConnectionProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 5eaf5f8820..df9ab8d01f 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -48,7 +48,7 @@ * Active-Active cluster(s) by using simple configuration which is passed through from Resilience4j - * https://resilience4j.readme.io/docs *

- * Support for manual failback is provided by way of {@link #setActiveMultiClusterIndex(int)} + * Support for manual failback is provided by way of {@link #setActiveCluster(Endpoint)} *

*/ // TODO: move? From a15fc64f434fd082ae4bf488a5cf984fc8aa29c3 Mon Sep 17 00:00:00 2001 From: atakavci Date: Thu, 10 Jul 2025 15:20:13 +0300 Subject: [PATCH 06/23] fix formatting --- .../redis/clients/jedis/MultiClusterClientConfig.java | 9 ++++----- .../clients/jedis/mcf/CircuitBreakerFailoverBase.java | 7 +++---- src/main/java/redis/clients/jedis/mcf/EchoStrategy.java | 2 +- .../redis/clients/jedis/mcf/HealthStatusChangeEvent.java | 2 -- 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 368c2057b1..9938e2b17a 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -225,8 +225,8 @@ public HostAndPort getHostAndPort() { return hostAndPort; } - public static Builder builder(HostAndPort hostAndPort, JedisClientConfig clientConfig){ - return new Builder( hostAndPort, clientConfig); + public static Builder builder(HostAndPort hostAndPort, JedisClientConfig clientConfig) { + return new Builder(hostAndPort, clientConfig); } public JedisClientConfig getJedisClientConfig() { @@ -241,7 +241,7 @@ public FailoverOptions getFailoverOptions() { return failoverOptions; } - public static class Builder{ + public static class Builder { private HostAndPort hostAndPort; private JedisClientConfig clientConfig; private GenericObjectPoolConfig connectionPoolConfig; @@ -293,8 +293,7 @@ public Builder(ClusterConfig[] clusterConfigs) { if (clusterConfigs == null || clusterConfigs.length < 1) throw new JedisValidationException( "ClusterClientConfigs are required for MultiClusterPooledConnectionProvider"); - - this.clusterConfigs = clusterConfigs; + this.clusterConfigs = clusterConfigs; } public Builder(List clusterConfigs) { diff --git a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java index 2dae7b5443..9b47759587 100644 --- a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java +++ b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java @@ -57,11 +57,10 @@ protected void clusterFailover(CircuitBreaker circuitBreaker) { // activeMultiClusterIndex persistence or custom logging provider.runClusterFailoverPostProcessor(provider.getCluster()); } - // this check relies on the fact that many failover attempts can hit with the same CB, + // this check relies on the fact that many failover attempts can hit with the same CB, // only the first one will trigger a failover, and make the CB FORCED_OPEN. - // when the rest reaches here, the active cluster is already the next one, and should be different than active CB. - // if its the same one and there are no more clusters to failover to, - // then throw an exception + // when the rest reaches here, the active cluster is already the next one, and should be different than + // active CB. If its the same one and there are no more clusters to failover to, then throw an exception else if (circuitBreaker == provider.getCluster().getCircuitBreaker() && !provider.canIterateOnceMore()) { throw new JedisConnectionException( "Cluster/database endpoint could not failover since the MultiClusterClientConfig was not " diff --git a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java index c9fbecc53e..9680d452b1 100644 --- a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java +++ b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java @@ -14,7 +14,7 @@ public class EchoStrategy implements HealthCheckStrategy { private UnifiedJedis jedis; public EchoStrategy(HostAndPort hostAndPort, JedisClientConfig jedisClientConfig) { - this(hostAndPort, jedisClientConfig, 1000, 1000); + this(hostAndPort, jedisClientConfig, 1000, 1000); } public EchoStrategy(HostAndPort hostAndPort, JedisClientConfig jedisClientConfig, int interval, int timeout) { diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatusChangeEvent.java b/src/main/java/redis/clients/jedis/mcf/HealthStatusChangeEvent.java index 20e1a1fe65..aefa2e06cc 100644 --- a/src/main/java/redis/clients/jedis/mcf/HealthStatusChangeEvent.java +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatusChangeEvent.java @@ -1,7 +1,5 @@ - package redis.clients.jedis.mcf; - public class HealthStatusChangeEvent { private final Endpoint endpoint; From cf38240e396a5882013e1cffa3b00d5339499a89 Mon Sep 17 00:00:00 2001 From: atakavci Date: Mon, 14 Jul 2025 17:32:50 +0300 Subject: [PATCH 07/23] - fix double registered listeners in healtstatusmgr - clear redundant catch - replace failover options and drop failoveroptions class - remove forced_unhealthy from healthstatus - fix failback check - add disabled flag to cluster - update/fix related tests --- .../jedis/MultiClusterClientConfig.java | 109 +++++++++++++-- .../jedis/mcf/CircuitBreakerFailoverBase.java | 2 - .../redis/clients/jedis/mcf/EchoStrategy.java | 2 +- .../clients/jedis/mcf/FailoverOptions.java | 132 ------------------ .../redis/clients/jedis/mcf/HealthStatus.java | 6 +- .../jedis/mcf/HealthStatusManager.java | 2 - .../MultiClusterPooledConnectionProvider.java | 63 +++++---- .../failover/FailoverIntegrationTest.java | 32 ++--- .../jedis/mcf/HealthCheckIntegrationTest.java | 43 +++--- .../clients/jedis/mcf/HealthCheckTest.java | 126 +++++++++++------ ...tiClusterPooledConnectionProviderTest.java | 18 +-- 11 files changed, 266 insertions(+), 269 deletions(-) delete mode 100644 src/main/java/redis/clients/jedis/mcf/FailoverOptions.java diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 9938e2b17a..2656c36631 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -12,7 +12,8 @@ import redis.clients.jedis.exceptions.JedisConnectionException; import redis.clients.jedis.exceptions.JedisValidationException; import redis.clients.jedis.mcf.ConnectionFailoverException; -import redis.clients.jedis.mcf.FailoverOptions; +import redis.clients.jedis.mcf.EchoStrategy; +import redis.clients.jedis.mcf.HealthCheckStrategy; /** * @author Allen Terleto (aterleto) @@ -32,6 +33,19 @@ @Experimental public final class MultiClusterClientConfig { + /** + * Interface for creating HealthCheckStrategy instances for specific endpoints + */ + public static interface StrategySupplier { + /** + * Creates a HealthCheckStrategy for the given endpoint. + * @param hostAndPort the endpoint to create a strategy for + * @param jedisClientConfig the client configuration, may be null for implementations that don't need it + * @return a HealthCheckStrategy instance + */ + HealthCheckStrategy get(HostAndPort hostAndPort, JedisClientConfig jedisClientConfig); + } + private static final int RETRY_MAX_ATTEMPTS_DEFAULT = 3; private static final int RETRY_WAIT_DURATION_DEFAULT = 500; // measured in milliseconds private static final int RETRY_WAIT_DURATION_EXPONENTIAL_BACKOFF_MULTIPLIER_DEFAULT = 2; @@ -130,6 +144,14 @@ public final class MultiClusterClientConfig { private List> fallbackExceptionList; + //////////// Failover Config //////////// + + /** Whether to retry failed commands during failover */ + private boolean retryOnFailover = false; + + /** Whether failback is enabled */ + private boolean failback = false; + public MultiClusterClientConfig(ClusterConfig[] clusterConfigs) { this.clusterConfigs = clusterConfigs; } @@ -194,12 +216,22 @@ public List> getFallbackExceptionList() { return fallbackExceptionList; } + public boolean isRetryOnFailover() { + return retryOnFailover; + } + + public boolean isFailback() { + return failback; + } + public static class ClusterConfig { private HostAndPort hostAndPort; private JedisClientConfig clientConfig; private GenericObjectPoolConfig connectionPoolConfig; - private FailoverOptions failoverOptions = FailoverOptions.builder().build(); + + private float weight = 1.0f; + private StrategySupplier healthCheckStrategySupplier; public ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig) { this.hostAndPort = hostAndPort; @@ -213,12 +245,12 @@ public ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig, this.connectionPoolConfig = connectionPoolConfig; } - private ClusterConfig(HostAndPort hostAndPort, JedisClientConfig clientConfig, - GenericObjectPoolConfig connectionPoolConfig, FailoverOptions failoverOptions) { - this.hostAndPort = hostAndPort; - this.clientConfig = clientConfig; - this.connectionPoolConfig = connectionPoolConfig; - this.failoverOptions = failoverOptions; + private ClusterConfig(Builder builder) { + this.hostAndPort = builder.hostAndPort; + this.clientConfig = builder.clientConfig; + this.connectionPoolConfig = builder.connectionPoolConfig; + this.weight = builder.weight; + this.healthCheckStrategySupplier = builder.healthCheckStrategySupplier; } public HostAndPort getHostAndPort() { @@ -237,15 +269,22 @@ public GenericObjectPoolConfig getConnectionPoolConfig() { return connectionPoolConfig; } - public FailoverOptions getFailoverOptions() { - return failoverOptions; + public float getWeight() { + return weight; + } + + public StrategySupplier getHealthCheckStrategySupplier() { + return healthCheckStrategySupplier; } public static class Builder { private HostAndPort hostAndPort; private JedisClientConfig clientConfig; private GenericObjectPoolConfig connectionPoolConfig; - private FailoverOptions failoverOptions = FailoverOptions.builder().build(); + + private float weight = 1.0f; + private StrategySupplier healthCheckStrategySupplier = EchoStrategy.DEFAULT; + private boolean healthCheckEnabled = true; public Builder(HostAndPort hostAndPort, JedisClientConfig clientConfig) { this.hostAndPort = hostAndPort; @@ -257,13 +296,39 @@ public Builder connectionPoolConfig(GenericObjectPoolConfig connecti return this; } - public Builder failoverOptions(FailoverOptions failoverOptions) { - this.failoverOptions = failoverOptions; + public Builder weight(float weight) { + this.weight = weight; + return this; + } + + public Builder healthCheckStrategySupplier(StrategySupplier healthCheckStrategySupplier) { + if (healthCheckStrategySupplier == null) { + throw new IllegalArgumentException("healthCheckStrategySupplier must not be null"); + } + this.healthCheckStrategySupplier = healthCheckStrategySupplier; + return this; + } + + public Builder healthCheckStrategy(HealthCheckStrategy healthCheckStrategy) { + if (healthCheckStrategy == null) { + throw new IllegalArgumentException("healthCheckStrategy must not be null"); + } + this.healthCheckStrategySupplier = (hostAndPort, jedisClientConfig) -> healthCheckStrategy; + return this; + } + + public Builder healthCheckEnabled(boolean healthCheckEnabled) { + this.healthCheckEnabled = healthCheckEnabled; + if (!healthCheckEnabled) { + this.healthCheckStrategySupplier = null; + } else if (healthCheckStrategySupplier == null) { + this.healthCheckStrategySupplier = EchoStrategy.DEFAULT; + } return this; } public ClusterConfig build() { - return new ClusterConfig(hostAndPort, clientConfig, connectionPoolConfig, failoverOptions); + return new ClusterConfig(this); } } } @@ -288,6 +353,9 @@ public static class Builder { private List circuitBreakerIgnoreExceptionList = null; private List> fallbackExceptionList = FALLBACK_EXCEPTIONS_DEFAULT; + private boolean retryOnFailover = false; + private boolean failback = false; + public Builder(ClusterConfig[] clusterConfigs) { if (clusterConfigs == null || clusterConfigs.length < 1) throw new JedisValidationException( @@ -380,6 +448,16 @@ public Builder fallbackExceptionList(List> fallbackEx return this; } + public Builder retryOnFailover(boolean retryOnFailover) { + this.retryOnFailover = retryOnFailover; + return this; + } + + public Builder failback(boolean failback) { + this.failback = failback; + return this; + } + public MultiClusterClientConfig build() { MultiClusterClientConfig config = new MultiClusterClientConfig(this.clusterConfigs); @@ -405,6 +483,9 @@ public MultiClusterClientConfig build() { config.fallbackExceptionList = this.fallbackExceptionList; + config.retryOnFailover = this.retryOnFailover; + config.failback = this.failback; + return config; } } diff --git a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java index 9b47759587..f4826a53d6 100644 --- a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java +++ b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java @@ -68,8 +68,6 @@ else if (circuitBreaker == provider.getCluster().getCircuitBreaker() && !provide + "If applicable, consider failing back OR restarting with an available cluster/database endpoint"); } // Ignore exceptions since we are already in a failure state - } catch (RuntimeException e) { - throw e; } finally { lock.unlock(); } diff --git a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java index 9680d452b1..015aadd191 100644 --- a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java +++ b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java @@ -5,7 +5,7 @@ import redis.clients.jedis.JedisClientConfig; import redis.clients.jedis.UnifiedJedis; import redis.clients.jedis.exceptions.JedisConnectionException; -import redis.clients.jedis.mcf.FailoverOptions.StrategySupplier; +import redis.clients.jedis.MultiClusterClientConfig.StrategySupplier; public class EchoStrategy implements HealthCheckStrategy { diff --git a/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java b/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java deleted file mode 100644 index a39674d635..0000000000 --- a/src/main/java/redis/clients/jedis/mcf/FailoverOptions.java +++ /dev/null @@ -1,132 +0,0 @@ -package redis.clients.jedis.mcf; - -import redis.clients.jedis.HostAndPort; -import redis.clients.jedis.JedisClientConfig; -import redis.clients.jedis.annots.Experimental; - -/** - * Configuration options for CircuitBreakerCommandExecutor - */ -@Experimental -public class FailoverOptions { - private final boolean retryOnFailover; - private final StrategySupplier healthCheckStrategySupplier; - private final float weight; - private boolean failback; - - private FailoverOptions(Builder builder) { - this.retryOnFailover = builder.retryOnFailover; - if (builder.healthCheckStrategySupplier != null) { - this.healthCheckStrategySupplier = builder.healthCheckStrategySupplier; - } else { - this.healthCheckStrategySupplier = builder.enableHealthCheck ? EchoStrategy.DEFAULT : null; - } - this.weight = builder.weight; - this.failback = builder.failback; - - } - - public static interface StrategySupplier { - /** - * Creates a HealthCheckStrategy for the given endpoint. - * @param hostAndPort the endpoint to create a strategy for - * @param jedisClientConfig the client configuration, may be null for implementations that don't need it - * @return a HealthCheckStrategy instance - */ - HealthCheckStrategy get(HostAndPort hostAndPort, JedisClientConfig jedisClientConfig); - } - - public StrategySupplier getStrategySupplier() { - return healthCheckStrategySupplier; - } - - public float getWeight() { - return weight; - } - - public boolean isFailbackEnabled() { - return failback; - } - - /** - * Gets whether to retry failed commands during failover - * @return true if retry is enabled, false otherwise - */ - public boolean isRetryOnFailover() { - return retryOnFailover; - } - - /** - * Creates a new builder with default options - * @return a new builder - */ - public static Builder builder() { - return new Builder(); - } - - /** - * Builder for FailoverOptions - */ - public static class Builder { - private boolean retryOnFailover = false; - private StrategySupplier healthCheckStrategySupplier; - private float weight = 1.0f; - private boolean failback; - private boolean enableHealthCheck = false; - - private Builder() { - } - - /** - * Sets whether to retry failed commands during failover - * @param retry true to retry, false otherwise - * @return this builder for method chaining - */ - public Builder retryOnFailover(boolean retry) { - this.retryOnFailover = retry; - return this; - } - - public Builder healthCheckStrategySupplier(StrategySupplier healthCheckStrategySupplier) { - if (healthCheckStrategySupplier == null) { - throw new IllegalArgumentException("healthCheckStrategySupplier must not be null"); - } - this.healthCheckStrategySupplier = healthCheckStrategySupplier; - return this; - } - - public Builder weight(float weight) { - this.weight = weight; - return this; - } - - public Builder failback(boolean failbackEnabled) { - this.failback = failbackEnabled; - return this; - } - - public Builder healthCheckStrategy(HealthCheckStrategy healthCheckStrategy) { - if (healthCheckStrategy == null) { - throw new IllegalArgumentException("healthCheckStrategy must not be null"); - } - this.healthCheckStrategySupplier = (hostAndPort, jedisClientConfig) -> healthCheckStrategy; - return this; - } - - public Builder enableHealthCheck(boolean enableHealthCheck) { - this.enableHealthCheck = enableHealthCheck; - if (!enableHealthCheck) { - this.healthCheckStrategySupplier = null; - } - return this; - } - - /** - * Builds a new FailoverOptions instance with the configured options - * @return a new FailoverOptions instance - */ - public FailoverOptions build() { - return new FailoverOptions(this); - } - } -} \ No newline at end of file diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatus.java b/src/main/java/redis/clients/jedis/mcf/HealthStatus.java index f28761d0b9..9e5ba4bbcd 100644 --- a/src/main/java/redis/clients/jedis/mcf/HealthStatus.java +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatus.java @@ -1,7 +1,7 @@ package redis.clients.jedis.mcf; public enum HealthStatus { - HEALTHY(0x01), UNHEALTHY(0x02), FORCED_UNHEALTHY(0x04); + HEALTHY(0x01), UNHEALTHY(0x02); private final int value; @@ -12,8 +12,4 @@ public enum HealthStatus { public boolean isHealthy() { return (this.value & HEALTHY.value) != 0; } - - public boolean isForcedUnhealthy() { - return (this.value & FORCED_UNHEALTHY.value) != 0; - } } diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java b/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java index 7bb4d315f1..39d71a1e80 100644 --- a/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java @@ -21,7 +21,6 @@ public void unregisterListener(HealthStatusListener listener) { public void registerListener(Endpoint endpoint, HealthStatusListener listener) { endpointListeners.computeIfAbsent(endpoint, k -> new CopyOnWriteArrayList<>()).add(listener); - listeners.add(listener); } public void unregisterListener(Endpoint endpoint, HealthStatusListener listener) { @@ -29,7 +28,6 @@ public void unregisterListener(Endpoint endpoint, HealthStatusListener listener) v.remove(listener); return v; }); - listeners.remove(listener); } public void notifyListeners(HealthStatusChangeEvent eventArgs) { diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index df9ab8d01f..15c60aea89 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -32,11 +32,10 @@ import redis.clients.jedis.mcf.HealthStatus; import redis.clients.jedis.mcf.HealthStatusChangeEvent; import redis.clients.jedis.mcf.HealthStatusManager; -import redis.clients.jedis.mcf.FailoverOptions.StrategySupplier; +import redis.clients.jedis.MultiClusterClientConfig.StrategySupplier; import redis.clients.jedis.util.Pool; import redis.clients.jedis.mcf.Endpoint; -import redis.clients.jedis.mcf.FailoverOptions; import redis.clients.jedis.mcf.HealthCheckStrategy; @@ -74,7 +73,7 @@ public class MultiClusterPooledConnectionProvider implements ConnectionProvider /** * Functional interface typically used for activeMultiClusterIndex persistence or custom logging after a successful * failover of a cluster/database endpoint (connection pool). Cluster/database endpoint info is passed as the sole - * parameter Example: cluster:2:redis-smart-cache.demo.com:12000 + * parameter Example: cluster:redis-smart-cache.demo.com:12000 */ private Consumer clusterFailoverPostProcessor; @@ -135,7 +134,7 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste for (ClusterConfig config : clusterConfigs) { GenericObjectPoolConfig poolConfig = config.getConnectionPoolConfig(); - String clusterId = "cluster:" + config.getFailoverOptions().getWeight() + ":" + config.getHostAndPort(); + String clusterId = "cluster:" + config.getHostAndPort(); Retry retry = RetryRegistry.of(retryConfig).retry(clusterId); @@ -158,10 +157,10 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste } else { pool = new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig()); } - Cluster cluster = new Cluster(pool, retry, circuitBreaker, config.getFailoverOptions()); + Cluster cluster = new Cluster(pool, retry, circuitBreaker, config.getWeight(), multiClusterClientConfig); multiClusterMap.put(config.getHostAndPort(), cluster); - StrategySupplier strategySupplier = config.getFailoverOptions().getStrategySupplier(); + StrategySupplier strategySupplier = config.getHealthCheckStrategySupplier(); if (strategySupplier != null) { HealthCheckStrategy hcs = strategySupplier.get(config.getHostAndPort(), config.getJedisClientConfig()); healthStatusManager.add(config.getHostAndPort(), hcs); @@ -189,7 +188,7 @@ private void handleStatusChange(HealthStatusChangeEvent eventArgs) { Cluster clusterWithHealthChange = multiClusterMap.get(endpoint); if (clusterWithHealthChange == null) return; - if (clusterWithHealthChange.isForcedUnhealthy()) return; + if (clusterWithHealthChange.isDisabled()) return; clusterWithHealthChange.setHealthStatus(newStatus); @@ -203,7 +202,9 @@ private void handleStatusChange(HealthStatusChangeEvent eventArgs) { } } } else if (clusterWithHealthChange == activeCluster) { - iterateActiveCluster(); + if (iterateActiveCluster() != null) { + this.runClusterFailoverPostProcessor(activeCluster); + } } } @@ -215,15 +216,15 @@ public Endpoint iterateActiveCluster() { + "provided with an additional cluster/database endpoint according to its prioritized sequence. " + "If applicable, consider failing back OR restarting with an available cluster/database endpoint"); } - setActiveCluster(clusterToIterate.getValue(), false); - return clusterToIterate.getKey(); + boolean changed = setActiveCluster(clusterToIterate.getValue(), false); + return changed ? clusterToIterate.getKey() : null; } private static Comparator> maxByWeight = Map.Entry . comparingByValue(Comparator.comparing(Cluster::getWeight)); private static Predicate> filterByHealth = c -> c.getValue().isHealthy(); - private static Predicate> filterByFailback = c -> c.getValue().isHealthy(); + private static Predicate> filterByFailback = c -> c.getValue().isFailbackEnabled(); private Map.Entry findWeightedHealthyCluster() { Cluster current = activeCluster; @@ -283,7 +284,7 @@ public void setActiveCluster(Endpoint endpoint) { setActiveCluster(cluster, true); } - private void setActiveCluster(Cluster cluster, boolean validateConnection) { + private boolean setActiveCluster(Cluster cluster, boolean validateConnection) { // Cluster cluster = clusterEntry.getValue(); // Field-level synchronization is used to avoid the edge case in which // incrementActiveMultiClusterIndex() is called at the same time @@ -293,7 +294,7 @@ private void setActiveCluster(Cluster cluster, boolean validateConnection) { // Allows an attempt to reset the current cluster from a FORCED_OPEN to CLOSED state in the event that no // failover is possible - if (activeCluster == cluster && !cluster.isCBForcedOpen()) return; + if (activeCluster == cluster && !cluster.isCBForcedOpen()) return false; if (validateConnection) validateTargetConnection(cluster); @@ -304,9 +305,11 @@ private void setActiveCluster(Cluster cluster, boolean validateConnection) { else log.warn("Cluster/database endpoint successfully updated from '{}' to '{}'", originalClusterName, cluster.circuitBreaker.getName()); if (!activeCluster.isFailbackEnabled()) { - activeCluster.setHealthStatus(HealthStatus.FORCED_UNHEALTHY); + activeCluster.disable(); } + Cluster temp = activeCluster; activeCluster = cluster; + return temp != activeCluster; } finally { activeClusterIndexLock.unlock(); } @@ -385,15 +388,20 @@ public static class Cluster { private final float weight; // it starts its life with the assumption of being healthy private HealthStatus healthStatus = HealthStatus.HEALTHY; - private FailoverOptions failoverOptions; + private MultiClusterClientConfig multiClusterClientConfig; + private boolean disabled = false; - public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circuitBreaker, - FailoverOptions failoverOptions) { + public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circuitBreaker, float weight, + MultiClusterClientConfig multiClusterClientConfig) { this.connectionPool = connectionPool; this.retry = retry; this.circuitBreaker = circuitBreaker; - this.failoverOptions = failoverOptions; - this.weight = failoverOptions.getWeight(); + this.weight = weight; + this.multiClusterClientConfig = multiClusterClientConfig; + } + + public void disable() { + disabled = true; } public Connection getConnection() { @@ -432,19 +440,26 @@ public boolean isCBForcedOpen() { } public boolean isHealthy() { - return healthStatus.isHealthy() && !isCBForcedOpen(); + return healthStatus.isHealthy() && !isCBForcedOpen() && !disabled; } public boolean isFailbackEnabled() { - return failoverOptions.isFailbackEnabled(); + return multiClusterClientConfig.isFailback(); } - public boolean isForcedUnhealthy() { - return healthStatus.isForcedUnhealthy(); + public boolean isDisabled() { + return disabled; } public boolean retryOnFailover() { - return failoverOptions.isRetryOnFailover(); + return multiClusterClientConfig.isRetryOnFailover(); + } + + @Override + public String toString() { + return circuitBreaker.getName() + "{" + "connectionPool=" + connectionPool + ", retry=" + retry + + ", circuitBreaker=" + circuitBreaker + ", weight=" + weight + ", healthStatus=" + healthStatus + + ", multiClusterClientConfig=" + multiClusterClientConfig + ", disabled=" + disabled + '}'; } } diff --git a/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java b/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java index 7f0d7e5e02..b43676c88d 100644 --- a/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/failover/FailoverIntegrationTest.java @@ -19,8 +19,6 @@ import redis.clients.jedis.MultiClusterClientConfig; import redis.clients.jedis.UnifiedJedis; import redis.clients.jedis.exceptions.JedisConnectionException; -import redis.clients.jedis.mcf.EchoStrategy; -import redis.clients.jedis.mcf.FailoverOptions; import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; import redis.clients.jedis.scenario.RecommendedSettings; @@ -177,15 +175,7 @@ private List getClusterConfigs(JedisClie EndpointConfig... endpoints) { return Arrays.stream(endpoints) - .map(e -> new MultiClusterClientConfig.ClusterConfig(e.getHostAndPort(), clientConfig)) - .collect(Collectors.toList()); - } - - private List getClusterConfigs(JedisClientConfig clientConfig, - FailoverOptions failoverOptions, EndpointConfig... endpoints) { - - return Arrays.stream(endpoints).map(e -> MultiClusterClientConfig.ClusterConfig - .builder(e.getHostAndPort(), clientConfig).failoverOptions(failoverOptions).build()) + .map(e -> MultiClusterClientConfig.ClusterConfig.builder(e.getHostAndPort(), clientConfig).build()) .collect(Collectors.toList()); } @@ -419,20 +409,20 @@ private MultiClusterPooledConnectionProvider createProvider() { * @return A configured provider */ private MultiClusterPooledConnectionProvider createProvider( - Function optionsCustomizer) { - FailoverOptions.Builder builder = FailoverOptions.builder(); - if (optionsCustomizer != null) { - builder = optionsCustomizer.apply(builder); - } + Function configCustomizer) { JedisClientConfig clientConfig = DefaultJedisClientConfig.builder() .socketTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS) .connectionTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS).build(); - MultiClusterClientConfig failoverConfig = new MultiClusterClientConfig.Builder( - getClusterConfigs(clientConfig, builder.build(), endpoint1, endpoint2)).retryMaxAttempts(1) - .retryWaitDuration(1).circuitBreakerSlidingWindowType(COUNT_BASED).circuitBreakerSlidingWindowSize(1) - .circuitBreakerFailureRateThreshold(100).circuitBreakerSlidingWindowMinCalls(1).build(); + MultiClusterClientConfig.Builder builder = new MultiClusterClientConfig.Builder( + getClusterConfigs(clientConfig, endpoint1, endpoint2)).retryMaxAttempts(1).retryWaitDuration(1) + .circuitBreakerSlidingWindowType(COUNT_BASED).circuitBreakerSlidingWindowSize(1) + .circuitBreakerFailureRateThreshold(100).circuitBreakerSlidingWindowMinCalls(1); - return new MultiClusterPooledConnectionProvider(failoverConfig); + if (configCustomizer != null) { + builder = configCustomizer.apply(builder); + } + + return new MultiClusterPooledConnectionProvider(builder.build()); } } diff --git a/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java b/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java index bb8230e93c..5fab284fdf 100644 --- a/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java @@ -4,17 +4,19 @@ import java.util.Arrays; import java.util.List; +import java.util.function.Function; import java.util.stream.Collectors; import org.junit.jupiter.api.Test; import io.github.resilience4j.circuitbreaker.CircuitBreakerConfig.SlidingWindowType; -import redis.clients.jedis.DefaultJedisClientConfig; + import redis.clients.jedis.EndpointConfig; import redis.clients.jedis.HostAndPorts; import redis.clients.jedis.JedisClientConfig; import redis.clients.jedis.MultiClusterClientConfig; import redis.clients.jedis.UnifiedJedis; +import redis.clients.jedis.MultiClusterClientConfig.ClusterConfig; import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; import redis.clients.jedis.scenario.RecommendedSettings; @@ -29,9 +31,8 @@ public class HealthCheckIntegrationTest { @Test public void testDisableHealthCheck() { - FailoverOptions fo1 = FailoverOptions.builder().build(); - - MultiClusterPooledConnectionProvider customProvider = getMCCF(fo1); + // No health check strategy supplier means health check is disabled + MultiClusterPooledConnectionProvider customProvider = getMCCF(null); try (UnifiedJedis customClient = new UnifiedJedis(customProvider)) { // Verify that the client can connect and execute commands String result = customClient.ping(); @@ -41,9 +42,11 @@ public void testDisableHealthCheck() { @Test public void testDefaultStrategySupplier() { - FailoverOptions fo1 = FailoverOptions.builder().enableHealthCheck(true).build(); - - MultiClusterPooledConnectionProvider customProvider = getMCCF(fo1); + // Create a default strategy supplier that creates EchoStrategy instances + MultiClusterClientConfig.StrategySupplier defaultSupplier = (hostAndPort, jedisClientConfig) -> { + return new EchoStrategy(hostAndPort, jedisClientConfig); + }; + MultiClusterPooledConnectionProvider customProvider = getMCCF(defaultSupplier); try (UnifiedJedis customClient = new UnifiedJedis(customProvider)) { // Verify that the client can connect and execute commands String result = customClient.ping(); @@ -54,8 +57,7 @@ public void testDefaultStrategySupplier() { @Test public void testCustomStrategySupplier() { // Create a StrategySupplier that uses the JedisClientConfig when available - FailoverOptions.StrategySupplier strategySupplier = (hostAndPort, jedisClientConfig) -> { - UnifiedJedis pinger = new UnifiedJedis(hostAndPort, jedisClientConfig); + MultiClusterClientConfig.StrategySupplier strategySupplier = (hostAndPort, jedisClientConfig) -> { return new HealthCheckStrategy() { @Override @@ -70,15 +72,18 @@ public int getTimeout() { @Override public HealthStatus doHealthCheck(Endpoint endpoint) { - return "OK".equals(pinger.ping()) ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; + // Create connection per health check to avoid resource leak + try (UnifiedJedis pinger = new UnifiedJedis(hostAndPort, jedisClientConfig)) { + return "OK".equals(pinger.ping()) ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; + } catch (Exception e) { + return HealthStatus.UNHEALTHY; + } } }; }; - FailoverOptions fo1 = FailoverOptions.builder().healthCheckStrategySupplier(strategySupplier).build(); - - MultiClusterPooledConnectionProvider customProvider = getMCCF(fo1); + MultiClusterPooledConnectionProvider customProvider = getMCCF(strategySupplier); try (UnifiedJedis customClient = new UnifiedJedis(customProvider)) { // Verify that the client can connect and execute commands String result = customClient.ping(); @@ -86,10 +91,14 @@ public HealthStatus doHealthCheck(Endpoint endpoint) { } } - private MultiClusterPooledConnectionProvider getMCCF(FailoverOptions fo) { - List clusterConfigs = Arrays.stream(new EndpointConfig[] { endpoint1 }) - .map(e -> MultiClusterClientConfig.ClusterConfig.builder(e.getHostAndPort(), clientConfig) - .failoverOptions(fo).build()) + private MultiClusterPooledConnectionProvider getMCCF(MultiClusterClientConfig.StrategySupplier strategySupplier) { + Function modifier = builder -> strategySupplier == null + ? builder.healthCheckEnabled(false) + : builder.healthCheckStrategySupplier(strategySupplier); + + List clusterConfigs = Arrays + .stream(new EndpointConfig[] { endpoint1 }).map(e -> modifier + .apply(MultiClusterClientConfig.ClusterConfig.builder(e.getHostAndPort(), clientConfig)).build()) .collect(Collectors.toList()); MultiClusterClientConfig mccf = new MultiClusterClientConfig.Builder(clusterConfigs).retryMaxAttempts(1) diff --git a/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java b/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java index 686f9bfb9c..0678f365e5 100644 --- a/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java +++ b/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java @@ -8,6 +8,7 @@ import redis.clients.jedis.DefaultJedisClientConfig; import redis.clients.jedis.HostAndPort; import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.MultiClusterClientConfig; import redis.clients.jedis.UnifiedJedis; import java.util.concurrent.CountDownLatch; @@ -64,14 +65,6 @@ void setUp() { void testHealthStatusIsHealthy() { assertTrue(HealthStatus.HEALTHY.isHealthy()); assertFalse(HealthStatus.UNHEALTHY.isHealthy()); - assertFalse(HealthStatus.FORCED_UNHEALTHY.isHealthy()); - } - - @Test - void testHealthStatusIsForcedUnhealthy() { - assertFalse(HealthStatus.HEALTHY.isForcedUnhealthy()); - assertFalse(HealthStatus.UNHEALTHY.isForcedUnhealthy()); - assertTrue(HealthStatus.FORCED_UNHEALTHY.isForcedUnhealthy()); } // ========== HostAndPort Tests ========== @@ -270,70 +263,125 @@ void testEchoStrategyCustomIntervalTimeout() { @Test void testEchoStrategyDefaultSupplier() { - FailoverOptions.StrategySupplier supplier = EchoStrategy.DEFAULT; + MultiClusterClientConfig.StrategySupplier supplier = EchoStrategy.DEFAULT; HealthCheckStrategy strategy = supplier.get(testEndpoint, testConfig); assertInstanceOf(EchoStrategy.class, strategy); } - // ========== FailoverOptions Tests ========== + // ========== Failover configuration Tests ========== @Test - void testFailoverOptionsBuilder() { - FailoverOptions options = FailoverOptions.builder().retryOnFailover(true).weight(2.5f).failback(true).build(); - - assertTrue(options.isRetryOnFailover()); - assertEquals(2.5f, options.getWeight()); - assertTrue(options.isFailbackEnabled()); + void testNewFieldLocations() { + // Test new field locations in ClusterConfig and MultiClusterClientConfig + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(testEndpoint, testConfig) + .weight(2.5f) + .build(); + + MultiClusterClientConfig multiConfig = new MultiClusterClientConfig.Builder(new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .retryOnFailover(true) + .failback(true) + .build(); + + assertEquals(2.5f, clusterConfig.getWeight()); + assertTrue(multiConfig.isRetryOnFailover()); + assertTrue(multiConfig.isFailback()); } @Test - void testFailoverOptionsDefaultValues() { - FailoverOptions options = FailoverOptions.builder().build(); + void testDefaultValues() { + // Test default values in ClusterConfig + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(testEndpoint, testConfig) + .build(); - assertFalse(options.isRetryOnFailover()); // Default is false - assertEquals(1.0f, options.getWeight()); // Default weight - assertFalse(options.isFailbackEnabled()); // Default is false + assertEquals(1.0f, clusterConfig.getWeight()); // Default weight + assertEquals(EchoStrategy.DEFAULT, clusterConfig.getHealthCheckStrategySupplier()); // Default is null (no health check) + + // Test default values in MultiClusterClientConfig + MultiClusterClientConfig multiConfig = new MultiClusterClientConfig.Builder(new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .build(); + + assertFalse(multiConfig.isRetryOnFailover()); // Default is false + assertFalse(multiConfig.isFailback()); // Default is false } @Test - void testFailoverOptionsWithHealthCheckStrategy() { + void testClusterConfigWithHealthCheckStrategy() { HealthCheckStrategy customStrategy = mock(HealthCheckStrategy.class); - FailoverOptions options = FailoverOptions.builder().healthCheckStrategy(customStrategy).build(); + MultiClusterClientConfig.StrategySupplier supplier = (hostAndPort, jedisClientConfig) -> customStrategy; - FailoverOptions.StrategySupplier supplier = options.getStrategySupplier(); - assertNotNull(supplier); + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(testEndpoint, testConfig) + .healthCheckStrategySupplier(supplier) + .build(); - HealthCheckStrategy result = supplier.get(testEndpoint, testConfig); + assertNotNull(clusterConfig.getHealthCheckStrategySupplier()); + HealthCheckStrategy result = clusterConfig.getHealthCheckStrategySupplier().get(testEndpoint, testConfig); assertEquals(customStrategy, result); } @Test - void testFailoverOptionsWithStrategySupplier() { - FailoverOptions.StrategySupplier customSupplier = (hostAndPort, jedisClientConfig) -> { + void testClusterConfigWithStrategySupplier() { + MultiClusterClientConfig.StrategySupplier customSupplier = (hostAndPort, jedisClientConfig) -> { return mock(HealthCheckStrategy.class); }; - FailoverOptions options = FailoverOptions.builder().healthCheckStrategySupplier(customSupplier).build(); + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(testEndpoint, testConfig) + .healthCheckStrategySupplier(customSupplier) + .build(); - assertEquals(customSupplier, options.getStrategySupplier()); + assertEquals(customSupplier, clusterConfig.getHealthCheckStrategySupplier()); } @Test - void testFailoverOptionsWithEnabledHealthCheck() { - FailoverOptions options = FailoverOptions.builder().enableHealthCheck(true).build(); + void testClusterConfigWithEchoStrategy() { + MultiClusterClientConfig.StrategySupplier echoSupplier = (hostAndPort, jedisClientConfig) -> { + return new EchoStrategy(hostAndPort, jedisClientConfig); + }; - FailoverOptions.StrategySupplier supplier = options.getStrategySupplier(); + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(testEndpoint, testConfig) + .healthCheckStrategySupplier(echoSupplier) + .build(); + + MultiClusterClientConfig.StrategySupplier supplier = clusterConfig.getHealthCheckStrategySupplier(); assertNotNull(supplier); - assertEquals(EchoStrategy.DEFAULT, supplier); + assertInstanceOf(EchoStrategy.class, supplier.get(testEndpoint, testConfig)); + } + + @Test + void testClusterConfigWithDefaultHealthCheck() { + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(testEndpoint, testConfig) + .build(); // Should use default EchoStrategy + + assertNotNull(clusterConfig.getHealthCheckStrategySupplier()); + assertEquals(EchoStrategy.DEFAULT, clusterConfig.getHealthCheckStrategySupplier()); } @Test - void testFailoverOptionsWithDisabledHealthCheck() { - FailoverOptions options = FailoverOptions.builder().enableHealthCheck(false).build(); + void testClusterConfigWithDisabledHealthCheck() { + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(testEndpoint, testConfig) + .healthCheckEnabled(false) + .build(); + + assertNull(clusterConfig.getHealthCheckStrategySupplier()); + } - assertNull(options.getStrategySupplier()); + @Test + void testClusterConfigHealthCheckEnabledExplicitly() { + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(testEndpoint, testConfig) + .healthCheckEnabled(true) + .build(); + + assertNotNull(clusterConfig.getHealthCheckStrategySupplier()); + assertEquals(EchoStrategy.DEFAULT, clusterConfig.getHealthCheckStrategySupplier()); } // ========== Integration Tests ========== @@ -375,9 +423,9 @@ public HealthStatus doHealthCheck(Endpoint endpoint) { } @Test - void testFailoverOptionsStrategySupplierPolymorphism() { + void testStrategySupplierPolymorphism() { // Test that the polymorphic design works correctly - FailoverOptions.StrategySupplier supplier = (hostAndPort, jedisClientConfig) -> { + MultiClusterClientConfig.StrategySupplier supplier = (hostAndPort, jedisClientConfig) -> { if (jedisClientConfig != null) { return new EchoStrategy(hostAndPort, jedisClientConfig, 500, 250); } else { diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java index 78ba57a3eb..cf9733e3e4 100644 --- a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java @@ -8,7 +8,7 @@ import redis.clients.jedis.exceptions.JedisConnectionException; import redis.clients.jedis.exceptions.JedisValidationException; import redis.clients.jedis.mcf.Endpoint; -import redis.clients.jedis.mcf.FailoverOptions; + import java.util.concurrent.atomic.AtomicBoolean; @@ -30,10 +30,10 @@ public void setUp() { ClusterConfig[] clusterConfigs = new ClusterConfig[2]; clusterConfigs[0] = ClusterConfig .builder(endpointStandalone0.getHostAndPort(), endpointStandalone0.getClientConfigBuilder().build()) - .failoverOptions(FailoverOptions.builder().weight(0.5f).build()).build(); + .weight(0.5f).build(); clusterConfigs[1] = ClusterConfig .builder(endpointStandalone1.getHostAndPort(), endpointStandalone1.getClientConfigBuilder().build()) - .failoverOptions(FailoverOptions.builder().weight(0.3f).build()).build(); + .weight(0.3f).build(); provider = new MultiClusterPooledConnectionProvider( new MultiClusterClientConfig.Builder(clusterConfigs).build()); @@ -95,10 +95,10 @@ public void testRunClusterFailoverPostProcessor() { ClusterConfig[] clusterConfigs = new ClusterConfig[2]; clusterConfigs[0] = ClusterConfig .builder(new HostAndPort("purposefully-incorrect", 0000), DefaultJedisClientConfig.builder().build()) - .failoverOptions(FailoverOptions.builder().weight(0.5f).build()).build(); + .weight(0.5f).healthCheckEnabled(false).build(); clusterConfigs[1] = ClusterConfig .builder(new HostAndPort("purposefully-incorrect", 0001), DefaultJedisClientConfig.builder().build()) - .failoverOptions(FailoverOptions.builder().weight(0.4f).build()).build(); + .weight(0.4f).healthCheckEnabled(false).build(); MultiClusterClientConfig.Builder builder = new MultiClusterClientConfig.Builder(clusterConfigs); @@ -115,13 +115,7 @@ public void testRunClusterFailoverPostProcessor() { try (UnifiedJedis jedis = new UnifiedJedis(localProvider)) { - // This should fail after 3 retries and meet the requirements to open the circuit on the next iteration - try { - jedis.get("foo"); - } catch (Exception e) { - } - - // This should fail after 3 retries and open the circuit which will trigger the post processor + // This will fail due to unable to connect and open the circuit which will trigger the post processor try { jedis.get("foo"); } catch (Exception e) { From c2fb34c4cd9ac5935dce683bf2bc1dd324946fa4 Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 16 Jul 2025 12:06:50 +0300 Subject: [PATCH 08/23] Update src/main/java/redis/clients/jedis/mcf/EchoStrategy.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- src/main/java/redis/clients/jedis/mcf/EchoStrategy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java index 015aadd191..e0431d50bb 100644 --- a/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java +++ b/src/main/java/redis/clients/jedis/mcf/EchoStrategy.java @@ -24,7 +24,7 @@ public EchoStrategy(HostAndPort hostAndPort, JedisClientConfig jedisClientConfig try { this.jedis = new UnifiedJedis(connFactory.makeObject().getObject()); } catch (Exception e) { - new JedisConnectionException("HealtCheck connection Failed!", e); + throw new JedisConnectionException("HealthCheck connection Failed!", e); } } From ade866d14d06cf42b78c8cb1fdd45802c4e6376d Mon Sep 17 00:00:00 2001 From: atakavci Date: Fri, 11 Jul 2025 11:26:24 +0300 Subject: [PATCH 09/23] - add remove endpoints --- .../jedis/MultiClusterClientConfig.java | 4 + .../MultiClusterPooledConnectionProvider.java | 169 +++++++++++---- .../jedis/mcf/HealthCheckIntegrationTest.java | 2 - .../MultiClusterDynamicEndpointUnitTest.java | 201 ++++++++++++++++++ 4 files changed, 333 insertions(+), 43 deletions(-) create mode 100644 src/test/java/redis/clients/jedis/mcf/MultiClusterDynamicEndpointUnitTest.java diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 2656c36631..8cba817eea 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -224,6 +224,10 @@ public boolean isFailback() { return failback; } + public static Builder builder(ClusterConfig[] clusterConfigs) { + return new Builder(clusterConfigs); + } + public static class ClusterConfig { private HostAndPort hostAndPort; diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 15c60aea89..02ed3b33ee 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -81,6 +81,11 @@ public class MultiClusterPooledConnectionProvider implements ConnectionProvider private HealthStatusManager healthStatusManager = new HealthStatusManager(); + // Store retry and circuit breaker configs for dynamic cluster addition/removal + private RetryConfig retryConfig; + private CircuitBreakerConfig circuitBreakerConfig; + private MultiClusterClientConfig multiClusterClientConfig; + public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiClusterClientConfig) { if (multiClusterClientConfig == null) throw new JedisValidationException( @@ -101,7 +106,7 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste if (retryIgnoreExceptionList != null) retryConfigBuilder.ignoreExceptions(retryIgnoreExceptionList.stream().toArray(Class[]::new)); - RetryConfig retryConfig = retryConfigBuilder.build(); + this.retryConfig = retryConfigBuilder.build(); ////////////// Configure Circuit Breaker //////////////////// @@ -126,30 +131,125 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste if (circuitBreakerIgnoreExceptionList != null) circuitBreakerConfigBuilder .ignoreExceptions(circuitBreakerIgnoreExceptionList.stream().toArray(Class[]::new)); - CircuitBreakerConfig circuitBreakerConfig = circuitBreakerConfigBuilder.build(); + this.circuitBreakerConfig = circuitBreakerConfigBuilder.build(); ////////////// Configure Cluster Map //////////////////// ClusterConfig[] clusterConfigs = multiClusterClientConfig.getClusterConfigs(); for (ClusterConfig config : clusterConfigs) { - GenericObjectPoolConfig poolConfig = config.getConnectionPoolConfig(); + addClusterInternal(multiClusterClientConfig, config); + } + + // selecting activeCluster with configuration values. + // all health status would be HEALTHY at this point + activeCluster = findWeightedHealthyClusterToIterate().getValue(); + + for (Endpoint endpoint : multiClusterMap.keySet()) { + healthStatusManager.registerListener(endpoint, this::handleStatusChange); + } + /// --- /// + + this.fallbackExceptionList = multiClusterClientConfig.getFallbackExceptionList(); + } + + /** + * Adds a new cluster endpoint to the provider. + * @param clusterConfig the configuration for the new cluster + * @throws JedisValidationException if the endpoint already exists + */ + public void add(ClusterConfig clusterConfig) { + if (clusterConfig == null) { + throw new JedisValidationException("ClusterConfig must not be null"); + } + + Endpoint endpoint = clusterConfig.getHostAndPort(); + if (multiClusterMap.containsKey(endpoint)) { + throw new JedisValidationException("Endpoint " + endpoint + " already exists in the provider"); + } + + activeClusterIndexLock.lock(); + try { + addClusterInternal(multiClusterClientConfig, clusterConfig); + healthStatusManager.registerListener(endpoint, this::handleStatusChange); + } finally { + activeClusterIndexLock.unlock(); + } + } + + /** + * Removes a cluster endpoint from the provider. + * @param endpoint the endpoint to remove + * @throws JedisValidationException if the endpoint doesn't exist or is the last remaining endpoint + */ + public void remove(Endpoint endpoint) { + if (endpoint == null) { + throw new JedisValidationException("Endpoint must not be null"); + } + + if (!multiClusterMap.containsKey(endpoint)) { + throw new JedisValidationException("Endpoint " + endpoint + " does not exist in the provider"); + } + + if (multiClusterMap.size() < 2) { + throw new JedisValidationException("Cannot remove the last remaining endpoint"); + } + + activeClusterIndexLock.lock(); + try { + Cluster clusterToRemove = multiClusterMap.get(endpoint); + boolean isActiveCluster = (activeCluster == clusterToRemove); + + if (isActiveCluster) { + log.info("Active cluster is being removed. Finding a new active cluster..."); + + // If we removed the active cluster, find a new one + if (isActiveCluster) { + Map.Entry candidateCluster = findWeightedHealthyClusterToIterate(); + if (candidateCluster != null) { + setActiveCluster(candidateCluster.getValue(), true); + } + } + } + + // Remove from health status manager first + healthStatusManager.unregisterListener(endpoint, this::handleStatusChange); + healthStatusManager.remove(endpoint); + + // Remove from cluster map + multiClusterMap.remove(endpoint); + + // Close the cluster resources + if (clusterToRemove != null) { + clusterToRemove.getConnectionPool().close(); + } + } finally { + activeClusterIndexLock.unlock(); + } + } + + /** + * Internal method to add a cluster configuration. This method is not thread-safe and should be called within + * appropriate locks. + */ + private void addClusterInternal(MultiClusterClientConfig multiClusterClientConfig, ClusterConfig config) { + GenericObjectPoolConfig poolConfig = config.getConnectionPoolConfig(); String clusterId = "cluster:" + config.getHostAndPort(); - Retry retry = RetryRegistry.of(retryConfig).retry(clusterId); + Retry retry = RetryRegistry.of(retryConfig).retry(clusterId); - Retry.EventPublisher retryPublisher = retry.getEventPublisher(); - retryPublisher.onRetry(event -> log.warn(String.valueOf(event))); - retryPublisher.onError(event -> log.error(String.valueOf(event))); + Retry.EventPublisher retryPublisher = retry.getEventPublisher(); + retryPublisher.onRetry(event -> log.warn(String.valueOf(event))); + retryPublisher.onError(event -> log.error(String.valueOf(event))); - CircuitBreaker circuitBreaker = CircuitBreakerRegistry.of(circuitBreakerConfig).circuitBreaker(clusterId); + CircuitBreaker circuitBreaker = CircuitBreakerRegistry.of(circuitBreakerConfig).circuitBreaker(clusterId); - CircuitBreaker.EventPublisher circuitBreakerEventPublisher = circuitBreaker.getEventPublisher(); - circuitBreakerEventPublisher.onCallNotPermitted(event -> log.error(String.valueOf(event))); - circuitBreakerEventPublisher.onError(event -> log.error(String.valueOf(event))); - circuitBreakerEventPublisher.onFailureRateExceeded(event -> log.error(String.valueOf(event))); - circuitBreakerEventPublisher.onSlowCallRateExceeded(event -> log.error(String.valueOf(event))); - circuitBreakerEventPublisher.onStateTransition(event -> log.warn(String.valueOf(event))); + CircuitBreaker.EventPublisher circuitBreakerEventPublisher = circuitBreaker.getEventPublisher(); + circuitBreakerEventPublisher.onCallNotPermitted(event -> log.error(String.valueOf(event))); + circuitBreakerEventPublisher.onError(event -> log.error(String.valueOf(event))); + circuitBreakerEventPublisher.onFailureRateExceeded(event -> log.error(String.valueOf(event))); + circuitBreakerEventPublisher.onSlowCallRateExceeded(event -> log.error(String.valueOf(event))); + circuitBreakerEventPublisher.onStateTransition(event -> log.warn(String.valueOf(event))); ConnectionPool pool; if (poolConfig != null) { @@ -160,23 +260,11 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste Cluster cluster = new Cluster(pool, retry, circuitBreaker, config.getWeight(), multiClusterClientConfig); multiClusterMap.put(config.getHostAndPort(), cluster); - StrategySupplier strategySupplier = config.getHealthCheckStrategySupplier(); - if (strategySupplier != null) { - HealthCheckStrategy hcs = strategySupplier.get(config.getHostAndPort(), config.getJedisClientConfig()); - healthStatusManager.add(config.getHostAndPort(), hcs); - } - } - - // selecting activeCluster with configuration values. - // all health status would be HEALTHY at this point - activeCluster = findWeightedHealthyCluster().getValue(); - - for (Endpoint endpoint : multiClusterMap.keySet()) { - healthStatusManager.registerListener(endpoint, this::handleStatusChange); + StrategySupplier strategySupplier = config.getHealthCheckStrategySupplier(); + if (strategySupplier != null) { + HealthCheckStrategy hcs = strategySupplier.get(config.getHostAndPort(), config.getJedisClientConfig()); + healthStatusManager.add(config.getHostAndPort(), hcs); } - /// --- /// - - this.fallbackExceptionList = multiClusterClientConfig.getFallbackExceptionList(); } private void handleStatusChange(HealthStatusChangeEvent eventArgs) { @@ -195,7 +283,7 @@ private void handleStatusChange(HealthStatusChangeEvent eventArgs) { if (newStatus.isHealthy()) { if (clusterWithHealthChange.isFailbackEnabled() && activeCluster != clusterWithHealthChange) { // lets check if weighted switching is possible - Map.Entry failbackCluster = findWeightedFailbackCluster(); + Map.Entry failbackCluster = findWeightedHealthyClusterToIterate(); if (failbackCluster == clusterWithHealthChange && clusterWithHealthChange.getWeight() > activeCluster.getWeight()) { setActiveCluster(clusterWithHealthChange, false); @@ -209,7 +297,7 @@ private void handleStatusChange(HealthStatusChangeEvent eventArgs) { } public Endpoint iterateActiveCluster() { - Map.Entry clusterToIterate = findWeightedHealthyCluster(); + Map.Entry clusterToIterate = findWeightedHealthyClusterToIterate(); if (clusterToIterate == null) { throw new JedisConnectionException( "Cluster/database endpoint could not failover since the MultiClusterClientConfig was not " @@ -226,16 +314,15 @@ public Endpoint iterateActiveCluster() { private static Predicate> filterByHealth = c -> c.getValue().isHealthy(); private static Predicate> filterByFailback = c -> c.getValue().isFailbackEnabled(); - private Map.Entry findWeightedHealthyCluster() { - Cluster current = activeCluster; - return multiClusterMap.entrySet().stream().filter(filterByHealth).filter(entry -> entry.getValue() != current) - .max(maxByWeight).orElse(null); - } + // private Map.Entry findWeightedHealthyCluster() { + // Cluster current = activeCluster; + // return multiClusterMap.entrySet().stream().filter(filterByHealth).filter(entry -> entry.getValue() != current) + // .max(maxByWeight).orElse(null); + // } - private Map.Entry findWeightedFailbackCluster() { - Cluster current = activeCluster; + private Map.Entry findWeightedHealthyClusterToIterate() { return multiClusterMap.entrySet().stream().filter(filterByHealth).filter(filterByFailback) - .filter(entry -> entry.getValue() != current).max(maxByWeight).orElse(null); + .filter(entry -> entry.getValue() != activeCluster).max(maxByWeight).orElse(null); } /** @@ -363,7 +450,7 @@ public CircuitBreaker getClusterCircuitBreaker(int multiClusterIndex) { * manually failback to an available cluster */ public boolean canIterateOnceMore() { - Map.Entry e = findWeightedHealthyCluster(); + Map.Entry e = findWeightedHealthyClusterToIterate(); return e != null; } diff --git a/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java b/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java index 5fab284fdf..eeb1e9caa7 100644 --- a/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java @@ -23,8 +23,6 @@ public class HealthCheckIntegrationTest { private final EndpointConfig endpoint1 = HostAndPorts.getRedisEndpoint("standalone0"); - private final EndpointConfig endpoint2 = HostAndPorts.getRedisEndpoint("standalone1"); - private final JedisClientConfig clientConfig = endpoint1.getClientConfigBuilder() .socketTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS) .connectionTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS).build(); diff --git a/src/test/java/redis/clients/jedis/mcf/MultiClusterDynamicEndpointUnitTest.java b/src/test/java/redis/clients/jedis/mcf/MultiClusterDynamicEndpointUnitTest.java new file mode 100644 index 0000000000..420c578588 --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/MultiClusterDynamicEndpointUnitTest.java @@ -0,0 +1,201 @@ +package redis.clients.jedis.mcf; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.MockedConstruction; + +import redis.clients.jedis.Connection; +import redis.clients.jedis.ConnectionPool; +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.EndpointConfig; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.HostAndPorts; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.MultiClusterClientConfig; +import redis.clients.jedis.MultiClusterClientConfig.ClusterConfig; +import redis.clients.jedis.exceptions.JedisValidationException; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockConstruction; +import static org.mockito.Mockito.when; + +public class MultiClusterDynamicEndpointUnitTest { + + private MultiClusterPooledConnectionProvider provider; + private JedisClientConfig clientConfig; + private final EndpointConfig endpoint1 = HostAndPorts.getRedisEndpoint("standalone0"); + private final EndpointConfig endpoint2 = HostAndPorts.getRedisEndpoint("standalone1"); + + @BeforeEach + void setUp() { + clientConfig = DefaultJedisClientConfig.builder().build(); + + // Create initial provider with endpoint1 + ClusterConfig initialConfig = createClusterConfig(endpoint1.getHostAndPort(), 1.0f); + + MultiClusterClientConfig multiConfig = new MultiClusterClientConfig.Builder( + new ClusterConfig[] { initialConfig }).build(); + + provider = new MultiClusterPooledConnectionProvider(multiConfig); + } + + // Helper method to create cluster configurations + private ClusterConfig createClusterConfig(HostAndPort hostAndPort, float weight) { + return ClusterConfig.builder(hostAndPort, clientConfig) + .weight(weight).build(); + } + + @Test + void testAddNewCluster() { + ClusterConfig newConfig = createClusterConfig(endpoint2.getHostAndPort(), 2.0f); + + // Should not throw exception + assertDoesNotThrow(() -> provider.add(newConfig)); + + // Verify the cluster was added by checking it can be retrieved + assertNotNull(provider.getCluster(endpoint2.getHostAndPort())); + } + + @Test + void testAddDuplicateCluster() { + ClusterConfig duplicateConfig = createClusterConfig(endpoint1.getHostAndPort(), 2.0f); + + // Should throw validation exception for duplicate endpoint + assertThrows(JedisValidationException.class, () -> provider.add(duplicateConfig)); + } + + @Test + void testAddNullClusterConfig() { + // Should throw validation exception for null config + assertThrows(JedisValidationException.class, () -> provider.add(null)); + } + + @Test + void testRemoveExistingCluster() { + Connection mockConnection = mock(Connection.class); + when(mockConnection.ping()).thenReturn(true); + + try (MockedConstruction mockedPool = mockPool(mockConnection)) { + // Create initial provider with endpoint1 + ClusterConfig clusterConfig1 = createClusterConfig(endpoint1.getHostAndPort(), 1.0f); + + MultiClusterClientConfig multiConfig = MultiClusterClientConfig + .builder(new ClusterConfig[] { clusterConfig1 }).build(); + + MultiClusterPooledConnectionProvider providerWithMockedPool = new MultiClusterPooledConnectionProvider( + multiConfig); + + // Add endpoint2 as second cluster + ClusterConfig newConfig = createClusterConfig(endpoint2.getHostAndPort(), 2.0f); + providerWithMockedPool.add(newConfig); + + // Now remove endpoint1 (original cluster) + assertDoesNotThrow(() -> providerWithMockedPool.remove(endpoint1.getHostAndPort())); + + // Verify endpoint1 was removed + assertNull(providerWithMockedPool.getCluster(endpoint1.getHostAndPort())); + // Verify endpoint2 still exists + assertNotNull(providerWithMockedPool.getCluster(endpoint2.getHostAndPort())); + } + } + + private MockedConstruction mockPool(Connection mockConnection) { + return mockConstruction(ConnectionPool.class, (mock, context) -> { + when(mock.getResource()).thenReturn(mockConnection); + doNothing().when(mock).close(); + }); + } + + @Test + void testRemoveNonExistentCluster() { + HostAndPort nonExistentEndpoint = new HostAndPort("localhost", 9999); + + // Should throw validation exception for non-existent endpoint + assertThrows(JedisValidationException.class, () -> provider.remove(nonExistentEndpoint)); + } + + @Test + void testRemoveLastRemainingCluster() { + // Should throw validation exception when trying to remove the last cluster + assertThrows(JedisValidationException.class, () -> provider.remove(endpoint1.getHostAndPort())); + } + + @Test + void testRemoveNullEndpoint() { + // Should throw validation exception for null endpoint + assertThrows(JedisValidationException.class, () -> provider.remove(null)); + } + + @Test + void testAddAndRemoveMultipleClusters() { + // Add endpoint2 as second cluster + ClusterConfig config2 = createClusterConfig(endpoint2.getHostAndPort(), 2.0f); + + // Create a third endpoint for this test + HostAndPort endpoint3 = new HostAndPort("localhost", 6381); + ClusterConfig config3 = createClusterConfig(endpoint3, 3.0f); + + provider.add(config2); + provider.add(config3); + + // Verify all clusters exist + assertNotNull(provider.getCluster(endpoint1.getHostAndPort())); + assertNotNull(provider.getCluster(endpoint2.getHostAndPort())); + assertNotNull(provider.getCluster(endpoint3)); + + // Remove endpoint2 + provider.remove(endpoint2.getHostAndPort()); + + // Verify correct cluster was removed + assertNull(provider.getCluster(endpoint2.getHostAndPort())); + assertNotNull(provider.getCluster(endpoint1.getHostAndPort())); + assertNotNull(provider.getCluster(endpoint3)); + } + + @Test + void testActiveClusterHandlingOnAdd() { + // The initial cluster should be active + assertNotNull(provider.getCluster()); + + // Add endpoint2 with higher weight + ClusterConfig newConfig = createClusterConfig(endpoint2.getHostAndPort(), 5.0f); + provider.add(newConfig); + + // Active cluster should still be valid (implementation may or may not switch) + assertNotNull(provider.getCluster()); + } + + @Test + void testActiveClusterHandlingOnRemove() { + Connection mockConnection = mock(Connection.class); + when(mockConnection.ping()).thenReturn(true); + + try (MockedConstruction mockedPool = mockPool(mockConnection)) { + // Create initial provider with endpoint1 + ClusterConfig clusterConfig1 = createClusterConfig(endpoint1.getHostAndPort(), 1.0f); + + MultiClusterClientConfig multiConfig = MultiClusterClientConfig + .builder(new ClusterConfig[] { clusterConfig1 }).build(); + + MultiClusterPooledConnectionProvider providerWithMockedPool = new MultiClusterPooledConnectionProvider( + multiConfig); + + // Add endpoint2 as second cluster + ClusterConfig newConfig = createClusterConfig(endpoint2.getHostAndPort(), 2.0f); + providerWithMockedPool.add(newConfig); + + // Get current active cluster + Object initialActiveCluster = providerWithMockedPool.getCluster(); + assertNotNull(initialActiveCluster); + + // Remove endpoint1 (original cluster, might be active) + providerWithMockedPool.remove(endpoint1.getHostAndPort()); + + // Should still have an active cluster + assertNotNull(providerWithMockedPool.getCluster()); + } + } +} From ca3378df20977f1245d9881bb165dcdea23082f2 Mon Sep 17 00:00:00 2001 From: atakavci Date: Tue, 15 Jul 2025 11:55:00 +0300 Subject: [PATCH 10/23] - replace cluster disabled with failbackCandidate - replace failback enabled with failbacksupported in client - fix formatting - set defaults --- .../jedis/MultiClusterClientConfig.java | 21 +++-- .../MultiClusterPooledConnectionProvider.java | 78 ++++++++++--------- .../clients/jedis/mcf/HealthCheckTest.java | 6 +- ...tiClusterPooledConnectionProviderTest.java | 2 +- 4 files changed, 57 insertions(+), 50 deletions(-) diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 8cba817eea..9f600b4aa6 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -147,10 +147,10 @@ public static interface StrategySupplier { //////////// Failover Config //////////// /** Whether to retry failed commands during failover */ - private boolean retryOnFailover = false; + private boolean retryOnFailover; - /** Whether failback is enabled */ - private boolean failback = false; + /** Whether failback is supported by client */ + private boolean isFailbackSupported; public MultiClusterClientConfig(ClusterConfig[] clusterConfigs) { this.clusterConfigs = clusterConfigs; @@ -220,8 +220,9 @@ public boolean isRetryOnFailover() { return retryOnFailover; } - public boolean isFailback() { - return failback; + /** Whether failback is supported by client */ + public boolean isFailbackSupported() { + return isFailbackSupported; } public static Builder builder(ClusterConfig[] clusterConfigs) { @@ -288,7 +289,6 @@ public static class Builder { private float weight = 1.0f; private StrategySupplier healthCheckStrategySupplier = EchoStrategy.DEFAULT; - private boolean healthCheckEnabled = true; public Builder(HostAndPort hostAndPort, JedisClientConfig clientConfig) { this.hostAndPort = hostAndPort; @@ -322,7 +322,6 @@ public Builder healthCheckStrategy(HealthCheckStrategy healthCheckStrategy) { } public Builder healthCheckEnabled(boolean healthCheckEnabled) { - this.healthCheckEnabled = healthCheckEnabled; if (!healthCheckEnabled) { this.healthCheckStrategySupplier = null; } else if (healthCheckStrategySupplier == null) { @@ -358,7 +357,7 @@ public static class Builder { private List> fallbackExceptionList = FALLBACK_EXCEPTIONS_DEFAULT; private boolean retryOnFailover = false; - private boolean failback = false; + private boolean isFailbackSupported = true; public Builder(ClusterConfig[] clusterConfigs) { @@ -457,8 +456,8 @@ public Builder retryOnFailover(boolean retryOnFailover) { return this; } - public Builder failback(boolean failback) { - this.failback = failback; + public Builder failbackSupported(boolean supported) { + this.isFailbackSupported = supported; return this; } @@ -488,7 +487,7 @@ public MultiClusterClientConfig build() { config.fallbackExceptionList = this.fallbackExceptionList; config.retryOnFailover = this.retryOnFailover; - config.failback = this.failback; + config.isFailbackSupported = this.isFailbackSupported; return config; } diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 02ed3b33ee..02697e8548 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -234,7 +234,7 @@ public void remove(Endpoint endpoint) { private void addClusterInternal(MultiClusterClientConfig multiClusterClientConfig, ClusterConfig config) { GenericObjectPoolConfig poolConfig = config.getConnectionPoolConfig(); - String clusterId = "cluster:" + config.getHostAndPort(); + String clusterId = "cluster:" + config.getHostAndPort(); Retry retry = RetryRegistry.of(retryConfig).retry(clusterId); @@ -251,14 +251,14 @@ private void addClusterInternal(MultiClusterClientConfig multiClusterClientConfi circuitBreakerEventPublisher.onSlowCallRateExceeded(event -> log.error(String.valueOf(event))); circuitBreakerEventPublisher.onStateTransition(event -> log.warn(String.valueOf(event))); - ConnectionPool pool; - if (poolConfig != null) { - pool = new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig(), poolConfig); - } else { - pool = new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig()); - } - Cluster cluster = new Cluster(pool, retry, circuitBreaker, config.getWeight(), multiClusterClientConfig); - multiClusterMap.put(config.getHostAndPort(), cluster); + ConnectionPool pool; + if (poolConfig != null) { + pool = new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig(), poolConfig); + } else { + pool = new ConnectionPool(config.getHostAndPort(), config.getJedisClientConfig()); + } + Cluster cluster = new Cluster(pool, retry, circuitBreaker, config.getWeight(), multiClusterClientConfig); + multiClusterMap.put(config.getHostAndPort(), cluster); StrategySupplier strategySupplier = config.getHealthCheckStrategySupplier(); if (strategySupplier != null) { @@ -276,12 +276,12 @@ private void handleStatusChange(HealthStatusChangeEvent eventArgs) { Cluster clusterWithHealthChange = multiClusterMap.get(endpoint); if (clusterWithHealthChange == null) return; - if (clusterWithHealthChange.isDisabled()) return; clusterWithHealthChange.setHealthStatus(newStatus); if (newStatus.isHealthy()) { - if (clusterWithHealthChange.isFailbackEnabled() && activeCluster != clusterWithHealthChange) { + if (clusterWithHealthChange.isFailbackSupported() && clusterWithHealthChange.isFailbackCandidate() + && activeCluster != clusterWithHealthChange) { // lets check if weighted switching is possible Map.Entry failbackCluster = findWeightedHealthyClusterToIterate(); if (failbackCluster == clusterWithHealthChange @@ -304,24 +304,22 @@ public Endpoint iterateActiveCluster() { + "provided with an additional cluster/database endpoint according to its prioritized sequence. " + "If applicable, consider failing back OR restarting with an available cluster/database endpoint"); } - boolean changed = setActiveCluster(clusterToIterate.getValue(), false); - return changed ? clusterToIterate.getKey() : null; + Cluster cluster = clusterToIterate.getValue(); + boolean changed = setActiveCluster(cluster, false); + if (!changed) return null; + if (cluster.isFailbackSupported()) { + cluster.setFailbackCandidate(false); + } + return clusterToIterate.getKey(); } private static Comparator> maxByWeight = Map.Entry . comparingByValue(Comparator.comparing(Cluster::getWeight)); private static Predicate> filterByHealth = c -> c.getValue().isHealthy(); - private static Predicate> filterByFailback = c -> c.getValue().isFailbackEnabled(); - - // private Map.Entry findWeightedHealthyCluster() { - // Cluster current = activeCluster; - // return multiClusterMap.entrySet().stream().filter(filterByHealth).filter(entry -> entry.getValue() != current) - // .max(maxByWeight).orElse(null); - // } private Map.Entry findWeightedHealthyClusterToIterate() { - return multiClusterMap.entrySet().stream().filter(filterByHealth).filter(filterByFailback) + return multiClusterMap.entrySet().stream().filter(filterByHealth) .filter(entry -> entry.getValue() != activeCluster).max(maxByWeight).orElse(null); } @@ -391,9 +389,6 @@ private boolean setActiveCluster(Cluster cluster, boolean validateConnection) { log.warn("Cluster/database endpoint '{}' successfully closed its circuit breaker", originalClusterName); else log.warn("Cluster/database endpoint successfully updated from '{}' to '{}'", originalClusterName, cluster.circuitBreaker.getName()); - if (!activeCluster.isFailbackEnabled()) { - activeCluster.disable(); - } Cluster temp = activeCluster; activeCluster = cluster; return temp != activeCluster; @@ -476,7 +471,9 @@ public static class Cluster { // it starts its life with the assumption of being healthy private HealthStatus healthStatus = HealthStatus.HEALTHY; private MultiClusterClientConfig multiClusterClientConfig; - private boolean disabled = false; + // it starts its life as a failback candidate, + // this changes under the condition that cluster failover to another and failback is not supported by client + private boolean failbackCandidate = true; public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circuitBreaker, float weight, MultiClusterClientConfig multiClusterClientConfig) { @@ -487,10 +484,6 @@ public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circui this.multiClusterClientConfig = multiClusterClientConfig; } - public void disable() { - disabled = true; - } - public Connection getConnection() { return connectionPool.getResource(); } @@ -527,26 +520,41 @@ public boolean isCBForcedOpen() { } public boolean isHealthy() { - return healthStatus.isHealthy() && !isCBForcedOpen() && !disabled; + return healthStatus.isHealthy() && !isCBForcedOpen(); } - public boolean isFailbackEnabled() { - return multiClusterClientConfig.isFailback(); + /** + * Whether this cluster is a failback candidate Cluster starts its life as a failback candidate, this changes + * under the condition that cluster failover to another and failback is not supported by client + */ + public boolean isFailbackCandidate() { + return failbackCandidate; } - public boolean isDisabled() { - return disabled; + /** + * Sets this cluster as a failback candidate + */ + public void setFailbackCandidate(boolean failbackCandidate) { + this.failbackCandidate = failbackCandidate; } public boolean retryOnFailover() { return multiClusterClientConfig.isRetryOnFailover(); } + /** + * Whether failback is supported by client + */ + public boolean isFailbackSupported() { + return multiClusterClientConfig.isFailbackSupported(); + } + @Override public String toString() { return circuitBreaker.getName() + "{" + "connectionPool=" + connectionPool + ", retry=" + retry + ", circuitBreaker=" + circuitBreaker + ", weight=" + weight + ", healthStatus=" + healthStatus - + ", multiClusterClientConfig=" + multiClusterClientConfig + ", disabled=" + disabled + '}'; + + ", multiClusterClientConfig=" + multiClusterClientConfig + ", failbackCandidate=" + failbackCandidate + + '}'; } } diff --git a/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java b/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java index 0678f365e5..023c1b62f2 100644 --- a/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java +++ b/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java @@ -281,12 +281,12 @@ void testNewFieldLocations() { MultiClusterClientConfig multiConfig = new MultiClusterClientConfig.Builder(new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) .retryOnFailover(true) - .failback(true) + .failbackSupported(false) .build(); assertEquals(2.5f, clusterConfig.getWeight()); assertTrue(multiConfig.isRetryOnFailover()); - assertTrue(multiConfig.isFailback()); + assertFalse(multiConfig.isFailbackSupported()); } @Test @@ -304,7 +304,7 @@ void testDefaultValues() { .build(); assertFalse(multiConfig.isRetryOnFailover()); // Default is false - assertFalse(multiConfig.isFailback()); // Default is false + assertTrue(multiConfig.isFailbackSupported()); // Default is true } @Test diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java index cf9733e3e4..5c64542c85 100644 --- a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java @@ -36,7 +36,7 @@ public void setUp() { .weight(0.3f).build(); provider = new MultiClusterPooledConnectionProvider( - new MultiClusterClientConfig.Builder(clusterConfigs).build()); + new MultiClusterClientConfig.Builder(clusterConfigs).failbackSupported(false).build()); } @Test From ddcec73baa03bc63a32b51c13742b48df9fc88fd Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 16 Jul 2025 08:48:35 +0300 Subject: [PATCH 11/23] - remove failback candidate - fix failing tests --- .../MultiClusterPooledConnectionProvider.java | 34 ++++++------------- ...tiClusterPooledConnectionProviderTest.java | 27 ++++++--------- 2 files changed, 20 insertions(+), 41 deletions(-) diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 02697e8548..31da53b79d 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -280,8 +280,7 @@ private void handleStatusChange(HealthStatusChangeEvent eventArgs) { clusterWithHealthChange.setHealthStatus(newStatus); if (newStatus.isHealthy()) { - if (clusterWithHealthChange.isFailbackSupported() && clusterWithHealthChange.isFailbackCandidate() - && activeCluster != clusterWithHealthChange) { + if (clusterWithHealthChange.isFailbackSupported() && activeCluster != clusterWithHealthChange) { // lets check if weighted switching is possible Map.Entry failbackCluster = findWeightedHealthyClusterToIterate(); if (failbackCluster == clusterWithHealthChange @@ -307,9 +306,6 @@ public Endpoint iterateActiveCluster() { Cluster cluster = clusterToIterate.getValue(); boolean changed = setActiveCluster(cluster, false); if (!changed) return null; - if (cluster.isFailbackSupported()) { - cluster.setFailbackCandidate(false); - } return clusterToIterate.getKey(); } @@ -471,9 +467,7 @@ public static class Cluster { // it starts its life with the assumption of being healthy private HealthStatus healthStatus = HealthStatus.HEALTHY; private MultiClusterClientConfig multiClusterClientConfig; - // it starts its life as a failback candidate, - // this changes under the condition that cluster failover to another and failback is not supported by client - private boolean failbackCandidate = true; + private boolean disabled = false; public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circuitBreaker, float weight, MultiClusterClientConfig multiClusterClientConfig) { @@ -520,26 +514,19 @@ public boolean isCBForcedOpen() { } public boolean isHealthy() { - return healthStatus.isHealthy() && !isCBForcedOpen(); + return healthStatus.isHealthy() && !isCBForcedOpen() && !disabled; } - /** - * Whether this cluster is a failback candidate Cluster starts its life as a failback candidate, this changes - * under the condition that cluster failover to another and failback is not supported by client - */ - public boolean isFailbackCandidate() { - return failbackCandidate; + public boolean retryOnFailover() { + return multiClusterClientConfig.isRetryOnFailover(); } - /** - * Sets this cluster as a failback candidate - */ - public void setFailbackCandidate(boolean failbackCandidate) { - this.failbackCandidate = failbackCandidate; + public boolean isDisabled() { + return disabled; } - public boolean retryOnFailover() { - return multiClusterClientConfig.isRetryOnFailover(); + public void setDisabled(boolean disabled) { + this.disabled = disabled; } /** @@ -553,8 +540,7 @@ public boolean isFailbackSupported() { public String toString() { return circuitBreaker.getName() + "{" + "connectionPool=" + connectionPool + ", retry=" + retry + ", circuitBreaker=" + circuitBreaker + ", weight=" + weight + ", healthStatus=" + healthStatus - + ", multiClusterClientConfig=" + multiClusterClientConfig + ", failbackCandidate=" + failbackCandidate - + '}'; + + ", multiClusterClientConfig=" + multiClusterClientConfig + '}'; } } diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java index 5c64542c85..4c9fd52b06 100644 --- a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java @@ -9,7 +9,6 @@ import redis.clients.jedis.exceptions.JedisValidationException; import redis.clients.jedis.mcf.Endpoint; - import java.util.concurrent.atomic.AtomicBoolean; import static org.junit.jupiter.api.Assertions.*; @@ -36,7 +35,7 @@ public void setUp() { .weight(0.3f).build(); provider = new MultiClusterPooledConnectionProvider( - new MultiClusterClientConfig.Builder(clusterConfigs).failbackSupported(false).build()); + new MultiClusterClientConfig.Builder(clusterConfigs).build()); } @Test @@ -56,16 +55,19 @@ public void testCircuitBreakerForcedTransitions() { } @Test - public void testIncrementActiveMultiClusterIndex() { + public void testIterateActiveCluster() { Endpoint e2 = provider.iterateActiveCluster(); assertEquals(endpointStandalone1.getHostAndPort(), e2); } @Test - public void testIncrementActiveMultiClusterIndexOutOfRange() { + public void testIterateActiveClusterOutOfRange() { provider.setActiveCluster(endpointStandalone0.getHostAndPort()); + provider.getCluster().setDisabled(true); Endpoint e2 = provider.iterateActiveCluster(); + provider.getCluster().setDisabled(true); + assertEquals(endpointStandalone1.getHostAndPort(), e2); assertThrows(JedisConnectionException.class, () -> provider.iterateActiveCluster()); // Should throw an @@ -73,20 +75,11 @@ public void testIncrementActiveMultiClusterIndexOutOfRange() { } @Test - public void testIsLastClusterCircuitBreakerForcedOpen() { + public void testCanIterateOnceMore() { provider.setActiveCluster(endpointStandalone0.getHostAndPort()); - - try { - provider.iterateActiveCluster(); - } catch (Exception e) { - } - - // This should set the isLastClusterCircuitBreakerForcedOpen to true - try { - provider.iterateActiveCluster(); - } catch (Exception e) { - } - + provider.getCluster().setDisabled(true); + provider.iterateActiveCluster(); + assertFalse(provider.canIterateOnceMore()); } From c1b6d5f7a7f9696a19513c95e2e561181adf91e1 Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 16 Jul 2025 13:21:08 +0300 Subject: [PATCH 12/23] - fix remove logic - fix failing tests --- .../MultiClusterPooledConnectionProvider.java | 16 +++--- .../MultiClusterDynamicEndpointUnitTest.java | 52 ++++++++++--------- 2 files changed, 36 insertions(+), 32 deletions(-) diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 31da53b79d..c271d3d7fb 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -28,6 +28,7 @@ import redis.clients.jedis.annots.Experimental; import redis.clients.jedis.annots.VisibleForTesting; import redis.clients.jedis.exceptions.JedisConnectionException; +import redis.clients.jedis.exceptions.JedisException; import redis.clients.jedis.exceptions.JedisValidationException; import redis.clients.jedis.mcf.HealthStatus; import redis.clients.jedis.mcf.HealthStatusChangeEvent; @@ -201,13 +202,13 @@ public void remove(Endpoint endpoint) { if (isActiveCluster) { log.info("Active cluster is being removed. Finding a new active cluster..."); - - // If we removed the active cluster, find a new one - if (isActiveCluster) { - Map.Entry candidateCluster = findWeightedHealthyClusterToIterate(); - if (candidateCluster != null) { - setActiveCluster(candidateCluster.getValue(), true); - } + Map.Entry candidateCluster = findWeightedHealthyClusterToIterate(); + if (candidateCluster != null) { + setActiveCluster(candidateCluster.getValue(), true); + log.info("New active cluster set to {}", candidateCluster.getKey()); + } else { + throw new JedisException( + "Cluster can not be removed due to no healthy cluster available to switch!"); } } @@ -220,6 +221,7 @@ public void remove(Endpoint endpoint) { // Close the cluster resources if (clusterToRemove != null) { + clusterToRemove.setDisabled(true); clusterToRemove.getConnectionPool().close(); } } finally { diff --git a/src/test/java/redis/clients/jedis/mcf/MultiClusterDynamicEndpointUnitTest.java b/src/test/java/redis/clients/jedis/mcf/MultiClusterDynamicEndpointUnitTest.java index 420c578588..7c05b911e8 100644 --- a/src/test/java/redis/clients/jedis/mcf/MultiClusterDynamicEndpointUnitTest.java +++ b/src/test/java/redis/clients/jedis/mcf/MultiClusterDynamicEndpointUnitTest.java @@ -44,8 +44,8 @@ void setUp() { // Helper method to create cluster configurations private ClusterConfig createClusterConfig(HostAndPort hostAndPort, float weight) { - return ClusterConfig.builder(hostAndPort, clientConfig) - .weight(weight).build(); + // Disable health check for unit tests to avoid real connections + return ClusterConfig.builder(hostAndPort, clientConfig).weight(weight).healthCheckEnabled(false).build(); } @Test @@ -85,20 +85,21 @@ void testRemoveExistingCluster() { MultiClusterClientConfig multiConfig = MultiClusterClientConfig .builder(new ClusterConfig[] { clusterConfig1 }).build(); - MultiClusterPooledConnectionProvider providerWithMockedPool = new MultiClusterPooledConnectionProvider( - multiConfig); + try (MultiClusterPooledConnectionProvider providerWithMockedPool = new MultiClusterPooledConnectionProvider( + multiConfig)) { - // Add endpoint2 as second cluster - ClusterConfig newConfig = createClusterConfig(endpoint2.getHostAndPort(), 2.0f); - providerWithMockedPool.add(newConfig); + // Add endpoint2 as second cluster + ClusterConfig newConfig = createClusterConfig(endpoint2.getHostAndPort(), 2.0f); + providerWithMockedPool.add(newConfig); - // Now remove endpoint1 (original cluster) - assertDoesNotThrow(() -> providerWithMockedPool.remove(endpoint1.getHostAndPort())); + // Now remove endpoint1 (original cluster) + assertDoesNotThrow(() -> providerWithMockedPool.remove(endpoint1.getHostAndPort())); - // Verify endpoint1 was removed - assertNull(providerWithMockedPool.getCluster(endpoint1.getHostAndPort())); - // Verify endpoint2 still exists - assertNotNull(providerWithMockedPool.getCluster(endpoint2.getHostAndPort())); + // Verify endpoint1 was removed + assertNull(providerWithMockedPool.getCluster(endpoint1.getHostAndPort())); + // Verify endpoint2 still exists + assertNotNull(providerWithMockedPool.getCluster(endpoint2.getHostAndPort())); + } } } @@ -180,22 +181,23 @@ void testActiveClusterHandlingOnRemove() { MultiClusterClientConfig multiConfig = MultiClusterClientConfig .builder(new ClusterConfig[] { clusterConfig1 }).build(); - MultiClusterPooledConnectionProvider providerWithMockedPool = new MultiClusterPooledConnectionProvider( - multiConfig); + try (MultiClusterPooledConnectionProvider providerWithMockedPool = new MultiClusterPooledConnectionProvider( + multiConfig)) { - // Add endpoint2 as second cluster - ClusterConfig newConfig = createClusterConfig(endpoint2.getHostAndPort(), 2.0f); - providerWithMockedPool.add(newConfig); + // Add endpoint2 as second cluster + ClusterConfig newConfig = createClusterConfig(endpoint2.getHostAndPort(), 2.0f); + providerWithMockedPool.add(newConfig); - // Get current active cluster - Object initialActiveCluster = providerWithMockedPool.getCluster(); - assertNotNull(initialActiveCluster); + // Get current active cluster + Object initialActiveCluster = providerWithMockedPool.getCluster(); + assertNotNull(initialActiveCluster); - // Remove endpoint1 (original cluster, might be active) - providerWithMockedPool.remove(endpoint1.getHostAndPort()); + // Remove endpoint1 (original cluster, might be active) + providerWithMockedPool.remove(endpoint1.getHostAndPort()); - // Should still have an active cluster - assertNotNull(providerWithMockedPool.getCluster()); + // Should still have an active cluster + assertNotNull(providerWithMockedPool.getCluster()); + } } } } From ff163302c9390484d0b4e6c0bf4792bd418e1915 Mon Sep 17 00:00:00 2001 From: atakavci Date: Thu, 17 Jul 2025 18:10:39 +0300 Subject: [PATCH 13/23] - periodic failback checks - introduce graceperiod - fix issue when CB is forced_open and gracePeriod is completed --- .../jedis/MultiClusterClientConfig.java | 31 ++ .../jedis/mcf/CircuitBreakerFailoverBase.java | 17 +- .../MultiClusterPooledConnectionProvider.java | 117 +++++- .../mcf/FailbackMechanismIntegrationTest.java | 379 ++++++++++++++++++ .../jedis/mcf/FailbackMechanismUnitTest.java | 184 +++++++++ .../jedis/mcf/PeriodicFailbackTest.java | 245 +++++++++++ ...tiClusterPooledConnectionProviderTest.java | 2 +- 7 files changed, 951 insertions(+), 24 deletions(-) create mode 100644 src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java create mode 100644 src/test/java/redis/clients/jedis/mcf/FailbackMechanismUnitTest.java create mode 100644 src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 9f600b4aa6..8c94c591fd 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -63,6 +63,9 @@ public static interface StrategySupplier { private static final List> FALLBACK_EXCEPTIONS_DEFAULT = Arrays .asList(CallNotPermittedException.class, ConnectionFailoverException.class); + private static final long FAILBACK_CHECK_INTERVAL_DEFAULT = 5000; // 5 seconds + private static final long GRACE_PERIOD_DEFAULT = 10000; // 10 seconds + private final ClusterConfig[] clusterConfigs; //////////// Retry Config - https://resilience4j.readme.io/docs/retry //////////// @@ -152,6 +155,12 @@ public static interface StrategySupplier { /** Whether failback is supported by client */ private boolean isFailbackSupported; + /** Interval in milliseconds to wait before attempting failback to a recovered cluster */ + private long failbackCheckInterval; + + /** Grace period in milliseconds to keep clusters disabled after they become unhealthy */ + private long gracePeriod; + public MultiClusterClientConfig(ClusterConfig[] clusterConfigs) { this.clusterConfigs = clusterConfigs; } @@ -225,6 +234,14 @@ public boolean isFailbackSupported() { return isFailbackSupported; } + public long getFailbackCheckInterval() { + return failbackCheckInterval; + } + + public long getGracePeriod() { + return gracePeriod; + } + public static Builder builder(ClusterConfig[] clusterConfigs) { return new Builder(clusterConfigs); } @@ -358,6 +375,8 @@ public static class Builder { private boolean retryOnFailover = false; private boolean isFailbackSupported = true; + private long failbackCheckInterval = FAILBACK_CHECK_INTERVAL_DEFAULT; + private long gracePeriod = GRACE_PERIOD_DEFAULT; public Builder(ClusterConfig[] clusterConfigs) { @@ -461,6 +480,16 @@ public Builder failbackSupported(boolean supported) { return this; } + public Builder failbackCheckInterval(long failbackCheckInterval) { + this.failbackCheckInterval = failbackCheckInterval; + return this; + } + + public Builder gracePeriod(long gracePeriod) { + this.gracePeriod = gracePeriod; + return this; + } + public MultiClusterClientConfig build() { MultiClusterClientConfig config = new MultiClusterClientConfig(this.clusterConfigs); @@ -488,6 +517,8 @@ public MultiClusterClientConfig build() { config.retryOnFailover = this.retryOnFailover; config.isFailbackSupported = this.isFailbackSupported; + config.failbackCheckInterval = this.failbackCheckInterval; + config.gracePeriod = this.gracePeriod; return config; } diff --git a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java index f4826a53d6..6bf51b4b56 100644 --- a/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java +++ b/src/main/java/redis/clients/jedis/mcf/CircuitBreakerFailoverBase.java @@ -6,6 +6,7 @@ import redis.clients.jedis.annots.Experimental; import redis.clients.jedis.exceptions.JedisConnectionException; import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider.Cluster; import redis.clients.jedis.util.IOUtils; /** @@ -46,16 +47,24 @@ protected void clusterFailover(CircuitBreaker circuitBreaker) { // Transitions state machine to a FORCED_OPEN state, stopping state transition, metrics and // event publishing. // To recover/transition from this forced state the user will need to manually failback + + Cluster activeCluster = provider.getCluster(); + // This should never happen in theory !! + if (activeCluster.getCircuitBreaker() != circuitBreaker) throw new IllegalStateException( + "A circuitbreaker failover can be triggered only by the active cluster!"); + + activeCluster.setGracePeriod(); circuitBreaker.transitionToForcedOpenState(); // Iterating the active cluster will allow subsequent calls to the executeCommand() to use the next // cluster's connection pool - according to the configuration's prioritization/order/weight // int activeMultiClusterIndex = provider.incrementActiveMultiClusterIndex1(); - provider.iterateActiveCluster(); + if (provider.iterateActiveCluster() != null) { - // Implementation is optionally provided during configuration. Typically, used for - // activeMultiClusterIndex persistence or custom logging - provider.runClusterFailoverPostProcessor(provider.getCluster()); + // Implementation is optionally provided during configuration. Typically, used for + // activeMultiClusterIndex persistence or custom logging + provider.runClusterFailoverPostProcessor(provider.getCluster()); + } } // this check relies on the fact that many failover attempts can hit with the same CB, // only the first one will trigger a failover, and make the CB FORCED_OPEN. diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index c271d3d7fb..d26e6fa8e0 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -16,6 +16,10 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + import java.util.function.Consumer; import java.util.function.Predicate; @@ -82,6 +86,13 @@ public class MultiClusterPooledConnectionProvider implements ConnectionProvider private HealthStatusManager healthStatusManager = new HealthStatusManager(); + // Failback mechanism fields + private final ScheduledExecutorService failbackScheduler = Executors.newSingleThreadScheduledExecutor(r -> { + Thread t = new Thread(r, "failback-scheduler"); + t.setDaemon(true); + return t; + }); + // Store retry and circuit breaker configs for dynamic cluster addition/removal private RetryConfig retryConfig; private CircuitBreakerConfig circuitBreakerConfig; @@ -151,6 +162,13 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste /// --- /// this.fallbackExceptionList = multiClusterClientConfig.getFallbackExceptionList(); + + // Start periodic failback checker + if (multiClusterClientConfig.isFailbackSupported()) { + long failbackInterval = multiClusterClientConfig.getFailbackCheckInterval(); + failbackScheduler.scheduleAtFixedRate(this::periodicFailbackCheck, failbackInterval, failbackInterval, + TimeUnit.MILLISECONDS); + } } /** @@ -194,6 +212,7 @@ public void remove(Endpoint endpoint) { if (multiClusterMap.size() < 2) { throw new JedisValidationException("Cannot remove the last remaining endpoint"); } + log.debug("Removing endpoint {}", endpoint); activeClusterIndexLock.lock(); try { @@ -251,7 +270,6 @@ private void addClusterInternal(MultiClusterClientConfig multiClusterClientConfi circuitBreakerEventPublisher.onError(event -> log.error(String.valueOf(event))); circuitBreakerEventPublisher.onFailureRateExceeded(event -> log.error(String.valueOf(event))); circuitBreakerEventPublisher.onSlowCallRateExceeded(event -> log.error(String.valueOf(event))); - circuitBreakerEventPublisher.onStateTransition(event -> log.warn(String.valueOf(event))); ConnectionPool pool; if (poolConfig != null) { @@ -281,19 +299,50 @@ private void handleStatusChange(HealthStatusChangeEvent eventArgs) { clusterWithHealthChange.setHealthStatus(newStatus); - if (newStatus.isHealthy()) { - if (clusterWithHealthChange.isFailbackSupported() && activeCluster != clusterWithHealthChange) { - // lets check if weighted switching is possible - Map.Entry failbackCluster = findWeightedHealthyClusterToIterate(); - if (failbackCluster == clusterWithHealthChange - && clusterWithHealthChange.getWeight() > activeCluster.getWeight()) { - setActiveCluster(clusterWithHealthChange, false); + if (!newStatus.isHealthy()) { + // Handle failover if this was the active cluster + if (clusterWithHealthChange == activeCluster) { + clusterWithHealthChange.setGracePeriod(); + if (iterateActiveCluster() != null) { + this.runClusterFailoverPostProcessor(activeCluster); } } - } else if (clusterWithHealthChange == activeCluster) { - if (iterateActiveCluster() != null) { - this.runClusterFailoverPostProcessor(activeCluster); + } + } + + /** + * Periodic failback checker - runs at configured intervals to check for failback opportunities + */ + private void periodicFailbackCheck() { + // Find the best candidate cluster for failback + Cluster bestCandidate = null; + float bestWeight = activeCluster.getWeight(); + + for (Map.Entry entry : multiClusterMap.entrySet()) { + Cluster cluster = entry.getValue(); + + // Skip if this is already the active cluster + if (cluster == activeCluster) { + continue; + } + + // Skip if cluster is not healthy + if (!cluster.isHealthy()) { + continue; } + + // This cluster is a valid candidate + if (cluster.getWeight() > bestWeight) { + bestCandidate = cluster; + bestWeight = cluster.getWeight(); + } + } + + // Perform failback if we found a better candidate + if (bestCandidate != null) { + log.info("Performing failback from {} to {} (higher weight cluster available)", + activeCluster.getCircuitBreaker().getName(), bestCandidate.getCircuitBreaker().getName()); + setActiveCluster(bestCandidate, true); } } @@ -397,7 +446,21 @@ private boolean setActiveCluster(Cluster cluster, boolean validateConnection) { @Override public void close() { - activeCluster.getConnectionPool().close(); + // Shutdown the failback scheduler + failbackScheduler.shutdown(); + try { + if (!failbackScheduler.awaitTermination(1, TimeUnit.SECONDS)) { + failbackScheduler.shutdownNow(); + } + } catch (InterruptedException e) { + failbackScheduler.shutdownNow(); + Thread.currentThread().interrupt(); + } + + // Close all cluster connection pools + for (Cluster cluster : multiClusterMap.values()) { + cluster.getConnectionPool().close(); + } } @Override @@ -425,18 +488,14 @@ public Cluster getCluster() { } @VisibleForTesting - public Cluster getCluster(Endpoint multiClusterIndex) { - return multiClusterMap.get(multiClusterIndex); + public Cluster getCluster(Endpoint endpoint) { + return multiClusterMap.get(endpoint); } public CircuitBreaker getClusterCircuitBreaker() { return activeCluster.getCircuitBreaker(); } - public CircuitBreaker getClusterCircuitBreaker(int multiClusterIndex) { - return activeCluster.getCircuitBreaker(); - } - /** * Indicates the final cluster/database endpoint (connection pool), according to the pre-configured list provided at * startup via the MultiClusterClientConfig, is unavailable and therefore no further failover is possible. Users can @@ -471,6 +530,9 @@ public static class Cluster { private MultiClusterClientConfig multiClusterClientConfig; private boolean disabled = false; + // Grace period tracking + private volatile long gracePeriodEndsAt = 0; + public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circuitBreaker, float weight, MultiClusterClientConfig multiClusterClientConfig) { this.connectionPool = connectionPool; @@ -512,11 +574,14 @@ public float getWeight() { } public boolean isCBForcedOpen() { + if (circuitBreaker.getState() == State.FORCED_OPEN && !isInGracePeriod()) { + circuitBreaker.transitionToClosedState(); + } return circuitBreaker.getState() == CircuitBreaker.State.FORCED_OPEN; } public boolean isHealthy() { - return healthStatus.isHealthy() && !isCBForcedOpen() && !disabled; + return healthStatus.isHealthy() && !isCBForcedOpen() && !disabled && !isInGracePeriod(); } public boolean retryOnFailover() { @@ -531,6 +596,20 @@ public void setDisabled(boolean disabled) { this.disabled = disabled; } + /** + * Checks if the cluster is currently in grace period + */ + public boolean isInGracePeriod() { + return System.currentTimeMillis() < gracePeriodEndsAt; + } + + /** + * Sets the grace period for this cluster + */ + public void setGracePeriod() { + gracePeriodEndsAt = System.currentTimeMillis() + multiClusterClientConfig.getGracePeriod(); + } + /** * Whether failback is supported by client */ diff --git a/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java new file mode 100644 index 0000000000..f74eee71ab --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java @@ -0,0 +1,379 @@ +package redis.clients.jedis.mcf; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.Mockito.*; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.MockedConstruction; +import org.mockito.junit.jupiter.MockitoExtension; + +import redis.clients.jedis.Connection; +import redis.clients.jedis.ConnectionPool; +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.MultiClusterClientConfig; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; +import java.lang.reflect.Method; + +@ExtendWith(MockitoExtension.class) +class FailbackMechanismIntegrationTest { + + private HostAndPort endpoint1; + private HostAndPort endpoint2; + private HostAndPort endpoint3; + private JedisClientConfig clientConfig; + + @BeforeEach + void setUp() { + endpoint1 = new HostAndPort("localhost", 6379); + endpoint2 = new HostAndPort("localhost", 6380); + endpoint3 = new HostAndPort("localhost", 6381); + clientConfig = DefaultJedisClientConfig.builder().build(); + } + + private MockedConstruction mockPool() { + Connection mockConnection = mock(Connection.class); + lenient().when(mockConnection.ping()).thenReturn(true); + return mockConstruction(ConnectionPool.class, (mock, context) -> { + when(mock.getResource()).thenReturn(mockConnection); + doNothing().when(mock).close(); + }); + } + + /** + * Helper method to trigger health status changes using reflection + */ + private void triggerHealthStatusChange(MultiClusterPooledConnectionProvider provider, HostAndPort endpoint, + HealthStatus oldStatus, HealthStatus newStatus) { + try { + Method handleStatusChangeMethod = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("handleStatusChange", HealthStatusChangeEvent.class); + handleStatusChangeMethod.setAccessible(true); + + HealthStatusChangeEvent event = new HealthStatusChangeEvent(endpoint, oldStatus, newStatus); + handleStatusChangeMethod.invoke(provider, event); + } catch (Exception e) { + throw new RuntimeException("Failed to trigger health status change", e); + } + } + + @Test + void testFailbackDisabledDoesNotPerformFailback() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + // Create clusters with different weights + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); // Lower weight + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f) // Higher weight + .healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(false) // Disabled + .failbackCheckInterval(100) // Short interval for testing + .build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster2 should be active (highest weight) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster2 unhealthy to force failover to cluster1 + triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster1 (only healthy option) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Make cluster2 healthy again (higher weight - would normally trigger failback) + triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Wait longer than failback interval + Thread.sleep(200); + + // Should still be on cluster1 since failback is disabled + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + } + } + } + + @Test + void testFailbackToHigherWeightCluster() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + // Create clusters with different weights + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(2.0f) // Higher weight + .healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(1.0f) // Lower weight + .healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(true) + .failbackCheckInterval(100) // Short interval for testing + .gracePeriod(100) + .build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster1 should be active (highest weight) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Make cluster1 unhealthy to force failover to cluster2 + triggerHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster2 (lower weight, but only healthy option) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster1 healthy again + triggerHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Wait for failback check interval + some buffer + Thread.sleep(250); + + // Should have failed back to cluster1 (higher weight) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + } + } + } + + @Test + void testNoFailbackToLowerWeightCluster() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + // Create three clusters with different weights to properly test no failback to lower weight + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f) // Lowest weight + .healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f) // Medium weight + .healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster3 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint3, clientConfig).weight(3.0f) // Highest weight + .healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2, cluster3 }).failbackSupported(true) + .failbackCheckInterval(100).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster3 should be active (highest weight) + assertEquals(provider.getCluster(endpoint3), provider.getCluster()); + + // Make cluster3 unhealthy to force failover to cluster2 (medium weight) + triggerHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster2 (highest weight among healthy clusters) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster1 (lowest weight) healthy - this should NOT trigger failback + // since we don't failback to lower weight clusters + triggerHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Wait for failback check interval + Thread.sleep(250); + + // Should still be on cluster2 (no failback to lower weight cluster1) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + } + } + } + + @Test + void testFailbackToHigherWeightClusterImmediately() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); // Higher weight + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); // Lower weight + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(true) + .failbackCheckInterval(100).gracePeriod(50).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster1 should be active (highest weight) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Make cluster1 unhealthy to force failover to cluster2 + triggerHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster2 (only healthy option) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster1 healthy again + triggerHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Wait for failback check + Thread.sleep(150); + + // Should have failed back to cluster1 immediately (higher weight, no stability period required) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + } + } + } + + @Test + void testUnhealthyClusterCancelsFailback() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); // Higher weight + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); // Lower weight + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(true) + .failbackCheckInterval(200).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster1 should be active (highest weight) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Make cluster1 unhealthy to force failover to cluster2 + triggerHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster2 (only healthy option) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster1 healthy again (should trigger failback attempt) + triggerHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Wait a bit + Thread.sleep(100); + + // Make cluster1 unhealthy again before failback completes + triggerHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Wait past the original failback interval + Thread.sleep(150); + + // Should still be on cluster2 (failback was cancelled due to cluster1 becoming unhealthy) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + } + } + } + + @Test + void testMultipleClusterFailbackPriority() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); // Lowest weight + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); // Medium weight + + MultiClusterClientConfig.ClusterConfig cluster3 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint3, clientConfig).weight(3.0f) // Highest weight + .healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2, cluster3 }).failbackSupported(true) + .failbackCheckInterval(100).gracePeriod(100).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster3 should be active (highest weight) + assertEquals(provider.getCluster(endpoint3), provider.getCluster()); + + // Make cluster3 unhealthy to force failover to cluster2 (next highest weight) + triggerHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster2 (highest weight among healthy clusters) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster3 healthy again + triggerHealthStatusChange(provider, endpoint3, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Wait for failback + Thread.sleep(200); + + // Should fail back to cluster3 (highest weight) + assertEquals(provider.getCluster(endpoint3), provider.getCluster()); + } + } + } + + @Test + void testGracePeriodDisablesClusterOnUnhealthy() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); // Lower weight + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); // Higher weight + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(true) + .failbackCheckInterval(100).gracePeriod(200) // 200ms grace period + .build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster2 should be active (highest weight) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Now make cluster2 unhealthy - it should be disabled for grace period + triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should failover to cluster1 + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Cluster2 should be in grace period + assertTrue(provider.getCluster(endpoint2).isInGracePeriod()); + } + } + } + + @Test + void testGracePeriodReEnablesClusterAfterPeriod() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); // Lower weight + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); // Higher weight + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(true) + .failbackCheckInterval(50) // Short interval for testing + .gracePeriod(100) // Short grace period for testing + .build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster2 should be active (highest weight) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster2 unhealthy to start grace period and force failover + triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should failover to cluster1 + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Cluster2 should be in grace period + assertTrue(provider.getCluster(endpoint2).isInGracePeriod()); + + // Make cluster2 healthy again while it's still in grace period + triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Should still be on cluster1 because cluster2 is in grace period + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Wait for grace period to expire + Thread.sleep(150); + + // Cluster2 should no longer be in grace period + assertFalse(provider.getCluster(endpoint2).isInGracePeriod()); + + // Wait for failback check to run + Thread.sleep(100); + + // Should now failback to cluster2 (higher weight) since grace period has expired + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + } + } + } + + +} diff --git a/src/test/java/redis/clients/jedis/mcf/FailbackMechanismUnitTest.java b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismUnitTest.java new file mode 100644 index 0000000000..535cdde2b9 --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismUnitTest.java @@ -0,0 +1,184 @@ +package redis.clients.jedis.mcf; + +import static org.junit.jupiter.api.Assertions.*; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.MultiClusterClientConfig; + +@ExtendWith(MockitoExtension.class) +class FailbackMechanismUnitTest { + + private HostAndPort endpoint1; + private JedisClientConfig clientConfig; + + @BeforeEach + void setUp() { + endpoint1 = new HostAndPort("localhost", 6379); + clientConfig = DefaultJedisClientConfig.builder().build(); + } + + @Test + void testFailbackCheckIntervalConfiguration() { + // Test default value + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .healthCheckEnabled(false) + .build(); + + MultiClusterClientConfig defaultConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .build(); + + assertEquals(5000, defaultConfig.getFailbackCheckInterval()); + + // Test custom value + MultiClusterClientConfig customConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .failbackCheckInterval(3000) + .build(); + + assertEquals(3000, customConfig.getFailbackCheckInterval()); + } + + @Test + void testFailbackSupportedConfiguration() { + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .healthCheckEnabled(false) + .build(); + + // Test default (should be true) + MultiClusterClientConfig defaultConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .build(); + + assertTrue(defaultConfig.isFailbackSupported()); + + // Test disabled + MultiClusterClientConfig disabledConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .failbackSupported(false) + .build(); + + assertFalse(disabledConfig.isFailbackSupported()); + } + + @Test + void testFailbackCheckIntervalValidation() { + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .healthCheckEnabled(false) + .build(); + + // Test zero interval (should be allowed) + MultiClusterClientConfig zeroConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .failbackCheckInterval(0) + .build(); + + assertEquals(0, zeroConfig.getFailbackCheckInterval()); + + // Test negative interval (should be allowed - implementation decision) + MultiClusterClientConfig negativeConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .failbackCheckInterval(-1000) + .build(); + + assertEquals(-1000, negativeConfig.getFailbackCheckInterval()); + } + + @Test + void testBuilderChaining() { + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .healthCheckEnabled(false) + .build(); + + // Test that builder methods can be chained + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .failbackSupported(true) + .failbackCheckInterval(2000) + .retryOnFailover(true) + .build(); + + assertTrue(config.isFailbackSupported()); + assertEquals(2000, config.getFailbackCheckInterval()); + assertTrue(config.isRetryOnFailover()); + } + + @Test + void testGracePeriodConfiguration() { + // Test default value + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .healthCheckEnabled(false) + .build(); + + MultiClusterClientConfig defaultConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .build(); + + assertEquals(10000, defaultConfig.getGracePeriod()); // Default is 10 seconds + + // Test custom value + MultiClusterClientConfig customConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .gracePeriod(5000) + .build(); + + assertEquals(5000, customConfig.getGracePeriod()); + } + + @Test + void testGracePeriodValidation() { + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .healthCheckEnabled(false) + .build(); + + // Test zero grace period (should be allowed) + MultiClusterClientConfig zeroConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .gracePeriod(0) + .build(); + + assertEquals(0, zeroConfig.getGracePeriod()); + + // Test negative grace period (should be allowed - implementation decision) + MultiClusterClientConfig negativeConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .gracePeriod(-1000) + .build(); + + assertEquals(-1000, negativeConfig.getGracePeriod()); + } + + @Test + void testGracePeriodBuilderChaining() { + MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .healthCheckEnabled(false) + .build(); + + // Test that builder methods can be chained + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) + .failbackSupported(true) + .failbackCheckInterval(2000) + .gracePeriod(8000) + .retryOnFailover(true) + .build(); + + assertTrue(config.isFailbackSupported()); + assertEquals(2000, config.getFailbackCheckInterval()); + assertEquals(8000, config.getGracePeriod()); + assertTrue(config.isRetryOnFailover()); + } +} diff --git a/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java b/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java new file mode 100644 index 0000000000..0d61aa9fa8 --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java @@ -0,0 +1,245 @@ +package redis.clients.jedis.mcf; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.Mockito.*; + +import java.lang.reflect.Method; + +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.MockedConstruction; +import org.mockito.junit.jupiter.MockitoExtension; + +import redis.clients.jedis.Connection; +import redis.clients.jedis.ConnectionPool; +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.MultiClusterClientConfig; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; +import redis.clients.jedis.util.Pool; + +@ExtendWith(MockitoExtension.class) +class PeriodicFailbackTest { + + private HostAndPort endpoint1; + private HostAndPort endpoint2; + private JedisClientConfig clientConfig; + + @BeforeEach + void setUp() { + endpoint1 = new HostAndPort("localhost", 6379); + endpoint2 = new HostAndPort("localhost", 6380); + clientConfig = DefaultJedisClientConfig.builder().build(); + } + + private MockedConstruction mockPool() { + Connection mockConnection = mock(Connection.class); + lenient().when(mockConnection.ping()).thenReturn(true); + return mockConstruction(ConnectionPool.class, (mock, context) -> { + when(mock.getResource()).thenReturn(mockConnection); + doNothing().when(mock).close(); + }); + } + + /** + * Helper method to trigger health status changes using reflection + */ + private void triggerHealthStatusChange(MultiClusterPooledConnectionProvider provider, HostAndPort endpoint, + HealthStatus oldStatus, HealthStatus newStatus) { + try { + Method handleStatusChangeMethod = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("handleStatusChange", HealthStatusChangeEvent.class); + handleStatusChangeMethod.setAccessible(true); + + HealthStatusChangeEvent event = new HealthStatusChangeEvent(endpoint, oldStatus, newStatus); + handleStatusChangeMethod.invoke(provider, event); + } catch (Exception e) { + throw new RuntimeException("Failed to trigger health status change", e); + } + } + + /** + * Helper method to trigger periodic failback check using reflection + */ + private void triggerPeriodicFailbackCheck(MultiClusterPooledConnectionProvider provider) { + try { + Method periodicFailbackCheckMethod = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("periodicFailbackCheck"); + periodicFailbackCheckMethod.setAccessible(true); + + periodicFailbackCheckMethod.invoke(provider); + } catch (Exception e) { + throw new RuntimeException("Failed to trigger periodic failback check", e); + } + } + + @Test + void testPeriodicFailbackCheckWithDisabledCluster() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(true) + .failbackCheckInterval(100).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster2 should be active (highest weight: 2.0f vs 1.0f) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Start grace period for cluster2 manually + provider.getCluster(endpoint2).setGracePeriod(); + provider.getCluster(endpoint2).setDisabled(true); + + // Force failover to cluster1 since cluster2 is disabled + provider.iterateActiveCluster(); + + // Manually trigger periodic check + triggerPeriodicFailbackCheck(provider); + + // Should still be on cluster1 (cluster2 is in grace period) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + } + } + } + + @Test + void testPeriodicFailbackCheckWithHealthyCluster() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(true) + .failbackCheckInterval(50).gracePeriod(100).build(); // Add grace period + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster2 should be active (highest weight: 2.0f vs 1.0f) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster2 unhealthy to force failover to cluster1 + triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster1 (cluster2 is in grace period) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Verify cluster2 is in grace period + assertTrue(provider.getCluster(endpoint2).isInGracePeriod()); + + // Make cluster2 healthy again (but it's still in grace period) + triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Trigger periodic check immediately - should still be on cluster1 + triggerPeriodicFailbackCheck(provider); + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Wait for grace period to expire + Thread.sleep(150); + + // Trigger periodic check after grace period expires + triggerPeriodicFailbackCheck(provider); + + // Should have failed back to cluster2 (higher weight, grace period expired) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + } + } + } + + @Test + void testPeriodicFailbackCheckWithFailbackDisabled() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).failbackSupported(false) // Disabled + .failbackCheckInterval(50).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster2 should be active (highest weight: 2.0f vs 1.0f) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster2 unhealthy to force failover to cluster1 + triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster1 + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Make cluster2 healthy again + triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Wait for stability period + Thread.sleep(100); + + // Trigger periodic check + triggerPeriodicFailbackCheck(provider); + + // Should still be on cluster1 (failback disabled) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + } + } + } + + @Test + void testPeriodicFailbackCheckSelectsHighestWeightCluster() throws InterruptedException { + try (MockedConstruction mockedPool = mockPool()) { + HostAndPort endpoint3 = new HostAndPort("localhost", 6381); + + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster3 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint3, clientConfig).weight(3.0f) // Highest weight + .healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2, cluster3 }).failbackSupported(true) + .failbackCheckInterval(50).gracePeriod(100).build(); // Add grace period + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Initially, cluster3 should be active (highest weight: 3.0f vs 2.0f vs 1.0f) + assertEquals(provider.getCluster(endpoint3), provider.getCluster()); + + // Make cluster3 unhealthy to force failover to cluster2 (next highest weight) + triggerHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster2 (weight 2.0f, higher than cluster1's 1.0f) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + + // Make cluster2 unhealthy to force failover to cluster1 + triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Should now be on cluster1 (only healthy cluster left) + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + + // Make cluster2 and cluster3 healthy again + triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + triggerHealthStatusChange(provider, endpoint3, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + + // Wait for grace period to expire + Thread.sleep(150); + + // Trigger periodic check + triggerPeriodicFailbackCheck(provider); + + // Should have failed back to cluster3 (highest weight, grace period expired) + assertEquals(provider.getCluster(endpoint3), provider.getCluster()); + } + } + } +} diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java index 4c9fd52b06..8263a595bf 100644 --- a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java @@ -41,7 +41,7 @@ public void setUp() { @Test public void testCircuitBreakerForcedTransitions() { - CircuitBreaker circuitBreaker = provider.getClusterCircuitBreaker(1); + CircuitBreaker circuitBreaker = provider.getClusterCircuitBreaker(); circuitBreaker.getState(); if (CircuitBreaker.State.FORCED_OPEN.equals(circuitBreaker.getState())) From c39fda1472e80fd85a51bdfbe02dbc8d7e18df95 Mon Sep 17 00:00:00 2001 From: atakavci Date: Sat, 19 Jul 2025 14:07:25 +0300 Subject: [PATCH 14/23] - introduce StatusTracker with purpose of waiting initial healthcheck results during consturction of provider - add HealthStatus.UNKNOWN as default for Cluster - handle status changes in order of events during initialization - add tests for status tracker and orderingof events - fix impacted unit&integ tests --- .../jedis/MultiClusterClientConfig.java | 6 + .../redis/clients/jedis/mcf/HealthCheck.java | 2 +- .../redis/clients/jedis/mcf/HealthStatus.java | 2 +- .../jedis/mcf/HealthStatusManager.java | 6 +- .../clients/jedis/mcf/StatusTracker.java | 71 ++++ .../MultiClusterPooledConnectionProvider.java | 125 ++++++- .../mcf/FailbackMechanismIntegrationTest.java | 8 +- .../jedis/mcf/HealthCheckIntegrationTest.java | 3 +- .../clients/jedis/mcf/HealthCheckTest.java | 91 ++--- .../mcf/HealthStatusEventOrderingTest.java | 343 ++++++++++++++++++ .../mcf/MultiClusterInitializationTest.java | 175 +++++++++ .../jedis/mcf/PeriodicFailbackTest.java | 9 +- .../clients/jedis/mcf/StatusTrackerTest.java | 243 +++++++++++++ 13 files changed, 1007 insertions(+), 77 deletions(-) create mode 100644 src/main/java/redis/clients/jedis/mcf/StatusTracker.java create mode 100644 src/test/java/redis/clients/jedis/mcf/HealthStatusEventOrderingTest.java create mode 100644 src/test/java/redis/clients/jedis/mcf/MultiClusterInitializationTest.java create mode 100644 src/test/java/redis/clients/jedis/mcf/StatusTrackerTest.java diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 8c94c591fd..35a35890c2 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -162,6 +162,12 @@ public static interface StrategySupplier { private long gracePeriod; public MultiClusterClientConfig(ClusterConfig[] clusterConfigs) { + if (clusterConfigs == null || clusterConfigs.length < 1) throw new JedisValidationException( + "ClusterClientConfigs are required for MultiClusterPooledConnectionProvider"); + for (ClusterConfig clusterConfig : clusterConfigs) { + if (clusterConfig == null) throw new IllegalArgumentException( + "ClusterClientConfigs must not contain null elements"); + } this.clusterConfigs = clusterConfigs; } diff --git a/src/main/java/redis/clients/jedis/mcf/HealthCheck.java b/src/main/java/redis/clients/jedis/mcf/HealthCheck.java index e23e0e5810..6b2e91fc11 100644 --- a/src/main/java/redis/clients/jedis/mcf/HealthCheck.java +++ b/src/main/java/redis/clients/jedis/mcf/HealthCheck.java @@ -27,7 +27,7 @@ public class HealthCheck { this.endpoint = endpoint; this.strategy = strategy; this.statusChangeCallback = statusChangeCallback; - statusRef.set(new SimpleEntry<>(0L, HealthStatus.HEALTHY)); + statusRef.set(new SimpleEntry<>(0L, HealthStatus.UNKNOWN)); } public Endpoint getEndpoint() { diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatus.java b/src/main/java/redis/clients/jedis/mcf/HealthStatus.java index 9e5ba4bbcd..2620f4131e 100644 --- a/src/main/java/redis/clients/jedis/mcf/HealthStatus.java +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatus.java @@ -1,7 +1,7 @@ package redis.clients.jedis.mcf; public enum HealthStatus { - HEALTHY(0x01), UNHEALTHY(0x02); + UNKNOWN(0x00), HEALTHY(0x01), UNHEALTHY(0x02); private final int value; diff --git a/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java b/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java index 39d71a1e80..c3787e12a1 100644 --- a/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java +++ b/src/main/java/redis/clients/jedis/mcf/HealthStatusManager.java @@ -72,6 +72,10 @@ public void removeAll(Endpoint[] endpoints) { public HealthStatus getHealthStatus(Endpoint endpoint) { HealthCheck healthCheck = healthChecks.get(endpoint); - return healthCheck != null ? healthCheck.getStatus() : HealthStatus.UNHEALTHY; + return healthCheck != null ? healthCheck.getStatus() : HealthStatus.UNKNOWN; + } + + public boolean hasHealthCheck(Endpoint endpoint) { + return healthChecks.get(endpoint) != null; } } diff --git a/src/main/java/redis/clients/jedis/mcf/StatusTracker.java b/src/main/java/redis/clients/jedis/mcf/StatusTracker.java new file mode 100644 index 0000000000..9563e94b2f --- /dev/null +++ b/src/main/java/redis/clients/jedis/mcf/StatusTracker.java @@ -0,0 +1,71 @@ +package redis.clients.jedis.mcf; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +import redis.clients.jedis.exceptions.JedisConnectionException; + +/** + * StatusTracker is responsible for tracking and waiting for health status changes for specific endpoints. It provides + * an event-driven approach to wait for health status transitions from UNKNOWN to either HEALTHY or UNHEALTHY. + */ +public class StatusTracker { + + private final HealthStatusManager healthStatusManager; + + public StatusTracker(HealthStatusManager healthStatusManager) { + this.healthStatusManager = healthStatusManager; + } + + /** + * Waits for a specific endpoint's health status to be determined (not UNKNOWN). Uses event-driven approach with + * CountDownLatch to avoid polling. + * @param endpoint the endpoint to wait for + * @return the determined health status (HEALTHY or UNHEALTHY) + * @throws JedisConnectionException if interrupted while waiting + */ + public HealthStatus waitForHealthStatus(Endpoint endpoint) { + // First check if status is already determined + HealthStatus currentStatus = healthStatusManager.getHealthStatus(endpoint); + if (currentStatus != HealthStatus.UNKNOWN) { + return currentStatus; + } + + // Set up event-driven waiting + final CountDownLatch latch = new CountDownLatch(1); + final AtomicReference resultStatus = new AtomicReference<>(); + + // Create a temporary listener for this specific endpoint + HealthStatusListener tempListener = new HealthStatusListener() { + @Override + public void onStatusChange(HealthStatusChangeEvent event) { + if (event.getEndpoint().equals(endpoint) && event.getNewStatus() != HealthStatus.UNKNOWN) { + resultStatus.set(event.getNewStatus()); + latch.countDown(); + } + } + }; + + // Register the temporary listener + healthStatusManager.registerListener(endpoint, tempListener); + + try { + // Double-check status after registering listener (race condition protection) + currentStatus = healthStatusManager.getHealthStatus(endpoint); + if (currentStatus != HealthStatus.UNKNOWN) { + return currentStatus; + } + + // Wait for the health status change event + latch.await(); + return resultStatus.get(); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new JedisConnectionException("Interrupted while waiting for health check result", e); + } finally { + // Clean up: unregister the temporary listener + healthStatusManager.unregisterListener(endpoint, tempListener); + } + } +} diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index d26e6fa8e0..374a6794b9 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -14,6 +14,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.stream.Collectors; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.ScheduledExecutorService; @@ -37,6 +39,7 @@ import redis.clients.jedis.mcf.HealthStatus; import redis.clients.jedis.mcf.HealthStatusChangeEvent; import redis.clients.jedis.mcf.HealthStatusManager; +import redis.clients.jedis.mcf.StatusTracker; import redis.clients.jedis.MultiClusterClientConfig.StrategySupplier; import redis.clients.jedis.util.Pool; @@ -85,6 +88,13 @@ public class MultiClusterPooledConnectionProvider implements ConnectionProvider private List> fallbackExceptionList; private HealthStatusManager healthStatusManager = new HealthStatusManager(); + private StatusTracker statusTracker; + + // Flag to control when handleHealthStatusChange should process events (only after initialization) + private volatile boolean initializationComplete = false; + + // Queue to hold health status events during initialization + private final ConcurrentLinkedQueue pendingHealthStatusChanges = new ConcurrentLinkedQueue<>(); // Failback mechanism fields private final ScheduledExecutorService failbackScheduler = Executors.newSingleThreadScheduledExecutor(r -> { @@ -148,17 +158,23 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste ////////////// Configure Cluster Map //////////////////// ClusterConfig[] clusterConfigs = multiClusterClientConfig.getClusterConfigs(); + + // Now add clusters - health checks will start but events will be queued for (ClusterConfig config : clusterConfigs) { addClusterInternal(multiClusterClientConfig, config); } - // selecting activeCluster with configuration values. - // all health status would be HEALTHY at this point - activeCluster = findWeightedHealthyClusterToIterate().getValue(); + // Initialize StatusTracker for waiting on health check results + statusTracker = new StatusTracker(healthStatusManager); - for (Endpoint endpoint : multiClusterMap.keySet()) { - healthStatusManager.registerListener(endpoint, this::handleStatusChange); - } + // Wait for initial health check results and select active cluster based on weights + activeCluster = waitForInitialHealthyCluster(); + + // Mark initialization as complete - handleHealthStatusChange can now process events + initializationComplete = true; + + // Process any events that were queued during initialization + processPendingHealthStatusChanges(); /// --- /// this.fallbackExceptionList = multiClusterClientConfig.getFallbackExceptionList(); @@ -189,7 +205,6 @@ public void add(ClusterConfig clusterConfig) { activeClusterIndexLock.lock(); try { addClusterInternal(multiClusterClientConfig, clusterConfig); - healthStatusManager.registerListener(endpoint, this::handleStatusChange); } finally { activeClusterIndexLock.unlock(); } @@ -232,7 +247,7 @@ public void remove(Endpoint endpoint) { } // Remove from health status manager first - healthStatusManager.unregisterListener(endpoint, this::handleStatusChange); + healthStatusManager.unregisterListener(endpoint, this::handleHealthStatusChange); healthStatusManager.remove(endpoint); // Remove from cluster map @@ -253,6 +268,10 @@ public void remove(Endpoint endpoint) { * appropriate locks. */ private void addClusterInternal(MultiClusterClientConfig multiClusterClientConfig, ClusterConfig config) { + if (multiClusterMap.containsKey(config.getHostAndPort())) { + throw new JedisValidationException( + "Endpoint " + config.getHostAndPort() + " already exists in the provider"); + } GenericObjectPoolConfig poolConfig = config.getConnectionPoolConfig(); String clusterId = "cluster:" + config.getHostAndPort(); @@ -283,12 +302,47 @@ private void addClusterInternal(MultiClusterClientConfig multiClusterClientConfi StrategySupplier strategySupplier = config.getHealthCheckStrategySupplier(); if (strategySupplier != null) { HealthCheckStrategy hcs = strategySupplier.get(config.getHostAndPort(), config.getJedisClientConfig()); + // Register listeners BEFORE adding clusters to avoid missing events + healthStatusManager.registerListener(config.getHostAndPort(), this::handleHealthStatusChange); healthStatusManager.add(config.getHostAndPort(), hcs); + } else { + cluster.setHealthStatus(HealthStatus.HEALTHY); } } - private void handleStatusChange(HealthStatusChangeEvent eventArgs) { + private void handleHealthStatusChange(HealthStatusChangeEvent eventArgs) { + // Queue events during initialization to process them later + if (!initializationComplete) { + pendingHealthStatusChanges.offer(eventArgs); + return; + } + if (!pendingHealthStatusChanges.isEmpty()) { + processPendingHealthStatusChanges(); + } + // Process the event immediately if initialization is complete + processStatusChangeEvent(eventArgs); + } + + /** + * Processes any health status events that were queued during initialization. This ensures that no events are lost + * during the initialization process. + */ + private void processPendingHealthStatusChanges() { + HealthStatusChangeEvent event; + // Synchronize to ensure the order of events when consuming the queue + synchronized (pendingHealthStatusChanges) { + // Process all queued events + while ((event = pendingHealthStatusChanges.poll()) != null) { + processStatusChangeEvent(event); + } + } + } + /** + * Processes a health status change event. This method contains the actual logic for handling status changes and can + * be called both for queued events and real-time events. + */ + private void processStatusChangeEvent(HealthStatusChangeEvent eventArgs) { Endpoint endpoint = eventArgs.getEndpoint(); HealthStatus newStatus = eventArgs.getNewStatus(); log.info("Health status changed for {} from {} to {}", endpoint, eventArgs.getOldStatus(), newStatus); @@ -310,6 +364,55 @@ private void handleStatusChange(HealthStatusChangeEvent eventArgs) { } } + /** + * Waits for initial health check results and selects the first healthy cluster based on weight priority. Blocks + * until at least one cluster becomes healthy or all clusters are determined to be unhealthy. + * @return the first healthy cluster found, ordered by weight (highest first) + * @throws JedisConnectionException if all clusters are unhealthy + */ + private Cluster waitForInitialHealthyCluster() { + // Sort clusters by weight in descending order + List> sortedClusters = multiClusterMap.entrySet().stream() + .sorted(Map.Entry. comparingByValue(Comparator.comparing(Cluster::getWeight).reversed())) + .collect(Collectors.toList()); + + log.info("Selecting initial cluster from {} configured clusters", sortedClusters.size()); + + // Select clusters in weight order + for (Map.Entry entry : sortedClusters) { + Endpoint endpoint = entry.getKey(); + Cluster cluster = entry.getValue(); + + log.info("Evaluating cluster {} (weight: {})", endpoint, cluster.getWeight()); + + HealthStatus status; + + // Check if health checks are enabled for this endpoint + if (healthStatusManager.hasHealthCheck(endpoint)) { + log.info("Health checks enabled for {}, waiting for result", endpoint); + // Wait for this cluster's health status to be determined + status = statusTracker.waitForHealthStatus(endpoint); + } else { + // No health check configured - assume healthy + log.info("No health check configured for cluster {}, deafulting to HEALTHY", endpoint); + status = HealthStatus.HEALTHY; + } + + cluster.setHealthStatus(status); + + if (status.isHealthy()) { + log.info("Found healthy cluster: {} (weight: {})", endpoint, cluster.getWeight()); + return cluster; + } else { + log.info("Cluster {} is unhealthy, trying next cluster", endpoint); + } + } + + // All clusters are unhealthy + throw new JedisConnectionException( + "All configured clusters are unhealthy. Cannot initialize MultiClusterPooledConnectionProvider."); + } + /** * Periodic failback checker - runs at configured intervals to check for failback opportunities */ @@ -525,8 +628,8 @@ public static class Cluster { private final Retry retry; private final CircuitBreaker circuitBreaker; private final float weight; - // it starts its life with the assumption of being healthy - private HealthStatus healthStatus = HealthStatus.HEALTHY; + // it starts its life with unknown health status, waiting for initial health check + private HealthStatus healthStatus = HealthStatus.UNKNOWN; private MultiClusterClientConfig multiClusterClientConfig; private boolean disabled = false; diff --git a/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java index f74eee71ab..dc06e5856d 100644 --- a/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java @@ -49,12 +49,12 @@ private MockedConstruction mockPool() { private void triggerHealthStatusChange(MultiClusterPooledConnectionProvider provider, HostAndPort endpoint, HealthStatus oldStatus, HealthStatus newStatus) { try { - Method handleStatusChangeMethod = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("handleStatusChange", HealthStatusChangeEvent.class); - handleStatusChangeMethod.setAccessible(true); + Method handleHealthStatusChange = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("handleHealthStatusChange", HealthStatusChangeEvent.class); + handleHealthStatusChange.setAccessible(true); HealthStatusChangeEvent event = new HealthStatusChangeEvent(endpoint, oldStatus, newStatus); - handleStatusChangeMethod.invoke(provider, event); + handleHealthStatusChange.invoke(provider, event); } catch (Exception e) { throw new RuntimeException("Failed to trigger health status change", e); } diff --git a/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java b/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java index eeb1e9caa7..ed2cf8139e 100644 --- a/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/mcf/HealthCheckIntegrationTest.java @@ -72,7 +72,8 @@ public int getTimeout() { public HealthStatus doHealthCheck(Endpoint endpoint) { // Create connection per health check to avoid resource leak try (UnifiedJedis pinger = new UnifiedJedis(hostAndPort, jedisClientConfig)) { - return "OK".equals(pinger.ping()) ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; + String result = pinger.ping(); + return "PONG".equals(result) ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; } catch (Exception e) { return HealthStatus.UNHEALTHY; } diff --git a/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java b/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java index 023c1b62f2..bfbfd452fe 100644 --- a/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java +++ b/src/test/java/redis/clients/jedis/mcf/HealthCheckTest.java @@ -59,28 +59,9 @@ void setUp() { testConfig = DefaultJedisClientConfig.builder().build(); } - // ========== HealthStatus Tests ========== - - @Test - void testHealthStatusIsHealthy() { - assertTrue(HealthStatus.HEALTHY.isHealthy()); - assertFalse(HealthStatus.UNHEALTHY.isHealthy()); - } - - // ========== HostAndPort Tests ========== - @Test - void testHostAndPortEquality() { - HostAndPort endpoint1 = new HostAndPort("localhost", 6379); - HostAndPort endpoint2 = new HostAndPort("localhost", 6379); - HostAndPort endpoint3 = new HostAndPort("localhost", 6380); - - assertEquals(endpoint1, endpoint2); - assertNotEquals(endpoint1, endpoint3); - assertEquals(endpoint1.hashCode(), endpoint2.hashCode()); - } - // ========== HealthCheckCollection Tests ========== + @Test void testHealthCheckCollectionAdd() { HealthCheckCollection collection = new HealthCheckCollection(); HealthCheck healthCheck = new HealthCheck(testEndpoint, mockStrategy, mockCallback); @@ -155,7 +136,7 @@ void testHealthCheckStatusUpdate() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); Consumer callback = event -> { - assertEquals(HealthStatus.HEALTHY, event.getOldStatus()); + assertEquals(HealthStatus.UNKNOWN, event.getOldStatus()); assertEquals(HealthStatus.UNHEALTHY, event.getNewStatus()); latch.countDown(); }; @@ -238,17 +219,35 @@ void testHealthStatusManagerEndpointSpecificListener() { } @Test - void testHealthStatusManagerLifecycle() { + void testHealthStatusManagerLifecycle() throws InterruptedException { HealthStatusManager manager = new HealthStatusManager(); // Before adding health check - assertEquals(HealthStatus.UNHEALTHY, manager.getHealthStatus(testEndpoint)); + assertEquals(HealthStatus.UNKNOWN, manager.getHealthStatus(testEndpoint)); + + // Set up event listener to wait for initial health check completion + CountDownLatch healthCheckCompleteLatch = new CountDownLatch(1); + HealthStatusListener listener = event -> healthCheckCompleteLatch.countDown(); + + // Register listener before adding health check to capture the initial event + manager.registerListener(testEndpoint, listener); + // Add health check - this will start async health checking manager.add(testEndpoint, alwaysHealthyStrategy); + + // Initially should still be UNKNOWN until first check completes + assertEquals(HealthStatus.UNKNOWN, manager.getHealthStatus(testEndpoint)); + + // Wait for initial health check to complete + assertTrue(healthCheckCompleteLatch.await(2, TimeUnit.SECONDS), + "Initial health check should complete within timeout"); + + // Now should be HEALTHY after initial check assertEquals(HealthStatus.HEALTHY, manager.getHealthStatus(testEndpoint)); + // Clean up and verify removal manager.remove(testEndpoint); - assertEquals(HealthStatus.UNHEALTHY, manager.getHealthStatus(testEndpoint)); + assertEquals(HealthStatus.UNKNOWN, manager.getHealthStatus(testEndpoint)); } // ========== EchoStrategy Tests ========== @@ -275,14 +274,11 @@ void testEchoStrategyDefaultSupplier() { void testNewFieldLocations() { // Test new field locations in ClusterConfig and MultiClusterClientConfig MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig - .builder(testEndpoint, testConfig) - .weight(2.5f) - .build(); + .builder(testEndpoint, testConfig).weight(2.5f).build(); - MultiClusterClientConfig multiConfig = new MultiClusterClientConfig.Builder(new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) - .retryOnFailover(true) - .failbackSupported(false) - .build(); + MultiClusterClientConfig multiConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { clusterConfig }).retryOnFailover(true) + .failbackSupported(false).build(); assertEquals(2.5f, clusterConfig.getWeight()); assertTrue(multiConfig.isRetryOnFailover()); @@ -293,15 +289,15 @@ void testNewFieldLocations() { void testDefaultValues() { // Test default values in ClusterConfig MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig - .builder(testEndpoint, testConfig) - .build(); + .builder(testEndpoint, testConfig).build(); assertEquals(1.0f, clusterConfig.getWeight()); // Default weight - assertEquals(EchoStrategy.DEFAULT, clusterConfig.getHealthCheckStrategySupplier()); // Default is null (no health check) + assertEquals(EchoStrategy.DEFAULT, clusterConfig.getHealthCheckStrategySupplier()); // Default is null (no + // health check) // Test default values in MultiClusterClientConfig - MultiClusterClientConfig multiConfig = new MultiClusterClientConfig.Builder(new MultiClusterClientConfig.ClusterConfig[]{clusterConfig}) - .build(); + MultiClusterClientConfig multiConfig = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { clusterConfig }).build(); assertFalse(multiConfig.isRetryOnFailover()); // Default is false assertTrue(multiConfig.isFailbackSupported()); // Default is true @@ -314,9 +310,7 @@ void testClusterConfigWithHealthCheckStrategy() { MultiClusterClientConfig.StrategySupplier supplier = (hostAndPort, jedisClientConfig) -> customStrategy; MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig - .builder(testEndpoint, testConfig) - .healthCheckStrategySupplier(supplier) - .build(); + .builder(testEndpoint, testConfig).healthCheckStrategySupplier(supplier).build(); assertNotNull(clusterConfig.getHealthCheckStrategySupplier()); HealthCheckStrategy result = clusterConfig.getHealthCheckStrategySupplier().get(testEndpoint, testConfig); @@ -330,9 +324,7 @@ void testClusterConfigWithStrategySupplier() { }; MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig - .builder(testEndpoint, testConfig) - .healthCheckStrategySupplier(customSupplier) - .build(); + .builder(testEndpoint, testConfig).healthCheckStrategySupplier(customSupplier).build(); assertEquals(customSupplier, clusterConfig.getHealthCheckStrategySupplier()); } @@ -344,9 +336,7 @@ void testClusterConfigWithEchoStrategy() { }; MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig - .builder(testEndpoint, testConfig) - .healthCheckStrategySupplier(echoSupplier) - .build(); + .builder(testEndpoint, testConfig).healthCheckStrategySupplier(echoSupplier).build(); MultiClusterClientConfig.StrategySupplier supplier = clusterConfig.getHealthCheckStrategySupplier(); assertNotNull(supplier); @@ -356,8 +346,7 @@ void testClusterConfigWithEchoStrategy() { @Test void testClusterConfigWithDefaultHealthCheck() { MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig - .builder(testEndpoint, testConfig) - .build(); // Should use default EchoStrategy + .builder(testEndpoint, testConfig).build(); // Should use default EchoStrategy assertNotNull(clusterConfig.getHealthCheckStrategySupplier()); assertEquals(EchoStrategy.DEFAULT, clusterConfig.getHealthCheckStrategySupplier()); @@ -366,9 +355,7 @@ void testClusterConfigWithDefaultHealthCheck() { @Test void testClusterConfigWithDisabledHealthCheck() { MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig - .builder(testEndpoint, testConfig) - .healthCheckEnabled(false) - .build(); + .builder(testEndpoint, testConfig).healthCheckEnabled(false).build(); assertNull(clusterConfig.getHealthCheckStrategySupplier()); } @@ -376,9 +363,7 @@ void testClusterConfigWithDisabledHealthCheck() { @Test void testClusterConfigHealthCheckEnabledExplicitly() { MultiClusterClientConfig.ClusterConfig clusterConfig = MultiClusterClientConfig.ClusterConfig - .builder(testEndpoint, testConfig) - .healthCheckEnabled(true) - .build(); + .builder(testEndpoint, testConfig).healthCheckEnabled(true).build(); assertNotNull(clusterConfig.getHealthCheckStrategySupplier()); assertEquals(EchoStrategy.DEFAULT, clusterConfig.getHealthCheckStrategySupplier()); diff --git a/src/test/java/redis/clients/jedis/mcf/HealthStatusEventOrderingTest.java b/src/test/java/redis/clients/jedis/mcf/HealthStatusEventOrderingTest.java new file mode 100644 index 0000000000..5a4fe37ada --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/HealthStatusEventOrderingTest.java @@ -0,0 +1,343 @@ +package redis.clients.jedis.mcf; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.Mockito.*; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.MockitoAnnotations; + +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; + +/** + * Test to verify that health status events are not missed between endpoint registration + * and listener registration, and that handleHealthStatusChange only processes events after + * initialization is complete. + */ +public class HealthStatusEventOrderingTest { + + private HostAndPort testEndpoint; + private JedisClientConfig testConfig; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + testEndpoint = new HostAndPort("localhost", 6379); + testConfig = DefaultJedisClientConfig.builder().build(); + } + + @Test + void testHealthStatusManagerEventOrdering() throws InterruptedException { + HealthStatusManager manager = new HealthStatusManager(); + + // Counter to track events received + AtomicInteger eventCount = new AtomicInteger(0); + CountDownLatch eventLatch = new CountDownLatch(1); + + // Create a listener that counts events + HealthStatusListener listener = event -> { + eventCount.incrementAndGet(); + eventLatch.countDown(); + }; + + // Register listener BEFORE adding endpoint (correct order) + manager.registerListener(testEndpoint, listener); + + // Create a strategy that immediately returns HEALTHY + HealthCheckStrategy immediateStrategy = new HealthCheckStrategy() { + @Override + public int getInterval() { + return 100; + } + + @Override + public int getTimeout() { + return 50; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + return HealthStatus.HEALTHY; + } + }; + + // Add endpoint - this should trigger health check and event + manager.add(testEndpoint, immediateStrategy); + + // Wait for event to be processed + assertTrue(eventLatch.await(2, TimeUnit.SECONDS), "Should receive health status event"); + + // Should have received at least one event (UNKNOWN -> HEALTHY) + assertTrue(eventCount.get() >= 1, "Should have received at least one health status event"); + + manager.remove(testEndpoint); + } + + + + @Test + void testInitializationEventQueuing() { + // This test simulates the new queuing behavior + + AtomicInteger processedEvents = new AtomicInteger(0); + boolean[] initComplete = {false}; + java.util.Queue eventQueue = new java.util.concurrent.ConcurrentLinkedQueue<>(); + + // Simulate handleHealthStatusChange logic with queuing + HealthStatusListener mockHandler = event -> { + if (!initComplete[0]) { + eventQueue.offer(event); // Queue events during initialization + return; + } + processedEvents.incrementAndGet(); + }; + + // Simulate processPendingEvents logic + Runnable processPendingEvents = () -> { + HealthStatusChangeEvent event; + while ((event = eventQueue.poll()) != null) { + processedEvents.incrementAndGet(); + } + }; + + // Simulate events during initialization + HealthStatusChangeEvent event1 = new HealthStatusChangeEvent(testEndpoint, HealthStatus.UNKNOWN, HealthStatus.HEALTHY); + HealthStatusChangeEvent event2 = new HealthStatusChangeEvent(testEndpoint, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + + // Events during initialization should be queued, not processed + mockHandler.onStatusChange(event1); + mockHandler.onStatusChange(event2); + assertEquals(0, processedEvents.get(), "Events during initialization should be queued, not processed"); + assertEquals(2, eventQueue.size(), "Events should be queued during initialization"); + + // Mark initialization complete and process pending events + initComplete[0] = true; + processPendingEvents.run(); + assertEquals(2, processedEvents.get(), "Queued events should be processed after initialization"); + assertEquals(0, eventQueue.size(), "Queue should be empty after processing"); + + // Events after initialization should be processed immediately + HealthStatusChangeEvent event3 = new HealthStatusChangeEvent(testEndpoint, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + mockHandler.onStatusChange(event3); + assertEquals(3, processedEvents.get(), "Events after initialization should be processed immediately"); + } + + @Test + void testHealthStatusManagerHasHealthCheck() { + HealthStatusManager manager = new HealthStatusManager(); + + // Initially no health check + assertFalse(manager.hasHealthCheck(testEndpoint), "Should not have health check initially"); + + // Create a simple strategy + HealthCheckStrategy strategy = new HealthCheckStrategy() { + @Override + public int getInterval() { return 100; } + + @Override + public int getTimeout() { return 50; } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { return HealthStatus.HEALTHY; } + }; + + // Add health check + manager.add(testEndpoint, strategy); + assertTrue(manager.hasHealthCheck(testEndpoint), "Should have health check after adding"); + + // Remove health check + manager.remove(testEndpoint); + assertFalse(manager.hasHealthCheck(testEndpoint), "Should not have health check after removing"); + } + + @Test + void testHealthStatusManagerConcurrentAccess() throws InterruptedException { + HealthStatusManager manager = new HealthStatusManager(); + + // Create multiple endpoints + HostAndPort endpoint1 = new HostAndPort("host1", 6379); + HostAndPort endpoint2 = new HostAndPort("host2", 6379); + + AtomicInteger eventsReceived = new AtomicInteger(0); + CountDownLatch allEventsLatch = new CountDownLatch(4); // Expect 4 events total + + // Create listeners for both endpoints + HealthStatusListener listener1 = event -> { + eventsReceived.incrementAndGet(); + allEventsLatch.countDown(); + }; + + HealthStatusListener listener2 = event -> { + eventsReceived.incrementAndGet(); + allEventsLatch.countDown(); + }; + + // Register listeners concurrently + Thread registerThread1 = new Thread(() -> manager.registerListener(endpoint1, listener1)); + Thread registerThread2 = new Thread(() -> manager.registerListener(endpoint2, listener2)); + + registerThread1.start(); + registerThread2.start(); + + registerThread1.join(); + registerThread2.join(); + + // Create strategies that return different statuses + HealthCheckStrategy strategy1 = new HealthCheckStrategy() { + @Override + public int getInterval() { return 100; } + @Override + public int getTimeout() { return 50; } + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { return HealthStatus.HEALTHY; } + }; + + HealthCheckStrategy strategy2 = new HealthCheckStrategy() { + @Override + public int getInterval() { return 100; } + @Override + public int getTimeout() { return 50; } + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { return HealthStatus.UNHEALTHY; } + }; + + // Add health checks concurrently + Thread addThread1 = new Thread(() -> manager.add(endpoint1, strategy1)); + Thread addThread2 = new Thread(() -> manager.add(endpoint2, strategy2)); + + addThread1.start(); + addThread2.start(); + + addThread1.join(); + addThread2.join(); + + // Wait for all events to be processed + assertTrue(allEventsLatch.await(3, TimeUnit.SECONDS), "Should receive all health status events"); + + // Verify final states + assertTrue(manager.hasHealthCheck(endpoint1)); + assertTrue(manager.hasHealthCheck(endpoint2)); + + // Clean up + manager.remove(endpoint1); + manager.remove(endpoint2); + } + + @Test + void testNoEventsAreMissedDuringRegistration() throws InterruptedException { + HealthStatusManager manager = new HealthStatusManager(); + + // Track all events received + AtomicInteger eventsReceived = new AtomicInteger(0); + CountDownLatch allEventsLatch = new CountDownLatch(2); // Expect 2 events: UNKNOWN->HEALTHY, HEALTHY->UNHEALTHY + + // Create a strategy that changes status quickly + AtomicInteger checkCount = new AtomicInteger(0); + HealthCheckStrategy rapidChangeStrategy = new HealthCheckStrategy() { + @Override + public int getInterval() { return 50; } // Very fast interval + + @Override + public int getTimeout() { return 25; } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + int count = checkCount.incrementAndGet(); + // First check: HEALTHY, second check: UNHEALTHY + return count == 1 ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; + } + }; + + // Create listener that tracks events + HealthStatusListener eventTracker = event -> { + eventsReceived.incrementAndGet(); + allEventsLatch.countDown(); + System.out.println("Event received: " + event.getOldStatus() + " -> " + event.getNewStatus()); + }; + + // Register listener BEFORE adding endpoint (correct order to prevent missing events) + manager.registerListener(testEndpoint, eventTracker); + + // Add endpoint - this should trigger rapid health checks + manager.add(testEndpoint, rapidChangeStrategy); + + // Wait for all expected events + assertTrue(allEventsLatch.await(3, TimeUnit.SECONDS), + "Should receive all health status change events within timeout"); + + // Verify we received the expected number of events + assertEquals(2, eventsReceived.get(), "Should have received exactly 2 health status change events"); + + // Clean up + manager.remove(testEndpoint); + } + + @Test + void testEventOrderingWithMultipleRapidChanges() throws InterruptedException { + HealthStatusManager manager = new HealthStatusManager(); + + // Track events in order + java.util.List receivedEvents = + java.util.Collections.synchronizedList(new java.util.ArrayList<>()); + CountDownLatch eventsLatch = new CountDownLatch(3); // Expect 3 transitions + + // Create a strategy that cycles through statuses + AtomicInteger checkCount = new AtomicInteger(0); + HealthCheckStrategy cyclingStrategy = new HealthCheckStrategy() { + @Override + public int getInterval() { return 30; } // Very fast + + @Override + public int getTimeout() { return 15; } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + int count = checkCount.incrementAndGet(); + switch (count) { + case 1: return HealthStatus.HEALTHY; + case 2: return HealthStatus.UNHEALTHY; + case 3: return HealthStatus.HEALTHY; + default: return HealthStatus.HEALTHY; + } + } + }; + + // Register listener to capture events in order + HealthStatusListener orderTracker = event -> { + receivedEvents.add(event); + eventsLatch.countDown(); + }; + + // Register listener BEFORE adding endpoint + manager.registerListener(testEndpoint, orderTracker); + + // Add endpoint to start health checks + manager.add(testEndpoint, cyclingStrategy); + + // Wait for all events + assertTrue(eventsLatch.await(5, TimeUnit.SECONDS), + "Should receive all rapid health status changes"); + + // Verify event sequence + assertEquals(3, receivedEvents.size(), "Should have received 3 events"); + + // Verify the sequence: UNKNOWN->HEALTHY, HEALTHY->UNHEALTHY, UNHEALTHY->HEALTHY + assertEquals(HealthStatus.UNKNOWN, receivedEvents.get(0).getOldStatus()); + assertEquals(HealthStatus.HEALTHY, receivedEvents.get(0).getNewStatus()); + + assertEquals(HealthStatus.HEALTHY, receivedEvents.get(1).getOldStatus()); + assertEquals(HealthStatus.UNHEALTHY, receivedEvents.get(1).getNewStatus()); + + assertEquals(HealthStatus.UNHEALTHY, receivedEvents.get(2).getOldStatus()); + assertEquals(HealthStatus.HEALTHY, receivedEvents.get(2).getNewStatus()); + + // Clean up + manager.remove(testEndpoint); + } +} diff --git a/src/test/java/redis/clients/jedis/mcf/MultiClusterInitializationTest.java b/src/test/java/redis/clients/jedis/mcf/MultiClusterInitializationTest.java new file mode 100644 index 0000000000..3c42aefea6 --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/MultiClusterInitializationTest.java @@ -0,0 +1,175 @@ +package redis.clients.jedis.mcf; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.Mockito.*; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.MockedConstruction; +import org.mockito.junit.jupiter.MockitoExtension; + +import redis.clients.jedis.Connection; +import redis.clients.jedis.ConnectionPool; +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.MultiClusterClientConfig; +import redis.clients.jedis.exceptions.JedisValidationException; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; + +/** + * Tests for MultiClusterPooledConnectionProvider initialization edge cases + */ +@ExtendWith(MockitoExtension.class) +public class MultiClusterInitializationTest { + + private HostAndPort endpoint1; + private HostAndPort endpoint2; + private HostAndPort endpoint3; + private JedisClientConfig clientConfig; + + @BeforeEach + void setUp() { + endpoint1 = new HostAndPort("localhost", 6379); + endpoint2 = new HostAndPort("localhost", 6380); + endpoint3 = new HostAndPort("localhost", 6381); + clientConfig = DefaultJedisClientConfig.builder().build(); + } + + private MockedConstruction mockPool() { + Connection mockConnection = mock(Connection.class); + lenient().when(mockConnection.ping()).thenReturn(true); + return mockConstruction(ConnectionPool.class, (mock, context) -> { + when(mock.getResource()).thenReturn(mockConnection); + doNothing().when(mock).close(); + }); + } + + @Test + void testInitializationWithMixedHealthCheckConfiguration() { + try (MockedConstruction mockedPool = mockPool()) { + // Create clusters with mixed health check configuration + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .weight(1.0f) + .healthCheckEnabled(false) // No health check + .build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig) + .weight(2.0f) + .healthCheckStrategySupplier(EchoStrategy.DEFAULT) // With health check + .build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }) + .build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Should initialize successfully + assertNotNull(provider.getCluster()); + + // Should select cluster1 (no health check, assumed healthy) or cluster2 based on weight + // Since cluster2 has higher weight and health checks, it should be selected if healthy + assertTrue(provider.getCluster() == provider.getCluster(endpoint1) || + provider.getCluster() == provider.getCluster(endpoint2)); + } + } + } + + @Test + void testInitializationWithAllHealthChecksDisabled() { + try (MockedConstruction mockedPool = mockPool()) { + // Create clusters with no health checks + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .weight(1.0f) + .healthCheckEnabled(false) + .build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig) + .weight(3.0f) // Higher weight + .healthCheckEnabled(false) + .build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }) + .build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Should select cluster2 (highest weight, no health checks) + assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + } + } + } + + @Test + void testInitializationWithSingleCluster() { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .weight(1.0f) + .healthCheckEnabled(false) + .build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster }) + .build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Should select the only available cluster + assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + } + } + } + + @Test + void testErrorHandlingWithNullConfiguration() { + assertThrows(JedisValidationException.class, () -> { + new MultiClusterPooledConnectionProvider(null); + }); + } + + @Test + void testErrorHandlingWithEmptyClusterArray() { + assertThrows(JedisValidationException.class, () -> { + new MultiClusterClientConfig.Builder(new MultiClusterClientConfig.ClusterConfig[0]).build(); + }); + } + + @Test + void testErrorHandlingWithNullClusterConfig() { + assertThrows(IllegalArgumentException.class, () -> { + new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { null }).build(); + }); + } + + @Test + void testInitializationWithZeroWeights() { + try (MockedConstruction mockedPool = mockPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig) + .weight(0.0f) // Zero weight + .healthCheckEnabled(false) + .build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig) + .weight(0.0f) // Zero weight + .healthCheckEnabled(false) + .build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }) + .build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Should still initialize and select one of the clusters + assertNotNull(provider.getCluster()); + } + } + } +} diff --git a/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java b/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java index 0d61aa9fa8..2a493704c8 100644 --- a/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java +++ b/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java @@ -19,7 +19,6 @@ import redis.clients.jedis.JedisClientConfig; import redis.clients.jedis.MultiClusterClientConfig; import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; -import redis.clients.jedis.util.Pool; @ExtendWith(MockitoExtension.class) class PeriodicFailbackTest { @@ -50,12 +49,12 @@ private MockedConstruction mockPool() { private void triggerHealthStatusChange(MultiClusterPooledConnectionProvider provider, HostAndPort endpoint, HealthStatus oldStatus, HealthStatus newStatus) { try { - Method handleStatusChangeMethod = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("handleStatusChange", HealthStatusChangeEvent.class); - handleStatusChangeMethod.setAccessible(true); + Method handleHealthStatusChange = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("handleHealthStatusChange", HealthStatusChangeEvent.class); + handleHealthStatusChange.setAccessible(true); HealthStatusChangeEvent event = new HealthStatusChangeEvent(endpoint, oldStatus, newStatus); - handleStatusChangeMethod.invoke(provider, event); + handleHealthStatusChange.invoke(provider, event); } catch (Exception e) { throw new RuntimeException("Failed to trigger health status change", e); } diff --git a/src/test/java/redis/clients/jedis/mcf/StatusTrackerTest.java b/src/test/java/redis/clients/jedis/mcf/StatusTrackerTest.java new file mode 100644 index 0000000000..3a97267c96 --- /dev/null +++ b/src/test/java/redis/clients/jedis/mcf/StatusTrackerTest.java @@ -0,0 +1,243 @@ +package redis.clients.jedis.mcf; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.Mockito.*; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import redis.clients.jedis.HostAndPort; + +public class StatusTrackerTest { + + @Mock + private HealthStatusManager mockHealthStatusManager; + + private StatusTracker statusTracker; + private HostAndPort testEndpoint; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + statusTracker = new StatusTracker(mockHealthStatusManager); + testEndpoint = new HostAndPort("localhost", 6379); + } + + @Test + void testWaitForHealthStatus_AlreadyDetermined() { + // Given: Health status is already HEALTHY + when(mockHealthStatusManager.getHealthStatus(testEndpoint)).thenReturn(HealthStatus.HEALTHY); + + // When: Waiting for health status + HealthStatus result = statusTracker.waitForHealthStatus(testEndpoint); + + // Then: Should return immediately without waiting + assertEquals(HealthStatus.HEALTHY, result); + verify(mockHealthStatusManager, never()).registerListener(eq(testEndpoint), any(HealthStatusListener.class)); + } + + @Test + void testWaitForHealthStatus_EventDriven() throws InterruptedException { + // Given: Health status is initially UNKNOWN + when(mockHealthStatusManager.getHealthStatus(testEndpoint)) + .thenReturn(HealthStatus.UNKNOWN) // First call + .thenReturn(HealthStatus.UNKNOWN); // Second call after registering listener + + // Capture the registered listener + final HealthStatusListener[] capturedListener = new HealthStatusListener[1]; + doAnswer(invocation -> { + capturedListener[0] = invocation.getArgument(1); + return null; + }).when(mockHealthStatusManager).registerListener(eq(testEndpoint), any(HealthStatusListener.class)); + + // When: Start waiting in a separate thread + CountDownLatch testLatch = new CountDownLatch(1); + final HealthStatus[] result = new HealthStatus[1]; + + Thread waitingThread = new Thread(() -> { + result[0] = statusTracker.waitForHealthStatus(testEndpoint); + testLatch.countDown(); + }); + waitingThread.start(); + + // Give some time for the listener to be registered + Thread.sleep(50); + + // Simulate health status change event + assertNotNull(capturedListener[0], "Listener should have been registered"); + HealthStatusChangeEvent event = new HealthStatusChangeEvent(testEndpoint, HealthStatus.UNKNOWN, HealthStatus.HEALTHY); + capturedListener[0].onStatusChange(event); + + // Then: Should complete and return the new status + assertTrue(testLatch.await(1, TimeUnit.SECONDS), "Should complete within timeout"); + assertEquals(HealthStatus.HEALTHY, result[0]); + + // Verify cleanup + verify(mockHealthStatusManager).unregisterListener(eq(testEndpoint), eq(capturedListener[0])); + } + + @Test + void testWaitForHealthStatus_IgnoresUnknownStatus() throws InterruptedException { + // Given: Health status is initially UNKNOWN + when(mockHealthStatusManager.getHealthStatus(testEndpoint)).thenReturn(HealthStatus.UNKNOWN); + + // Capture the registered listener + final HealthStatusListener[] capturedListener = new HealthStatusListener[1]; + doAnswer(invocation -> { + capturedListener[0] = invocation.getArgument(1); + return null; + }).when(mockHealthStatusManager).registerListener(eq(testEndpoint), any(HealthStatusListener.class)); + + // When: Start waiting in a separate thread + CountDownLatch testLatch = new CountDownLatch(1); + final HealthStatus[] result = new HealthStatus[1]; + + Thread waitingThread = new Thread(() -> { + result[0] = statusTracker.waitForHealthStatus(testEndpoint); + testLatch.countDown(); + }); + waitingThread.start(); + + // Give some time for the listener to be registered + Thread.sleep(50); + + // Simulate UNKNOWN status change (should be ignored) + assertNotNull(capturedListener[0], "Listener should have been registered"); + HealthStatusChangeEvent unknownEvent = new HealthStatusChangeEvent(testEndpoint, HealthStatus.UNKNOWN, HealthStatus.UNKNOWN); + capturedListener[0].onStatusChange(unknownEvent); + + // Should not complete yet + assertFalse(testLatch.await(100, TimeUnit.MILLISECONDS), "Should not complete with UNKNOWN status"); + + // Now send a real status change + HealthStatusChangeEvent realEvent = new HealthStatusChangeEvent(testEndpoint, HealthStatus.UNKNOWN, HealthStatus.UNHEALTHY); + capturedListener[0].onStatusChange(realEvent); + + // Then: Should complete now + assertTrue(testLatch.await(1, TimeUnit.SECONDS), "Should complete with real status"); + assertEquals(HealthStatus.UNHEALTHY, result[0]); + } + + @Test + void testWaitForHealthStatus_IgnoresOtherEndpoints() throws InterruptedException { + // Given: Health status is initially UNKNOWN + when(mockHealthStatusManager.getHealthStatus(testEndpoint)).thenReturn(HealthStatus.UNKNOWN); + HostAndPort otherEndpoint = new HostAndPort("other", 6379); + + // Capture the registered listener + final HealthStatusListener[] capturedListener = new HealthStatusListener[1]; + doAnswer(invocation -> { + capturedListener[0] = invocation.getArgument(1); + return null; + }).when(mockHealthStatusManager).registerListener(eq(testEndpoint), any(HealthStatusListener.class)); + + // When: Start waiting in a separate thread + CountDownLatch testLatch = new CountDownLatch(1); + final HealthStatus[] result = new HealthStatus[1]; + + Thread waitingThread = new Thread(() -> { + result[0] = statusTracker.waitForHealthStatus(testEndpoint); + testLatch.countDown(); + }); + waitingThread.start(); + + // Give some time for the listener to be registered + Thread.sleep(50); + + // Simulate status change for different endpoint (should be ignored) + assertNotNull(capturedListener[0], "Listener should have been registered"); + HealthStatusChangeEvent otherEvent = new HealthStatusChangeEvent(otherEndpoint, HealthStatus.UNKNOWN, HealthStatus.HEALTHY); + capturedListener[0].onStatusChange(otherEvent); + + // Should not complete yet + assertFalse(testLatch.await(100, TimeUnit.MILLISECONDS), "Should not complete with other endpoint"); + + // Now send event for correct endpoint + HealthStatusChangeEvent correctEvent = new HealthStatusChangeEvent(testEndpoint, HealthStatus.UNKNOWN, HealthStatus.HEALTHY); + capturedListener[0].onStatusChange(correctEvent); + + // Then: Should complete now + assertTrue(testLatch.await(1, TimeUnit.SECONDS), "Should complete with correct endpoint"); + assertEquals(HealthStatus.HEALTHY, result[0]); + } + + @Test + void testWaitForHealthStatus_InterruptHandling() { + // Given: Health status is initially UNKNOWN and will stay that way + when(mockHealthStatusManager.getHealthStatus(testEndpoint)).thenReturn(HealthStatus.UNKNOWN); + + // When: Interrupt the waiting thread + Thread testThread = new Thread(() -> { + try { + statusTracker.waitForHealthStatus(testEndpoint); + fail("Should have thrown JedisConnectionException due to interrupt"); + } catch (Exception e) { + assertTrue(e.getMessage().contains("Interrupted while waiting")); + assertTrue(Thread.currentThread().isInterrupted()); + } + }); + + testThread.start(); + + // Give thread time to start waiting + try { + Thread.sleep(50); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + // Interrupt the waiting thread + testThread.interrupt(); + + try { + testThread.join(1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + assertFalse(testThread.isAlive(), "Test thread should have completed"); + } + + @Test + void testWaitForHealthStatus_RaceConditionProtection() { + // Given: Health status changes between first check and listener registration + when(mockHealthStatusManager.getHealthStatus(testEndpoint)) + .thenReturn(HealthStatus.UNKNOWN) // First call + .thenReturn(HealthStatus.HEALTHY); // Second call after registering listener + + // When: Waiting for health status + HealthStatus result = statusTracker.waitForHealthStatus(testEndpoint); + + // Then: Should return the status from the second check without waiting + assertEquals(HealthStatus.HEALTHY, result); + + // Verify listener was registered and unregistered + verify(mockHealthStatusManager).registerListener(eq(testEndpoint), any(HealthStatusListener.class)); + verify(mockHealthStatusManager).unregisterListener(eq(testEndpoint), any(HealthStatusListener.class)); + } + + @Test + void testWaitForHealthStatus_ListenerCleanupOnException() { + // Given: Health status is initially UNKNOWN + when(mockHealthStatusManager.getHealthStatus(testEndpoint)).thenReturn(HealthStatus.UNKNOWN); + + // Mock registerListener to throw an exception + doThrow(new RuntimeException("Registration failed")) + .when(mockHealthStatusManager).registerListener(eq(testEndpoint), any(HealthStatusListener.class)); + + // When: Waiting for health status + assertThrows(RuntimeException.class, () -> { + statusTracker.waitForHealthStatus(testEndpoint); + }); + + // Then: Should still attempt to unregister (cleanup in finally block) + verify(mockHealthStatusManager).registerListener(eq(testEndpoint), any(HealthStatusListener.class)); + // Note: unregisterListener might not be called if registerListener fails, + // but the finally block should handle this gracefully + } +} From 975ab784d8f92d119c3ed9337e29a36c46f4d082 Mon Sep 17 00:00:00 2001 From: atakavci Date: Sat, 19 Jul 2025 14:15:16 +0300 Subject: [PATCH 15/23] - introduce forceActiveCluster by duration - fix formatting --- .../jedis/MultiClusterClientConfig.java | 4 +-- .../MultiClusterPooledConnectionProvider.java | 29 +++++++++++++++++-- 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java index 35a35890c2..a0708f8325 100644 --- a/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java +++ b/src/main/java/redis/clients/jedis/MultiClusterClientConfig.java @@ -165,8 +165,8 @@ public MultiClusterClientConfig(ClusterConfig[] clusterConfigs) { if (clusterConfigs == null || clusterConfigs.length < 1) throw new JedisValidationException( "ClusterClientConfigs are required for MultiClusterPooledConnectionProvider"); for (ClusterConfig clusterConfig : clusterConfigs) { - if (clusterConfig == null) throw new IllegalArgumentException( - "ClusterClientConfigs must not contain null elements"); + if (clusterConfig == null) + throw new IllegalArgumentException("ClusterClientConfigs must not contain null elements"); } this.clusterConfigs = clusterConfigs; } diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 374a6794b9..83faf9de26 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -378,7 +378,7 @@ private Cluster waitForInitialHealthyCluster() { log.info("Selecting initial cluster from {} configured clusters", sortedClusters.size()); - // Select clusters in weight order + // Select cluster in weight order for (Map.Entry entry : sortedClusters) { Endpoint endpoint = entry.getKey(); Cluster cluster = entry.getValue(); @@ -519,6 +519,21 @@ public void setActiveCluster(Endpoint endpoint) { setActiveCluster(cluster, true); } + public void forceActiveCluster(Endpoint endpoint, long forcedActiveDuration) { + Cluster cluster = multiClusterMap.get(endpoint); + cluster.clearGracePeriod(); + if (!cluster.isHealthy()) { + throw new JedisValidationException("Provided endpoint: " + endpoint + + " is not healthy. Please consider a healthy endpoint from the configuration"); + } + multiClusterMap.entrySet().stream().forEach(entry -> { + if (entry.getKey() != endpoint) { + entry.getValue().setGracePeriod(forcedActiveDuration); + } + }); + setActiveCluster(endpoint); + } + private boolean setActiveCluster(Cluster cluster, boolean validateConnection) { // Cluster cluster = clusterEntry.getValue(); // Field-level synchronization is used to avoid the edge case in which @@ -710,7 +725,17 @@ public boolean isInGracePeriod() { * Sets the grace period for this cluster */ public void setGracePeriod() { - gracePeriodEndsAt = System.currentTimeMillis() + multiClusterClientConfig.getGracePeriod(); + setGracePeriod(multiClusterClientConfig.getGracePeriod()); + } + + public void setGracePeriod(long gracePeriod) { + long endTime = System.currentTimeMillis() + gracePeriod; + if (endTime < gracePeriodEndsAt) return; + gracePeriodEndsAt = endTime; + } + + public void clearGracePeriod() { + gracePeriodEndsAt = 0; } /** From 405101e5ff32cbf0771612db9d3e6bb180598641 Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 23 Jul 2025 11:11:26 +0300 Subject: [PATCH 16/23] - fix failing tests by waiting on clusters to get healthy --- ...tiClusterPooledConnectionProviderTest.java | 33 ++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java index 8263a595bf..ea783c0081 100644 --- a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderTest.java @@ -1,6 +1,10 @@ package redis.clients.jedis.providers; import io.github.resilience4j.circuitbreaker.CircuitBreaker; + +import org.awaitility.Awaitility; +import org.awaitility.Durations; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import redis.clients.jedis.*; @@ -8,7 +12,9 @@ import redis.clients.jedis.exceptions.JedisConnectionException; import redis.clients.jedis.exceptions.JedisValidationException; import redis.clients.jedis.mcf.Endpoint; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider.Cluster; +import java.util.Arrays; import java.util.concurrent.atomic.AtomicBoolean; import static org.junit.jupiter.api.Assertions.*; @@ -38,6 +44,12 @@ public void setUp() { new MultiClusterClientConfig.Builder(clusterConfigs).build()); } + @AfterEach + public void destroy() { + provider.close(); + provider = null; + } + @Test public void testCircuitBreakerForcedTransitions() { @@ -55,13 +67,19 @@ public void testCircuitBreakerForcedTransitions() { } @Test - public void testIterateActiveCluster() { + public void testIterateActiveCluster() throws InterruptedException { + waitForClustersToGetHealthy(provider.getCluster(endpointStandalone0.getHostAndPort()), + provider.getCluster(endpointStandalone1.getHostAndPort())); + Endpoint e2 = provider.iterateActiveCluster(); assertEquals(endpointStandalone1.getHostAndPort(), e2); } @Test public void testIterateActiveClusterOutOfRange() { + waitForClustersToGetHealthy(provider.getCluster(endpointStandalone0.getHostAndPort()), + provider.getCluster(endpointStandalone1.getHostAndPort())); + provider.setActiveCluster(endpointStandalone0.getHostAndPort()); provider.getCluster().setDisabled(true); @@ -69,13 +87,15 @@ public void testIterateActiveClusterOutOfRange() { provider.getCluster().setDisabled(true); assertEquals(endpointStandalone1.getHostAndPort(), e2); - - assertThrows(JedisConnectionException.class, () -> provider.iterateActiveCluster()); // Should throw an - // exception + // Should throw an exception + assertThrows(JedisConnectionException.class, () -> provider.iterateActiveCluster()); } @Test public void testCanIterateOnceMore() { + waitForClustersToGetHealthy(provider.getCluster(endpointStandalone0.getHostAndPort()), + provider.getCluster(endpointStandalone1.getHostAndPort())); + provider.setActiveCluster(endpointStandalone0.getHostAndPort()); provider.getCluster().setDisabled(true); provider.iterateActiveCluster(); @@ -83,6 +103,11 @@ public void testCanIterateOnceMore() { assertFalse(provider.canIterateOnceMore()); } + private void waitForClustersToGetHealthy(Cluster... clusters) { + Awaitility.await().pollInterval(Durations.ONE_HUNDRED_MILLISECONDS).atMost(Durations.TWO_SECONDS) + .until(() -> Arrays.stream(clusters).allMatch(Cluster::isHealthy)); + } + @Test public void testRunClusterFailoverPostProcessor() { ClusterConfig[] clusterConfigs = new ClusterConfig[2]; From 607c66da0bc7f1c6946ea69b9ff916107ac933a7 Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 23 Jul 2025 16:05:06 +0300 Subject: [PATCH 17/23] - fix failing scenario test - downgrade logback version for slf4j compatibility - increase timeouts for faultInjector --- pom.xml | 2 +- .../MultiClusterPooledConnectionProvider.java | 2 + .../scenario/ActiveActiveFailoverTest.java | 86 +++++++++++-------- .../jedis/scenario/FaultInjectionClient.java | 4 +- 4 files changed, 53 insertions(+), 41 deletions(-) diff --git a/pom.xml b/pom.xml index 38f3289acb..05e07a8d25 100644 --- a/pom.xml +++ b/pom.xml @@ -143,7 +143,7 @@ ch.qos.logback logback-classic - 1.3.15 + 1.2.12 test diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 83faf9de26..c511790314 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -650,6 +650,7 @@ public static class Cluster { // Grace period tracking private volatile long gracePeriodEndsAt = 0; + private final Logger log = LoggerFactory.getLogger(getClass()); public Cluster(ConnectionPool connectionPool, Retry retry, CircuitBreaker circuitBreaker, float weight, MultiClusterClientConfig multiClusterClientConfig) { @@ -693,6 +694,7 @@ public float getWeight() { public boolean isCBForcedOpen() { if (circuitBreaker.getState() == State.FORCED_OPEN && !isInGracePeriod()) { + log.info("Transitioning circuit breaker from FORCED_OPEN to CLOSED state due to end of grace period!"); circuitBreaker.transitionToClosedState(); } return circuitBreaker.getState() == CircuitBreaker.State.FORCED_OPEN; diff --git a/src/test/java/redis/clients/jedis/scenario/ActiveActiveFailoverTest.java b/src/test/java/redis/clients/jedis/scenario/ActiveActiveFailoverTest.java index f72923d0fc..6383f1e869 100644 --- a/src/test/java/redis/clients/jedis/scenario/ActiveActiveFailoverTest.java +++ b/src/test/java/redis/clients/jedis/scenario/ActiveActiveFailoverTest.java @@ -8,11 +8,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import redis.clients.jedis.*; +import redis.clients.jedis.MultiClusterClientConfig.ClusterConfig; import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; import redis.clients.jedis.exceptions.JedisConnectionException; import java.io.IOException; -import java.time.Duration; import java.time.Instant; import java.util.HashMap; import java.util.Map; @@ -25,10 +25,7 @@ import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assumptions.assumeTrue; -@Tags({ - @Tag("failover"), - @Tag("scenario") -}) +@Tags({ @Tag("failover"), @Tag("scenario") }) public class ActiveActiveFailoverTest { private static final Logger log = LoggerFactory.getLogger(ActiveActiveFailoverTest.class); @@ -52,13 +49,13 @@ public void testFailover() { MultiClusterClientConfig.ClusterConfig[] clusterConfig = new MultiClusterClientConfig.ClusterConfig[2]; JedisClientConfig config = endpoint.getClientConfigBuilder() - .socketTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS) - .connectionTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS).build(); + .socketTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS) + .connectionTimeoutMillis(RecommendedSettings.DEFAULT_TIMEOUT_MS).build(); - clusterConfig[0] = new MultiClusterClientConfig.ClusterConfig(endpoint.getHostAndPort(0), - config, RecommendedSettings.poolConfig); - clusterConfig[1] = new MultiClusterClientConfig.ClusterConfig(endpoint.getHostAndPort(1), - config, RecommendedSettings.poolConfig); + clusterConfig[0] = ClusterConfig.builder(endpoint.getHostAndPort(0), config) + .connectionPoolConfig(RecommendedSettings.poolConfig).weight(1.0f).build(); + clusterConfig[1] = ClusterConfig.builder(endpoint.getHostAndPort(1), config) + .connectionPoolConfig(RecommendedSettings.poolConfig).weight(0.5f).build(); MultiClusterClientConfig.Builder builder = new MultiClusterClientConfig.Builder(clusterConfig); @@ -67,6 +64,10 @@ public void testFailover() { builder.circuitBreakerSlidingWindowMinCalls(1); builder.circuitBreakerFailureRateThreshold(10.0f); // percentage of failures to trigger circuit breaker + builder.failbackSupported(true); + builder.failbackCheckInterval(1000); + builder.gracePeriod(10000); + builder.retryWaitDuration(10); builder.retryMaxAttempts(1); builder.retryWaitDurationExponentialBackoffMultiplier(1); @@ -79,6 +80,10 @@ class FailoverReporter implements Consumer { Instant failoverAt = null; + boolean failbackHappened = false; + + Instant failbackAt = null; + public String getCurrentClusterName() { return currentClusterName; } @@ -86,17 +91,19 @@ public String getCurrentClusterName() { @Override public void accept(String clusterName) { this.currentClusterName = clusterName; - log.info( - "\n\n====FailoverEvent=== \nJedis failover to cluster: {}\n====FailoverEvent===\n\n", - clusterName); - - failoverHappened = true; - failoverAt = Instant.now(); + log.info("\n\n====FailoverEvent=== \nJedis failover to cluster: {}\n====FailoverEvent===\n\n", clusterName); + + if (failoverHappened) { + failbackHappened = true; + failbackAt = Instant.now(); + } else { + failoverHappened = true; + failoverAt = Instant.now(); + } } } - MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider( - builder.build()); + MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(builder.build()); FailoverReporter reporter = new FailoverReporter(); provider.setClusterFailoverPostProcessor(reporter); provider.setActiveCluster(endpoint.getHostAndPort(0)); @@ -117,15 +124,17 @@ public void accept(String clusterName) { int retryingDelay = 5; while (true) { try { - Map executionInfo = new HashMap() {{ - put("threadId", String.valueOf(threadId)); - put("cluster", reporter.getCurrentClusterName()); - }}; + Map executionInfo = new HashMap() { + { + put("threadId", String.valueOf(threadId)); + put("cluster", reporter.getCurrentClusterName()); + } + }; client.xadd("execution_log", StreamEntryID.NEW_ENTRY, executionInfo); if (attempt > 0) { log.info("Thread {} recovered after {} ms. Threads still not recovered: {}", threadId, - attempt * retryingDelay, retryingThreadsCounter.decrementAndGet()); + attempt * retryingDelay, retryingThreadsCounter.decrementAndGet()); } break; @@ -134,15 +143,13 @@ public void accept(String clusterName) { if (reporter.failoverHappened) { long failedCommands = failedCommandsAfterFailover.incrementAndGet(); lastFailedCommandAt.set(Instant.now()); - log.warn( - "Thread {} failed to execute command after failover. Failed commands after failover: {}", - threadId, failedCommands); + log.warn("Thread {} failed to execute command after failover. Failed commands after failover: {}", threadId, + failedCommands); } if (attempt == 0) { long failedThreads = retryingThreadsCounter.incrementAndGet(); - log.warn("Thread {} failed to execute command. Failed threads: {}", threadId, - failedThreads); + log.warn("Thread {} failed to execute command. Failed threads: {}", threadId, failedThreads); } try { Thread.sleep(retryingDelay); @@ -153,20 +160,21 @@ public void accept(String clusterName) { } } return true; - }, 18); + }, 4); fakeApp.setKeepExecutingForSeconds(30); Thread t = new Thread(fakeApp); t.start(); HashMap params = new HashMap<>(); params.put("bdb_id", endpoint.getBdbId()); - params.put("rlutil_command", "pause_bdb"); + params.put("actions", + "[{\"type\":\"execute_rlutil_command\",\"params\":{\"rlutil_command\":\"pause_bdb\"}},{\"type\":\"wait\",\"params\":{\"wait_time\":\"15\"}},{\"type\":\"execute_rlutil_command\",\"params\":{\"rlutil_command\":\"resume_bdb\"}}]"); FaultInjectionClient.TriggerActionResponse actionResponse = null; try { - log.info("Triggering bdb_pause"); - actionResponse = faultClient.triggerAction("execute_rlutil_command", params); + log.info("Triggering bdb_pause + wait 15 seconds + bdb_resume"); + actionResponse = faultClient.triggerAction("sequence_of_actions", params); } catch (IOException e) { fail("Fault Injection Server error:" + e.getMessage()); } @@ -182,15 +190,17 @@ public void accept(String clusterName) { ConnectionPool pool = provider.getCluster(endpoint.getHostAndPort(0)).getConnectionPool(); - log.info("First connection pool state: active: {}, idle: {}", pool.getNumActive(), - pool.getNumIdle()); - log.info("Full failover time: {} s", - Duration.between(reporter.failoverAt, lastFailedCommandAt.get()).getSeconds()); + log.info("First connection pool state: active: {}, idle: {}", pool.getNumActive(), pool.getNumIdle()); + log.info("Failover happened at: {}", reporter.failoverAt); + log.info("Failback happened at: {}", reporter.failbackAt); + log.info("Last failed command at: {}", lastFailedCommandAt.get()); assertEquals(0, pool.getNumActive()); assertTrue(fakeApp.capturedExceptions().isEmpty()); + assertTrue(reporter.failoverHappened); + assertTrue(reporter.failbackHappened); client.close(); } -} +} \ No newline at end of file diff --git a/src/test/java/redis/clients/jedis/scenario/FaultInjectionClient.java b/src/test/java/redis/clients/jedis/scenario/FaultInjectionClient.java index c4e1c5717b..17170251cd 100644 --- a/src/test/java/redis/clients/jedis/scenario/FaultInjectionClient.java +++ b/src/test/java/redis/clients/jedis/scenario/FaultInjectionClient.java @@ -89,8 +89,8 @@ public boolean isCompleted(Duration checkInterval, Duration delayAfter, Duration private static CloseableHttpClient getHttpClient() { RequestConfig requestConfig = RequestConfig.custom() - .setConnectionRequestTimeout(5000, TimeUnit.MILLISECONDS) - .setResponseTimeout(5000, TimeUnit.MILLISECONDS).build(); + .setConnectionRequestTimeout(10000, TimeUnit.MILLISECONDS) + .setResponseTimeout(10000, TimeUnit.MILLISECONDS).build(); return HttpClientBuilder.create() .setDefaultRequestConfig(requestConfig).build(); From aaac8f7299c772957255acab2f68f4437b3f340c Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 23 Jul 2025 16:18:25 +0300 Subject: [PATCH 18/23] - adressing reviews and feedback --- .../java/redis/clients/jedis/mcf/HealthCheck.java | 4 ++-- .../redis/clients/jedis/mcf/StatusTracker.java | 8 +++++++- .../MultiClusterPooledConnectionProvider.java | 14 ++++++++++---- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/main/java/redis/clients/jedis/mcf/HealthCheck.java b/src/main/java/redis/clients/jedis/mcf/HealthCheck.java index 6b2e91fc11..2f4cbd915f 100644 --- a/src/main/java/redis/clients/jedis/mcf/HealthCheck.java +++ b/src/main/java/redis/clients/jedis/mcf/HealthCheck.java @@ -92,9 +92,9 @@ private void safeUpdate(long owner, HealthStatus status) { } return current; }); - if (oldStatus.getKey() != owner || oldStatus.getValue() != status) { + if (oldStatus.getValue() != status) { // notify listeners - notifyListeners(oldStatus.getValue(), newStatus.getValue()); + notifyListeners(oldStatus.getValue(), status); } } diff --git a/src/main/java/redis/clients/jedis/mcf/StatusTracker.java b/src/main/java/redis/clients/jedis/mcf/StatusTracker.java index 9563e94b2f..26364f783c 100644 --- a/src/main/java/redis/clients/jedis/mcf/StatusTracker.java +++ b/src/main/java/redis/clients/jedis/mcf/StatusTracker.java @@ -1,9 +1,11 @@ package redis.clients.jedis.mcf; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import redis.clients.jedis.exceptions.JedisConnectionException; +import redis.clients.jedis.exceptions.JedisValidationException; /** * StatusTracker is responsible for tracking and waiting for health status changes for specific endpoints. It provides @@ -57,7 +59,11 @@ public void onStatusChange(HealthStatusChangeEvent event) { } // Wait for the health status change event - latch.await(); + // just for safety to not block indefinitely + boolean completed = latch.await(60, TimeUnit.SECONDS); + if (!completed) { + throw new JedisValidationException("Timeout while waiting for health check result"); + } return resultStatus.get(); } catch (InterruptedException e) { diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index c511790314..0d307eb25c 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -15,6 +15,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -97,8 +98,9 @@ public class MultiClusterPooledConnectionProvider implements ConnectionProvider private final ConcurrentLinkedQueue pendingHealthStatusChanges = new ConcurrentLinkedQueue<>(); // Failback mechanism fields + private static final AtomicInteger failbackThreadCounter = new AtomicInteger(1); private final ScheduledExecutorService failbackScheduler = Executors.newSingleThreadScheduledExecutor(r -> { - Thread t = new Thread(r, "failback-scheduler"); + Thread t = new Thread(r, "jedis-failback-" + failbackThreadCounter.getAndIncrement()); t.setDaemon(true); return t; }); @@ -333,7 +335,11 @@ private void processPendingHealthStatusChanges() { synchronized (pendingHealthStatusChanges) { // Process all queued events while ((event = pendingHealthStatusChanges.poll()) != null) { - processStatusChangeEvent(event); + Endpoint endpoint = event.getEndpoint(); + boolean latestInTheQueue = !pendingHealthStatusChanges.stream().anyMatch(e -> e.getEndpoint().equals(endpoint)); + if (latestInTheQueue) { + processStatusChangeEvent(event); + } } } } @@ -345,7 +351,7 @@ private void processPendingHealthStatusChanges() { private void processStatusChangeEvent(HealthStatusChangeEvent eventArgs) { Endpoint endpoint = eventArgs.getEndpoint(); HealthStatus newStatus = eventArgs.getNewStatus(); - log.info("Health status changed for {} from {} to {}", endpoint, eventArgs.getOldStatus(), newStatus); + log.debug("Health status changed for {} from {} to {}", endpoint, eventArgs.getOldStatus(), newStatus); Cluster clusterWithHealthChange = multiClusterMap.get(endpoint); @@ -394,7 +400,7 @@ private Cluster waitForInitialHealthyCluster() { status = statusTracker.waitForHealthStatus(endpoint); } else { // No health check configured - assume healthy - log.info("No health check configured for cluster {}, deafulting to HEALTHY", endpoint); + log.info("No health check configured for cluster {}, defaulting to HEALTHY", endpoint); status = HealthStatus.HEALTHY; } From 2ffffef46b77b9233861446c4f25a72360e0fcab Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 23 Jul 2025 18:05:11 +0300 Subject: [PATCH 19/23] - fix formatting --- .../jedis/providers/MultiClusterPooledConnectionProvider.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 0d307eb25c..2793ccba97 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -336,7 +336,8 @@ private void processPendingHealthStatusChanges() { // Process all queued events while ((event = pendingHealthStatusChanges.poll()) != null) { Endpoint endpoint = event.getEndpoint(); - boolean latestInTheQueue = !pendingHealthStatusChanges.stream().anyMatch(e -> e.getEndpoint().equals(endpoint)); + boolean latestInTheQueue = !pendingHealthStatusChanges.stream() + .anyMatch(e -> e.getEndpoint().equals(endpoint)); if (latestInTheQueue) { processStatusChangeEvent(event); } From e6e1121b54463cf0c66eed80fa2c56e96b8bb4e1 Mon Sep 17 00:00:00 2001 From: atakavci Date: Wed, 23 Jul 2025 18:05:58 +0300 Subject: [PATCH 20/23] - fix formatting --- src/main/java/redis/clients/jedis/mcf/StatusTracker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/redis/clients/jedis/mcf/StatusTracker.java b/src/main/java/redis/clients/jedis/mcf/StatusTracker.java index 26364f783c..ed3d4ee8de 100644 --- a/src/main/java/redis/clients/jedis/mcf/StatusTracker.java +++ b/src/main/java/redis/clients/jedis/mcf/StatusTracker.java @@ -61,7 +61,7 @@ public void onStatusChange(HealthStatusChangeEvent event) { // Wait for the health status change event // just for safety to not block indefinitely boolean completed = latch.await(60, TimeUnit.SECONDS); - if (!completed) { + if (!completed) { throw new JedisValidationException("Timeout while waiting for health check result"); } return resultStatus.get(); From b8d4e87427cf90e25c307371c9dc35d5823ccaff Mon Sep 17 00:00:00 2001 From: atakavci Date: Thu, 24 Jul 2025 14:02:27 +0300 Subject: [PATCH 21/23] - get rid of the queue and event ordering for healthstatus change in MultiClusterPooledConnectionProvider - add test for init and post init events - fix failing tests --- .../MultiClusterPooledConnectionProvider.java | 111 ++--- .../mcf/HealthStatusEventOrderingTest.java | 343 -------------- ...erProviderHealthStatusChangeEventTest.java | 438 ++++++++++++++++++ 3 files changed, 475 insertions(+), 417 deletions(-) delete mode 100644 src/test/java/redis/clients/jedis/mcf/HealthStatusEventOrderingTest.java create mode 100644 src/test/java/redis/clients/jedis/providers/MultiClusterProviderHealthStatusChangeEventTest.java diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 2793ccba97..8efcf3559b 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -94,9 +94,6 @@ public class MultiClusterPooledConnectionProvider implements ConnectionProvider // Flag to control when handleHealthStatusChange should process events (only after initialization) private volatile boolean initializationComplete = false; - // Queue to hold health status events during initialization - private final ConcurrentLinkedQueue pendingHealthStatusChanges = new ConcurrentLinkedQueue<>(); - // Failback mechanism fields private static final AtomicInteger failbackThreadCounter = new AtomicInteger(1); private final ScheduledExecutorService failbackScheduler = Executors.newSingleThreadScheduledExecutor(r -> { @@ -174,11 +171,9 @@ public MultiClusterPooledConnectionProvider(MultiClusterClientConfig multiCluste // Mark initialization as complete - handleHealthStatusChange can now process events initializationComplete = true; - - // Process any events that were queued during initialization - processPendingHealthStatusChanges(); - /// --- /// - + if (!activeCluster.isHealthy()) { + activeCluster = waitForInitialHealthyCluster(); + } this.fallbackExceptionList = multiClusterClientConfig.getFallbackExceptionList(); // Start periodic failback checker @@ -249,7 +244,7 @@ public void remove(Endpoint endpoint) { } // Remove from health status manager first - healthStatusManager.unregisterListener(endpoint, this::handleHealthStatusChange); + healthStatusManager.unregisterListener(endpoint, this::onHealthStatusChange); healthStatusManager.remove(endpoint); // Remove from cluster map @@ -305,64 +300,28 @@ private void addClusterInternal(MultiClusterClientConfig multiClusterClientConfi if (strategySupplier != null) { HealthCheckStrategy hcs = strategySupplier.get(config.getHostAndPort(), config.getJedisClientConfig()); // Register listeners BEFORE adding clusters to avoid missing events - healthStatusManager.registerListener(config.getHostAndPort(), this::handleHealthStatusChange); + healthStatusManager.registerListener(config.getHostAndPort(), this::onHealthStatusChange); healthStatusManager.add(config.getHostAndPort(), hcs); } else { cluster.setHealthStatus(HealthStatus.HEALTHY); } } - private void handleHealthStatusChange(HealthStatusChangeEvent eventArgs) { - // Queue events during initialization to process them later - if (!initializationComplete) { - pendingHealthStatusChanges.offer(eventArgs); - return; - } - if (!pendingHealthStatusChanges.isEmpty()) { - processPendingHealthStatusChanges(); - } - // Process the event immediately if initialization is complete - processStatusChangeEvent(eventArgs); - } - /** - * Processes any health status events that were queued during initialization. This ensures that no events are lost - * during the initialization process. + * Handles health status changes for clusters. This method is called by the health status manager when the health + * status of a cluster changes. */ - private void processPendingHealthStatusChanges() { - HealthStatusChangeEvent event; - // Synchronize to ensure the order of events when consuming the queue - synchronized (pendingHealthStatusChanges) { - // Process all queued events - while ((event = pendingHealthStatusChanges.poll()) != null) { - Endpoint endpoint = event.getEndpoint(); - boolean latestInTheQueue = !pendingHealthStatusChanges.stream() - .anyMatch(e -> e.getEndpoint().equals(endpoint)); - if (latestInTheQueue) { - processStatusChangeEvent(event); - } - } - } - } - - /** - * Processes a health status change event. This method contains the actual logic for handling status changes and can - * be called both for queued events and real-time events. - */ - private void processStatusChangeEvent(HealthStatusChangeEvent eventArgs) { + private void onHealthStatusChange(HealthStatusChangeEvent eventArgs) { Endpoint endpoint = eventArgs.getEndpoint(); HealthStatus newStatus = eventArgs.getNewStatus(); log.debug("Health status changed for {} from {} to {}", endpoint, eventArgs.getOldStatus(), newStatus); - Cluster clusterWithHealthChange = multiClusterMap.get(endpoint); if (clusterWithHealthChange == null) return; clusterWithHealthChange.setHealthStatus(newStatus); - - if (!newStatus.isHealthy()) { - // Handle failover if this was the active cluster - if (clusterWithHealthChange == activeCluster) { + if (initializationComplete) { + if (!newStatus.isHealthy() && clusterWithHealthChange == activeCluster) { clusterWithHealthChange.setGracePeriod(); if (iterateActiveCluster() != null) { this.runClusterFailoverPostProcessor(activeCluster); @@ -424,35 +383,39 @@ private Cluster waitForInitialHealthyCluster() { * Periodic failback checker - runs at configured intervals to check for failback opportunities */ private void periodicFailbackCheck() { - // Find the best candidate cluster for failback - Cluster bestCandidate = null; - float bestWeight = activeCluster.getWeight(); + try { + // Find the best candidate cluster for failback + Cluster bestCandidate = null; + float bestWeight = activeCluster.getWeight(); - for (Map.Entry entry : multiClusterMap.entrySet()) { - Cluster cluster = entry.getValue(); + for (Map.Entry entry : multiClusterMap.entrySet()) { + Cluster cluster = entry.getValue(); - // Skip if this is already the active cluster - if (cluster == activeCluster) { - continue; - } + // Skip if this is already the active cluster + if (cluster == activeCluster) { + continue; + } - // Skip if cluster is not healthy - if (!cluster.isHealthy()) { - continue; - } + // Skip if cluster is not healthy + if (!cluster.isHealthy()) { + continue; + } - // This cluster is a valid candidate - if (cluster.getWeight() > bestWeight) { - bestCandidate = cluster; - bestWeight = cluster.getWeight(); + // This cluster is a valid candidate + if (cluster.getWeight() > bestWeight) { + bestCandidate = cluster; + bestWeight = cluster.getWeight(); + } } - } - // Perform failback if we found a better candidate - if (bestCandidate != null) { - log.info("Performing failback from {} to {} (higher weight cluster available)", - activeCluster.getCircuitBreaker().getName(), bestCandidate.getCircuitBreaker().getName()); - setActiveCluster(bestCandidate, true); + // Perform failback if we found a better candidate + if (bestCandidate != null) { + log.info("Performing failback from {} to {} (higher weight cluster available)", + activeCluster.getCircuitBreaker().getName(), bestCandidate.getCircuitBreaker().getName()); + setActiveCluster(bestCandidate, true); + } + } catch (Exception e) { + log.error("Error during periodic failback check", e); } } diff --git a/src/test/java/redis/clients/jedis/mcf/HealthStatusEventOrderingTest.java b/src/test/java/redis/clients/jedis/mcf/HealthStatusEventOrderingTest.java deleted file mode 100644 index 5a4fe37ada..0000000000 --- a/src/test/java/redis/clients/jedis/mcf/HealthStatusEventOrderingTest.java +++ /dev/null @@ -1,343 +0,0 @@ -package redis.clients.jedis.mcf; - -import static org.junit.jupiter.api.Assertions.*; -import static org.mockito.Mockito.*; - -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.MockitoAnnotations; - -import redis.clients.jedis.DefaultJedisClientConfig; -import redis.clients.jedis.HostAndPort; -import redis.clients.jedis.JedisClientConfig; - -/** - * Test to verify that health status events are not missed between endpoint registration - * and listener registration, and that handleHealthStatusChange only processes events after - * initialization is complete. - */ -public class HealthStatusEventOrderingTest { - - private HostAndPort testEndpoint; - private JedisClientConfig testConfig; - - @BeforeEach - void setUp() { - MockitoAnnotations.openMocks(this); - testEndpoint = new HostAndPort("localhost", 6379); - testConfig = DefaultJedisClientConfig.builder().build(); - } - - @Test - void testHealthStatusManagerEventOrdering() throws InterruptedException { - HealthStatusManager manager = new HealthStatusManager(); - - // Counter to track events received - AtomicInteger eventCount = new AtomicInteger(0); - CountDownLatch eventLatch = new CountDownLatch(1); - - // Create a listener that counts events - HealthStatusListener listener = event -> { - eventCount.incrementAndGet(); - eventLatch.countDown(); - }; - - // Register listener BEFORE adding endpoint (correct order) - manager.registerListener(testEndpoint, listener); - - // Create a strategy that immediately returns HEALTHY - HealthCheckStrategy immediateStrategy = new HealthCheckStrategy() { - @Override - public int getInterval() { - return 100; - } - - @Override - public int getTimeout() { - return 50; - } - - @Override - public HealthStatus doHealthCheck(Endpoint endpoint) { - return HealthStatus.HEALTHY; - } - }; - - // Add endpoint - this should trigger health check and event - manager.add(testEndpoint, immediateStrategy); - - // Wait for event to be processed - assertTrue(eventLatch.await(2, TimeUnit.SECONDS), "Should receive health status event"); - - // Should have received at least one event (UNKNOWN -> HEALTHY) - assertTrue(eventCount.get() >= 1, "Should have received at least one health status event"); - - manager.remove(testEndpoint); - } - - - - @Test - void testInitializationEventQueuing() { - // This test simulates the new queuing behavior - - AtomicInteger processedEvents = new AtomicInteger(0); - boolean[] initComplete = {false}; - java.util.Queue eventQueue = new java.util.concurrent.ConcurrentLinkedQueue<>(); - - // Simulate handleHealthStatusChange logic with queuing - HealthStatusListener mockHandler = event -> { - if (!initComplete[0]) { - eventQueue.offer(event); // Queue events during initialization - return; - } - processedEvents.incrementAndGet(); - }; - - // Simulate processPendingEvents logic - Runnable processPendingEvents = () -> { - HealthStatusChangeEvent event; - while ((event = eventQueue.poll()) != null) { - processedEvents.incrementAndGet(); - } - }; - - // Simulate events during initialization - HealthStatusChangeEvent event1 = new HealthStatusChangeEvent(testEndpoint, HealthStatus.UNKNOWN, HealthStatus.HEALTHY); - HealthStatusChangeEvent event2 = new HealthStatusChangeEvent(testEndpoint, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); - - // Events during initialization should be queued, not processed - mockHandler.onStatusChange(event1); - mockHandler.onStatusChange(event2); - assertEquals(0, processedEvents.get(), "Events during initialization should be queued, not processed"); - assertEquals(2, eventQueue.size(), "Events should be queued during initialization"); - - // Mark initialization complete and process pending events - initComplete[0] = true; - processPendingEvents.run(); - assertEquals(2, processedEvents.get(), "Queued events should be processed after initialization"); - assertEquals(0, eventQueue.size(), "Queue should be empty after processing"); - - // Events after initialization should be processed immediately - HealthStatusChangeEvent event3 = new HealthStatusChangeEvent(testEndpoint, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); - mockHandler.onStatusChange(event3); - assertEquals(3, processedEvents.get(), "Events after initialization should be processed immediately"); - } - - @Test - void testHealthStatusManagerHasHealthCheck() { - HealthStatusManager manager = new HealthStatusManager(); - - // Initially no health check - assertFalse(manager.hasHealthCheck(testEndpoint), "Should not have health check initially"); - - // Create a simple strategy - HealthCheckStrategy strategy = new HealthCheckStrategy() { - @Override - public int getInterval() { return 100; } - - @Override - public int getTimeout() { return 50; } - - @Override - public HealthStatus doHealthCheck(Endpoint endpoint) { return HealthStatus.HEALTHY; } - }; - - // Add health check - manager.add(testEndpoint, strategy); - assertTrue(manager.hasHealthCheck(testEndpoint), "Should have health check after adding"); - - // Remove health check - manager.remove(testEndpoint); - assertFalse(manager.hasHealthCheck(testEndpoint), "Should not have health check after removing"); - } - - @Test - void testHealthStatusManagerConcurrentAccess() throws InterruptedException { - HealthStatusManager manager = new HealthStatusManager(); - - // Create multiple endpoints - HostAndPort endpoint1 = new HostAndPort("host1", 6379); - HostAndPort endpoint2 = new HostAndPort("host2", 6379); - - AtomicInteger eventsReceived = new AtomicInteger(0); - CountDownLatch allEventsLatch = new CountDownLatch(4); // Expect 4 events total - - // Create listeners for both endpoints - HealthStatusListener listener1 = event -> { - eventsReceived.incrementAndGet(); - allEventsLatch.countDown(); - }; - - HealthStatusListener listener2 = event -> { - eventsReceived.incrementAndGet(); - allEventsLatch.countDown(); - }; - - // Register listeners concurrently - Thread registerThread1 = new Thread(() -> manager.registerListener(endpoint1, listener1)); - Thread registerThread2 = new Thread(() -> manager.registerListener(endpoint2, listener2)); - - registerThread1.start(); - registerThread2.start(); - - registerThread1.join(); - registerThread2.join(); - - // Create strategies that return different statuses - HealthCheckStrategy strategy1 = new HealthCheckStrategy() { - @Override - public int getInterval() { return 100; } - @Override - public int getTimeout() { return 50; } - @Override - public HealthStatus doHealthCheck(Endpoint endpoint) { return HealthStatus.HEALTHY; } - }; - - HealthCheckStrategy strategy2 = new HealthCheckStrategy() { - @Override - public int getInterval() { return 100; } - @Override - public int getTimeout() { return 50; } - @Override - public HealthStatus doHealthCheck(Endpoint endpoint) { return HealthStatus.UNHEALTHY; } - }; - - // Add health checks concurrently - Thread addThread1 = new Thread(() -> manager.add(endpoint1, strategy1)); - Thread addThread2 = new Thread(() -> manager.add(endpoint2, strategy2)); - - addThread1.start(); - addThread2.start(); - - addThread1.join(); - addThread2.join(); - - // Wait for all events to be processed - assertTrue(allEventsLatch.await(3, TimeUnit.SECONDS), "Should receive all health status events"); - - // Verify final states - assertTrue(manager.hasHealthCheck(endpoint1)); - assertTrue(manager.hasHealthCheck(endpoint2)); - - // Clean up - manager.remove(endpoint1); - manager.remove(endpoint2); - } - - @Test - void testNoEventsAreMissedDuringRegistration() throws InterruptedException { - HealthStatusManager manager = new HealthStatusManager(); - - // Track all events received - AtomicInteger eventsReceived = new AtomicInteger(0); - CountDownLatch allEventsLatch = new CountDownLatch(2); // Expect 2 events: UNKNOWN->HEALTHY, HEALTHY->UNHEALTHY - - // Create a strategy that changes status quickly - AtomicInteger checkCount = new AtomicInteger(0); - HealthCheckStrategy rapidChangeStrategy = new HealthCheckStrategy() { - @Override - public int getInterval() { return 50; } // Very fast interval - - @Override - public int getTimeout() { return 25; } - - @Override - public HealthStatus doHealthCheck(Endpoint endpoint) { - int count = checkCount.incrementAndGet(); - // First check: HEALTHY, second check: UNHEALTHY - return count == 1 ? HealthStatus.HEALTHY : HealthStatus.UNHEALTHY; - } - }; - - // Create listener that tracks events - HealthStatusListener eventTracker = event -> { - eventsReceived.incrementAndGet(); - allEventsLatch.countDown(); - System.out.println("Event received: " + event.getOldStatus() + " -> " + event.getNewStatus()); - }; - - // Register listener BEFORE adding endpoint (correct order to prevent missing events) - manager.registerListener(testEndpoint, eventTracker); - - // Add endpoint - this should trigger rapid health checks - manager.add(testEndpoint, rapidChangeStrategy); - - // Wait for all expected events - assertTrue(allEventsLatch.await(3, TimeUnit.SECONDS), - "Should receive all health status change events within timeout"); - - // Verify we received the expected number of events - assertEquals(2, eventsReceived.get(), "Should have received exactly 2 health status change events"); - - // Clean up - manager.remove(testEndpoint); - } - - @Test - void testEventOrderingWithMultipleRapidChanges() throws InterruptedException { - HealthStatusManager manager = new HealthStatusManager(); - - // Track events in order - java.util.List receivedEvents = - java.util.Collections.synchronizedList(new java.util.ArrayList<>()); - CountDownLatch eventsLatch = new CountDownLatch(3); // Expect 3 transitions - - // Create a strategy that cycles through statuses - AtomicInteger checkCount = new AtomicInteger(0); - HealthCheckStrategy cyclingStrategy = new HealthCheckStrategy() { - @Override - public int getInterval() { return 30; } // Very fast - - @Override - public int getTimeout() { return 15; } - - @Override - public HealthStatus doHealthCheck(Endpoint endpoint) { - int count = checkCount.incrementAndGet(); - switch (count) { - case 1: return HealthStatus.HEALTHY; - case 2: return HealthStatus.UNHEALTHY; - case 3: return HealthStatus.HEALTHY; - default: return HealthStatus.HEALTHY; - } - } - }; - - // Register listener to capture events in order - HealthStatusListener orderTracker = event -> { - receivedEvents.add(event); - eventsLatch.countDown(); - }; - - // Register listener BEFORE adding endpoint - manager.registerListener(testEndpoint, orderTracker); - - // Add endpoint to start health checks - manager.add(testEndpoint, cyclingStrategy); - - // Wait for all events - assertTrue(eventsLatch.await(5, TimeUnit.SECONDS), - "Should receive all rapid health status changes"); - - // Verify event sequence - assertEquals(3, receivedEvents.size(), "Should have received 3 events"); - - // Verify the sequence: UNKNOWN->HEALTHY, HEALTHY->UNHEALTHY, UNHEALTHY->HEALTHY - assertEquals(HealthStatus.UNKNOWN, receivedEvents.get(0).getOldStatus()); - assertEquals(HealthStatus.HEALTHY, receivedEvents.get(0).getNewStatus()); - - assertEquals(HealthStatus.HEALTHY, receivedEvents.get(1).getOldStatus()); - assertEquals(HealthStatus.UNHEALTHY, receivedEvents.get(1).getNewStatus()); - - assertEquals(HealthStatus.UNHEALTHY, receivedEvents.get(2).getOldStatus()); - assertEquals(HealthStatus.HEALTHY, receivedEvents.get(2).getNewStatus()); - - // Clean up - manager.remove(testEndpoint); - } -} diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterProviderHealthStatusChangeEventTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterProviderHealthStatusChangeEventTest.java new file mode 100644 index 0000000000..a8ba8bf0f7 --- /dev/null +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterProviderHealthStatusChangeEventTest.java @@ -0,0 +1,438 @@ +package redis.clients.jedis.providers; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.Mockito.*; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.MockedConstruction; +import org.mockito.junit.jupiter.MockitoExtension; + +import redis.clients.jedis.Connection; +import redis.clients.jedis.ConnectionPool; +import redis.clients.jedis.DefaultJedisClientConfig; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisClientConfig; +import redis.clients.jedis.MultiClusterClientConfig; +import redis.clients.jedis.mcf.Endpoint; +import redis.clients.jedis.mcf.HealthCheckStrategy; +import redis.clients.jedis.mcf.HealthStatus; +import redis.clients.jedis.mcf.HealthStatusChangeEvent; +import redis.clients.jedis.mcf.HealthStatusListener; +import redis.clients.jedis.mcf.HealthStatusManager; + +/** + * Tests for MultiClusterPooledConnectionProvider event handling behavior during initialization and throughout its + * lifecycle with HealthStatusChangeEvents. + */ +@ExtendWith(MockitoExtension.class) +public class MultiClusterProviderHealthStatusChangeEventTest { + + private HostAndPort endpoint1; + private HostAndPort endpoint2; + private HostAndPort endpoint3; + private JedisClientConfig clientConfig; + + @BeforeEach + void setUp() { + endpoint1 = new HostAndPort("localhost", 6879); + endpoint2 = new HostAndPort("localhost", 6880); + endpoint3 = new HostAndPort("localhost", 6881); + clientConfig = DefaultJedisClientConfig.builder().build(); + } + + private MockedConstruction mockConnectionPool() { + Connection mockConnection = mock(Connection.class); + lenient().when(mockConnection.ping()).thenReturn(true); + return mockConstruction(ConnectionPool.class, (mock, context) -> { + when(mock.getResource()).thenReturn(mockConnection); + doNothing().when(mock).close(); + }); + } + + @Test + void testProviderInitializationCompletes() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + // Create clusters without health checks (will be assumed healthy) + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Verify initialization completed successfully + Field initCompleteField = MultiClusterPooledConnectionProvider.class + .getDeclaredField("initializationComplete"); + initCompleteField.setAccessible(true); + + assertTrue((Boolean) initCompleteField.get(provider), + "Initialization should be complete after provider construction"); + + // Provider should have selected a cluster successfully + assertNotNull(provider.getCluster(), "Provider should have an active cluster"); + } + } + } + + @Test + void testEventsProcessedAfterInitialization() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + // Create clusters without health checks + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(0.5f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Create a health status change event + HealthStatusChangeEvent event = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, + HealthStatus.UNHEALTHY); + + // Use reflection to call onHealthStatusChange (simulating post-init event) + Method handleMethod = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("onHealthStatusChange", HealthStatusChangeEvent.class); + handleMethod.setAccessible(true); + + // This should process immediately since initialization is complete + assertDoesNotThrow(() -> { + handleMethod.invoke(provider, event); + }, "Post-initialization events should be processed immediately"); + + // Verify the cluster status was updated + assertEquals(HealthStatus.UNHEALTHY, provider.getCluster(endpoint1).getHealthStatus(), + "Cluster health status should be updated after post-init event"); + } + } + } + + @Test + void testMultipleEventsProcessedSequentially() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(0.5f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Get method for handling events + Method handleMethod = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("onHealthStatusChange", HealthStatusChangeEvent.class); + handleMethod.setAccessible(true); + + // Verify initial state + assertEquals(HealthStatus.HEALTHY, provider.getCluster(endpoint1).getHealthStatus(), + "Should start as HEALTHY"); + + // Simulate multiple rapid events for the same endpoint + HealthStatusChangeEvent event1 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, + HealthStatus.UNHEALTHY); + HealthStatusChangeEvent event2 = new HealthStatusChangeEvent(endpoint1, HealthStatus.UNHEALTHY, + HealthStatus.HEALTHY); + HealthStatusChangeEvent event3 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, + HealthStatus.UNHEALTHY); + + // Process events sequentially (post-init behavior) + handleMethod.invoke(provider, event1); + assertEquals(HealthStatus.UNHEALTHY, provider.getCluster(endpoint1).getHealthStatus(), + "Should be UNHEALTHY after first event"); + + handleMethod.invoke(provider, event2); + assertEquals(HealthStatus.HEALTHY, provider.getCluster(endpoint1).getHealthStatus(), + "Should be HEALTHY after second event"); + + handleMethod.invoke(provider, event3); + assertEquals(HealthStatus.UNHEALTHY, provider.getCluster(endpoint1).getHealthStatus(), + "Should be UNHEALTHY after third event"); + } + } + } + + @Test + void testEventsForMultipleEndpointsPreserveOrder() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // This test verifies that multiple endpoints are properly initialized + + // Verify both clusters are initialized properly + assertNotNull(provider.getCluster(endpoint1), "Cluster 1 should be available"); + assertNotNull(provider.getCluster(endpoint2), "Cluster 2 should be available"); + + // Both should be healthy (no health checks = assumed healthy) + assertTrue(provider.getCluster(endpoint1).isHealthy(), "Cluster 1 should be healthy"); + assertTrue(provider.getCluster(endpoint2).isHealthy(), "Cluster 2 should be healthy"); + } + } + } + + @Test + void testInitializationCompleteFlagBehavior() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1 }).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Use reflection to check initialization flag + Field initCompleteField = MultiClusterPooledConnectionProvider.class + .getDeclaredField("initializationComplete"); + initCompleteField.setAccessible(true); + + // After construction, initialization should be complete + assertTrue((Boolean) initCompleteField.get(provider), + "Initialization should be complete after provider construction"); + + // Verify provider is functional + assertNotNull(provider.getCluster(), "Provider should have an active cluster"); + } + } + } + + @Test + void testEventProcessingWithMixedHealthCheckConfiguration() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + // One cluster with health checks disabled, one with enabled (but mocked) + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false) // No health checks + .build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false) // No health checks for + // simplicity + .build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Both clusters should be available and healthy + assertNotNull(provider.getCluster(endpoint1), "Cluster 1 should be available"); + assertNotNull(provider.getCluster(endpoint2), "Cluster 2 should be available"); + + assertTrue(provider.getCluster(endpoint1).isHealthy(), "Cluster 1 should be healthy"); + assertTrue(provider.getCluster(endpoint2).isHealthy(), "Cluster 2 should be healthy"); + + // Provider should select the higher weight cluster (endpoint2) + assertEquals(provider.getCluster(endpoint2), provider.getCluster(), + "Should select higher weight cluster as active"); + } + } + } + + @Test + void testNoEventsLostDuringInitialization() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1 }).build(); + + // This test verifies that the provider initializes correctly and doesn't lose events + // In practice, with health checks disabled, no events should be generated during init + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Verify successful initialization + assertNotNull(provider.getCluster(), "Provider should have initialized successfully"); + assertEquals(provider.getCluster(endpoint1), provider.getCluster(), + "Should have selected the configured cluster"); + assertTrue(provider.getCluster().isHealthy(), + "Cluster should be healthy (assumed healthy with no health checks)"); + } + } + } + + // ========== POST-INITIALIZATION EVENT ORDERING TESTS ========== + + @Test + void testPostInitEventOrderingWithMultipleEndpoints() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(0.5f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster3 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint3, clientConfig).weight(0.2f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2, cluster3 }).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Get method for sending events + + Method handleMethod = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("onHealthStatusChange", HealthStatusChangeEvent.class); + handleMethod.setAccessible(true); + + // Create events in specific order + HealthStatusChangeEvent event1 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, + HealthStatus.UNHEALTHY); + HealthStatusChangeEvent event2 = new HealthStatusChangeEvent(endpoint2, HealthStatus.HEALTHY, + HealthStatus.UNHEALTHY); + HealthStatusChangeEvent event3 = new HealthStatusChangeEvent(endpoint1, HealthStatus.UNHEALTHY, + HealthStatus.HEALTHY); + + // Send events (should be processed immediately since init is complete) + handleMethod.invoke(provider, event1); + handleMethod.invoke(provider, event2); + handleMethod.invoke(provider, event3); + + // Verify events were processed and cluster states updated + assertEquals(HealthStatus.HEALTHY, provider.getCluster(endpoint1).getHealthStatus(), + "Endpoint1 should have final HEALTHY status"); + assertEquals(HealthStatus.UNHEALTHY, provider.getCluster(endpoint2).getHealthStatus(), + "Endpoint2 should have UNHEALTHY status"); + } + } + } + + @Test + void testPostInitRapidEventsOptimization() throws Exception { + try (MockedConstruction mockedPool = mockConnectionPool()) { + MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig + .builder(endpoint2, clientConfig).weight(0.5f).healthCheckEnabled(false).build(); + + MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( + new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); + + try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { + // Verify initial state + assertEquals(HealthStatus.HEALTHY, provider.getCluster(endpoint1).getHealthStatus(), + "Should start as HEALTHY"); + + Method handleMethod = MultiClusterPooledConnectionProvider.class + .getDeclaredMethod("onHealthStatusChange", HealthStatusChangeEvent.class); + handleMethod.setAccessible(true); + + // Send rapid sequence of events (should all be processed since init is complete) + HealthStatusChangeEvent event1 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, + HealthStatus.UNHEALTHY); + HealthStatusChangeEvent event2 = new HealthStatusChangeEvent(endpoint1, HealthStatus.UNHEALTHY, + HealthStatus.HEALTHY); + HealthStatusChangeEvent event3 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, + HealthStatus.UNHEALTHY); + + // Process events immediately (post-init behavior) + handleMethod.invoke(provider, event1); + handleMethod.invoke(provider, event2); + handleMethod.invoke(provider, event3); + + // Final state should reflect the last event + assertEquals(HealthStatus.UNHEALTHY, provider.getCluster(endpoint1).getHealthStatus(), + "Should have final UNHEALTHY status from last event"); + } + } + } + + @Test + void testHealthStatusManagerEventOrdering() throws InterruptedException { + HealthStatusManager manager = new HealthStatusManager(); + + // Counter to track events received + AtomicInteger eventCount = new AtomicInteger(0); + CountDownLatch eventLatch = new CountDownLatch(1); + + // Create a listener that counts events + HealthStatusListener listener = event -> { + eventCount.incrementAndGet(); + eventLatch.countDown(); + }; + + // Register listener BEFORE adding endpoint (correct order to prevent missing events) + manager.registerListener(endpoint1, listener); + + // Create a strategy that immediately returns HEALTHY + HealthCheckStrategy immediateStrategy = new HealthCheckStrategy() { + @Override + public int getInterval() { + return 100; + } + + @Override + public int getTimeout() { + return 50; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + return HealthStatus.HEALTHY; + } + }; + + // Add endpoint - this should trigger health check and event + manager.add(endpoint1, immediateStrategy); + + // Wait for event to be processed + assertTrue(eventLatch.await(2, TimeUnit.SECONDS), "Should receive health status event"); + + // Should have received at least one event (UNKNOWN -> HEALTHY) + assertTrue(eventCount.get() >= 1, "Should have received at least one health status event"); + + manager.remove(endpoint1); + } + + @Test + void testHealthStatusManagerHasHealthCheck() { + HealthStatusManager manager = new HealthStatusManager(); + + // Initially no health check + assertFalse(manager.hasHealthCheck(endpoint1), "Should not have health check initially"); + + // Create a simple strategy + HealthCheckStrategy strategy = new HealthCheckStrategy() { + @Override + public int getInterval() { + return 100; + } + + @Override + public int getTimeout() { + return 50; + } + + @Override + public HealthStatus doHealthCheck(Endpoint endpoint) { + return HealthStatus.HEALTHY; + } + }; + + // Add health check + manager.add(endpoint1, strategy); + assertTrue(manager.hasHealthCheck(endpoint1), "Should have health check after adding"); + + // Remove health check + manager.remove(endpoint1); + assertFalse(manager.hasHealthCheck(endpoint1), "Should not have health check after removing"); + } +} From 1ae7219ea632aec034a44e86e3c5b20b736571fc Mon Sep 17 00:00:00 2001 From: atakavci Date: Thu, 24 Jul 2025 16:03:42 +0300 Subject: [PATCH 22/23] - replace use of reflection with helper methods - fix failing tests due to method name change --- .../MultiClusterPooledConnectionProvider.java | 6 +- .../mcf/FailbackMechanismIntegrationTest.java | 51 +++---- .../jedis/mcf/PeriodicFailbackTest.java | 62 ++------ ...ClusterPooledConnectionProviderHelper.java | 17 +++ ...erProviderHealthStatusChangeEventTest.java | 134 +++--------------- 5 files changed, 73 insertions(+), 197 deletions(-) create mode 100644 src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderHelper.java diff --git a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java index 8efcf3559b..2ee9ccb6fb 100644 --- a/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java +++ b/src/main/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProvider.java @@ -311,7 +311,8 @@ private void addClusterInternal(MultiClusterClientConfig multiClusterClientConfi * Handles health status changes for clusters. This method is called by the health status manager when the health * status of a cluster changes. */ - private void onHealthStatusChange(HealthStatusChangeEvent eventArgs) { + @VisibleForTesting + void onHealthStatusChange(HealthStatusChangeEvent eventArgs) { Endpoint endpoint = eventArgs.getEndpoint(); HealthStatus newStatus = eventArgs.getNewStatus(); log.debug("Health status changed for {} from {} to {}", endpoint, eventArgs.getOldStatus(), newStatus); @@ -382,7 +383,8 @@ private Cluster waitForInitialHealthyCluster() { /** * Periodic failback checker - runs at configured intervals to check for failback opportunities */ - private void periodicFailbackCheck() { + @VisibleForTesting + void periodicFailbackCheck() { try { // Find the best candidate cluster for failback Cluster bestCandidate = null; diff --git a/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java index dc06e5856d..0ea9fa0d7c 100644 --- a/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java @@ -16,7 +16,7 @@ import redis.clients.jedis.JedisClientConfig; import redis.clients.jedis.MultiClusterClientConfig; import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; -import java.lang.reflect.Method; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProviderHelper; @ExtendWith(MockitoExtension.class) class FailbackMechanismIntegrationTest { @@ -43,23 +43,6 @@ private MockedConstruction mockPool() { }); } - /** - * Helper method to trigger health status changes using reflection - */ - private void triggerHealthStatusChange(MultiClusterPooledConnectionProvider provider, HostAndPort endpoint, - HealthStatus oldStatus, HealthStatus newStatus) { - try { - Method handleHealthStatusChange = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("handleHealthStatusChange", HealthStatusChangeEvent.class); - handleHealthStatusChange.setAccessible(true); - - HealthStatusChangeEvent event = new HealthStatusChangeEvent(endpoint, oldStatus, newStatus); - handleHealthStatusChange.invoke(provider, event); - } catch (Exception e) { - throw new RuntimeException("Failed to trigger health status change", e); - } - } - @Test void testFailbackDisabledDoesNotPerformFailback() throws InterruptedException { try (MockedConstruction mockedPool = mockPool()) { @@ -81,13 +64,13 @@ void testFailbackDisabledDoesNotPerformFailback() throws InterruptedException { assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster2 unhealthy to force failover to cluster1 - triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster1 (only healthy option) assertEquals(provider.getCluster(endpoint1), provider.getCluster()); // Make cluster2 healthy again (higher weight - would normally trigger failback) - triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait longer than failback interval Thread.sleep(200); @@ -121,13 +104,13 @@ void testFailbackToHigherWeightCluster() throws InterruptedException { assertEquals(provider.getCluster(endpoint1), provider.getCluster()); // Make cluster1 unhealthy to force failover to cluster2 - triggerHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster2 (lower weight, but only healthy option) assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster1 healthy again - triggerHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for failback check interval + some buffer Thread.sleep(250); @@ -163,14 +146,14 @@ void testNoFailbackToLowerWeightCluster() throws InterruptedException { assertEquals(provider.getCluster(endpoint3), provider.getCluster()); // Make cluster3 unhealthy to force failover to cluster2 (medium weight) - triggerHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster2 (highest weight among healthy clusters) assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster1 (lowest weight) healthy - this should NOT trigger failback // since we don't failback to lower weight clusters - triggerHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for failback check interval Thread.sleep(250); @@ -199,13 +182,13 @@ void testFailbackToHigherWeightClusterImmediately() throws InterruptedException assertEquals(provider.getCluster(endpoint1), provider.getCluster()); // Make cluster1 unhealthy to force failover to cluster2 - triggerHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster2 (only healthy option) assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster1 healthy again - triggerHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for failback check Thread.sleep(150); @@ -234,19 +217,19 @@ void testUnhealthyClusterCancelsFailback() throws InterruptedException { assertEquals(provider.getCluster(endpoint1), provider.getCluster()); // Make cluster1 unhealthy to force failover to cluster2 - triggerHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster2 (only healthy option) assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster1 healthy again (should trigger failback attempt) - triggerHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait a bit Thread.sleep(100); // Make cluster1 unhealthy again before failback completes - triggerHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Wait past the original failback interval Thread.sleep(150); @@ -279,13 +262,13 @@ void testMultipleClusterFailbackPriority() throws InterruptedException { assertEquals(provider.getCluster(endpoint3), provider.getCluster()); // Make cluster3 unhealthy to force failover to cluster2 (next highest weight) - triggerHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster2 (highest weight among healthy clusters) assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster3 healthy again - triggerHealthStatusChange(provider, endpoint3, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint3, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for failback Thread.sleep(200); @@ -315,7 +298,7 @@ void testGracePeriodDisablesClusterOnUnhealthy() throws InterruptedException { assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Now make cluster2 unhealthy - it should be disabled for grace period - triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should failover to cluster1 assertEquals(provider.getCluster(endpoint1), provider.getCluster()); @@ -346,7 +329,7 @@ void testGracePeriodReEnablesClusterAfterPeriod() throws InterruptedException { assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster2 unhealthy to start grace period and force failover - triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should failover to cluster1 assertEquals(provider.getCluster(endpoint1), provider.getCluster()); @@ -355,7 +338,7 @@ void testGracePeriodReEnablesClusterAfterPeriod() throws InterruptedException { assertTrue(provider.getCluster(endpoint2).isInGracePeriod()); // Make cluster2 healthy again while it's still in grace period - triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Should still be on cluster1 because cluster2 is in grace period assertEquals(provider.getCluster(endpoint1), provider.getCluster()); diff --git a/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java b/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java index 2a493704c8..43d336fe8a 100644 --- a/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java +++ b/src/test/java/redis/clients/jedis/mcf/PeriodicFailbackTest.java @@ -3,12 +3,9 @@ import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.*; -import java.lang.reflect.Method; - 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.MockedConstruction; import org.mockito.junit.jupiter.MockitoExtension; @@ -19,6 +16,7 @@ import redis.clients.jedis.JedisClientConfig; import redis.clients.jedis.MultiClusterClientConfig; import redis.clients.jedis.providers.MultiClusterPooledConnectionProvider; +import redis.clients.jedis.providers.MultiClusterPooledConnectionProviderHelper; @ExtendWith(MockitoExtension.class) class PeriodicFailbackTest { @@ -43,38 +41,6 @@ private MockedConstruction mockPool() { }); } - /** - * Helper method to trigger health status changes using reflection - */ - private void triggerHealthStatusChange(MultiClusterPooledConnectionProvider provider, HostAndPort endpoint, - HealthStatus oldStatus, HealthStatus newStatus) { - try { - Method handleHealthStatusChange = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("handleHealthStatusChange", HealthStatusChangeEvent.class); - handleHealthStatusChange.setAccessible(true); - - HealthStatusChangeEvent event = new HealthStatusChangeEvent(endpoint, oldStatus, newStatus); - handleHealthStatusChange.invoke(provider, event); - } catch (Exception e) { - throw new RuntimeException("Failed to trigger health status change", e); - } - } - - /** - * Helper method to trigger periodic failback check using reflection - */ - private void triggerPeriodicFailbackCheck(MultiClusterPooledConnectionProvider provider) { - try { - Method periodicFailbackCheckMethod = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("periodicFailbackCheck"); - periodicFailbackCheckMethod.setAccessible(true); - - periodicFailbackCheckMethod.invoke(provider); - } catch (Exception e) { - throw new RuntimeException("Failed to trigger periodic failback check", e); - } - } - @Test void testPeriodicFailbackCheckWithDisabledCluster() throws InterruptedException { try (MockedConstruction mockedPool = mockPool()) { @@ -100,7 +66,7 @@ void testPeriodicFailbackCheckWithDisabledCluster() throws InterruptedException provider.iterateActiveCluster(); // Manually trigger periodic check - triggerPeriodicFailbackCheck(provider); + MultiClusterPooledConnectionProviderHelper.periodicFailbackCheck(provider); // Should still be on cluster1 (cluster2 is in grace period) assertEquals(provider.getCluster(endpoint1), provider.getCluster()); @@ -126,7 +92,7 @@ void testPeriodicFailbackCheckWithHealthyCluster() throws InterruptedException { assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster2 unhealthy to force failover to cluster1 - triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster1 (cluster2 is in grace period) assertEquals(provider.getCluster(endpoint1), provider.getCluster()); @@ -135,17 +101,17 @@ void testPeriodicFailbackCheckWithHealthyCluster() throws InterruptedException { assertTrue(provider.getCluster(endpoint2).isInGracePeriod()); // Make cluster2 healthy again (but it's still in grace period) - triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Trigger periodic check immediately - should still be on cluster1 - triggerPeriodicFailbackCheck(provider); + MultiClusterPooledConnectionProviderHelper.periodicFailbackCheck(provider); assertEquals(provider.getCluster(endpoint1), provider.getCluster()); // Wait for grace period to expire Thread.sleep(150); // Trigger periodic check after grace period expires - triggerPeriodicFailbackCheck(provider); + MultiClusterPooledConnectionProviderHelper.periodicFailbackCheck(provider); // Should have failed back to cluster2 (higher weight, grace period expired) assertEquals(provider.getCluster(endpoint2), provider.getCluster()); @@ -171,19 +137,19 @@ void testPeriodicFailbackCheckWithFailbackDisabled() throws InterruptedException assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster2 unhealthy to force failover to cluster1 - triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster1 assertEquals(provider.getCluster(endpoint1), provider.getCluster()); // Make cluster2 healthy again - triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for stability period Thread.sleep(100); // Trigger periodic check - triggerPeriodicFailbackCheck(provider); + MultiClusterPooledConnectionProviderHelper.periodicFailbackCheck(provider); // Should still be on cluster1 (failback disabled) assertEquals(provider.getCluster(endpoint1), provider.getCluster()); @@ -215,26 +181,26 @@ void testPeriodicFailbackCheckSelectsHighestWeightCluster() throws InterruptedEx assertEquals(provider.getCluster(endpoint3), provider.getCluster()); // Make cluster3 unhealthy to force failover to cluster2 (next highest weight) - triggerHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint3, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster2 (weight 2.0f, higher than cluster1's 1.0f) assertEquals(provider.getCluster(endpoint2), provider.getCluster()); // Make cluster2 unhealthy to force failover to cluster1 - triggerHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Should now be on cluster1 (only healthy cluster left) assertEquals(provider.getCluster(endpoint1), provider.getCluster()); // Make cluster2 and cluster3 healthy again - triggerHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); - triggerHealthStatusChange(provider, endpoint3, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint3, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for grace period to expire Thread.sleep(150); // Trigger periodic check - triggerPeriodicFailbackCheck(provider); + MultiClusterPooledConnectionProviderHelper.periodicFailbackCheck(provider); // Should have failed back to cluster3 (highest weight, grace period expired) assertEquals(provider.getCluster(endpoint3), provider.getCluster()); diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderHelper.java b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderHelper.java new file mode 100644 index 0000000000..138d524ec0 --- /dev/null +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterPooledConnectionProviderHelper.java @@ -0,0 +1,17 @@ +package redis.clients.jedis.providers; + +import redis.clients.jedis.mcf.Endpoint; +import redis.clients.jedis.mcf.HealthStatus; +import redis.clients.jedis.mcf.HealthStatusChangeEvent; + +public class MultiClusterPooledConnectionProviderHelper { + + public static void onHealthStatusChange(MultiClusterPooledConnectionProvider provider, Endpoint endpoint, + HealthStatus oldStatus, HealthStatus newStatus) { + provider.onHealthStatusChange(new HealthStatusChangeEvent(endpoint, oldStatus, newStatus)); + } + + public static void periodicFailbackCheck(MultiClusterPooledConnectionProvider provider) { + provider.periodicFailbackCheck(); + } +} diff --git a/src/test/java/redis/clients/jedis/providers/MultiClusterProviderHealthStatusChangeEventTest.java b/src/test/java/redis/clients/jedis/providers/MultiClusterProviderHealthStatusChangeEventTest.java index a8ba8bf0f7..cb9993c8c7 100644 --- a/src/test/java/redis/clients/jedis/providers/MultiClusterProviderHealthStatusChangeEventTest.java +++ b/src/test/java/redis/clients/jedis/providers/MultiClusterProviderHealthStatusChangeEventTest.java @@ -3,8 +3,6 @@ import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.*; -import java.lang.reflect.Field; -import java.lang.reflect.Method; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -24,7 +22,6 @@ import redis.clients.jedis.mcf.Endpoint; import redis.clients.jedis.mcf.HealthCheckStrategy; import redis.clients.jedis.mcf.HealthStatus; -import redis.clients.jedis.mcf.HealthStatusChangeEvent; import redis.clients.jedis.mcf.HealthStatusListener; import redis.clients.jedis.mcf.HealthStatusManager; @@ -57,34 +54,6 @@ private MockedConstruction mockConnectionPool() { }); } - @Test - void testProviderInitializationCompletes() throws Exception { - try (MockedConstruction mockedPool = mockConnectionPool()) { - // Create clusters without health checks (will be assumed healthy) - MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig - .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); - - MultiClusterClientConfig.ClusterConfig cluster2 = MultiClusterClientConfig.ClusterConfig - .builder(endpoint2, clientConfig).weight(2.0f).healthCheckEnabled(false).build(); - - MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( - new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); - - try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { - // Verify initialization completed successfully - Field initCompleteField = MultiClusterPooledConnectionProvider.class - .getDeclaredField("initializationComplete"); - initCompleteField.setAccessible(true); - - assertTrue((Boolean) initCompleteField.get(provider), - "Initialization should be complete after provider construction"); - - // Provider should have selected a cluster successfully - assertNotNull(provider.getCluster(), "Provider should have an active cluster"); - } - } - } - @Test void testEventsProcessedAfterInitialization() throws Exception { try (MockedConstruction mockedPool = mockConnectionPool()) { @@ -98,18 +67,10 @@ void testEventsProcessedAfterInitialization() throws Exception { new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { - // Create a health status change event - HealthStatusChangeEvent event = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, - HealthStatus.UNHEALTHY); - - // Use reflection to call onHealthStatusChange (simulating post-init event) - Method handleMethod = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("onHealthStatusChange", HealthStatusChangeEvent.class); - handleMethod.setAccessible(true); - // This should process immediately since initialization is complete assertDoesNotThrow(() -> { - handleMethod.invoke(provider, event); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); }, "Post-initialization events should be processed immediately"); // Verify the cluster status was updated @@ -131,33 +92,24 @@ void testMultipleEventsProcessedSequentially() throws Exception { new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2 }).build(); try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { - // Get method for handling events - Method handleMethod = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("onHealthStatusChange", HealthStatusChangeEvent.class); - handleMethod.setAccessible(true); - // Verify initial state assertEquals(HealthStatus.HEALTHY, provider.getCluster(endpoint1).getHealthStatus(), "Should start as HEALTHY"); // Simulate multiple rapid events for the same endpoint - HealthStatusChangeEvent event1 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, - HealthStatus.UNHEALTHY); - HealthStatusChangeEvent event2 = new HealthStatusChangeEvent(endpoint1, HealthStatus.UNHEALTHY, - HealthStatus.HEALTHY); - HealthStatusChangeEvent event3 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, - HealthStatus.UNHEALTHY); - // Process events sequentially (post-init behavior) - handleMethod.invoke(provider, event1); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); assertEquals(HealthStatus.UNHEALTHY, provider.getCluster(endpoint1).getHealthStatus(), "Should be UNHEALTHY after first event"); - handleMethod.invoke(provider, event2); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); assertEquals(HealthStatus.HEALTHY, provider.getCluster(endpoint1).getHealthStatus(), "Should be HEALTHY after second event"); - handleMethod.invoke(provider, event3); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); assertEquals(HealthStatus.UNHEALTHY, provider.getCluster(endpoint1).getHealthStatus(), "Should be UNHEALTHY after third event"); } @@ -190,31 +142,6 @@ void testEventsForMultipleEndpointsPreserveOrder() throws Exception { } } - @Test - void testInitializationCompleteFlagBehavior() throws Exception { - try (MockedConstruction mockedPool = mockConnectionPool()) { - MultiClusterClientConfig.ClusterConfig cluster1 = MultiClusterClientConfig.ClusterConfig - .builder(endpoint1, clientConfig).weight(1.0f).healthCheckEnabled(false).build(); - - MultiClusterClientConfig config = new MultiClusterClientConfig.Builder( - new MultiClusterClientConfig.ClusterConfig[] { cluster1 }).build(); - - try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { - // Use reflection to check initialization flag - Field initCompleteField = MultiClusterPooledConnectionProvider.class - .getDeclaredField("initializationComplete"); - initCompleteField.setAccessible(true); - - // After construction, initialization should be complete - assertTrue((Boolean) initCompleteField.get(provider), - "Initialization should be complete after provider construction"); - - // Verify provider is functional - assertNotNull(provider.getCluster(), "Provider should have an active cluster"); - } - } - } - @Test void testEventProcessingWithMixedHealthCheckConfiguration() throws Exception { try (MockedConstruction mockedPool = mockConnectionPool()) { @@ -286,24 +213,13 @@ void testPostInitEventOrderingWithMultipleEndpoints() throws Exception { new MultiClusterClientConfig.ClusterConfig[] { cluster1, cluster2, cluster3 }).build(); try (MultiClusterPooledConnectionProvider provider = new MultiClusterPooledConnectionProvider(config)) { - // Get method for sending events - - Method handleMethod = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("onHealthStatusChange", HealthStatusChangeEvent.class); - handleMethod.setAccessible(true); - - // Create events in specific order - HealthStatusChangeEvent event1 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, - HealthStatus.UNHEALTHY); - HealthStatusChangeEvent event2 = new HealthStatusChangeEvent(endpoint2, HealthStatus.HEALTHY, - HealthStatus.UNHEALTHY); - HealthStatusChangeEvent event3 = new HealthStatusChangeEvent(endpoint1, HealthStatus.UNHEALTHY, - HealthStatus.HEALTHY); - - // Send events (should be processed immediately since init is complete) - handleMethod.invoke(provider, event1); - handleMethod.invoke(provider, event2); - handleMethod.invoke(provider, event3); + // Process events immediately (post-init behavior) + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, + HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Verify events were processed and cluster states updated assertEquals(HealthStatus.HEALTHY, provider.getCluster(endpoint1).getHealthStatus(), @@ -331,22 +247,14 @@ void testPostInitRapidEventsOptimization() throws Exception { assertEquals(HealthStatus.HEALTHY, provider.getCluster(endpoint1).getHealthStatus(), "Should start as HEALTHY"); - Method handleMethod = MultiClusterPooledConnectionProvider.class - .getDeclaredMethod("onHealthStatusChange", HealthStatusChangeEvent.class); - handleMethod.setAccessible(true); - // Send rapid sequence of events (should all be processed since init is complete) - HealthStatusChangeEvent event1 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, - HealthStatus.UNHEALTHY); - HealthStatusChangeEvent event2 = new HealthStatusChangeEvent(endpoint1, HealthStatus.UNHEALTHY, - HealthStatus.HEALTHY); - HealthStatusChangeEvent event3 = new HealthStatusChangeEvent(endpoint1, HealthStatus.HEALTHY, - HealthStatus.UNHEALTHY); - // Process events immediately (post-init behavior) - handleMethod.invoke(provider, event1); - handleMethod.invoke(provider, event2); - handleMethod.invoke(provider, event3); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); + MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, + HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Final state should reflect the last event assertEquals(HealthStatus.UNHEALTHY, provider.getCluster(endpoint1).getHealthStatus(), From 004a0227286ebac65b003d7f97edd53737e23449 Mon Sep 17 00:00:00 2001 From: atakavci Date: Mon, 11 Aug 2025 19:58:36 +0300 Subject: [PATCH 23/23] - replace 'sleep' with 'await', feedback from @a-TODO-rov --- .../mcf/FailbackMechanismIntegrationTest.java | 45 +++++++++---------- 1 file changed, 21 insertions(+), 24 deletions(-) diff --git a/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java index 1b3e4502b6..5a02f6dfeb 100644 --- a/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java +++ b/src/test/java/redis/clients/jedis/mcf/FailbackMechanismIntegrationTest.java @@ -1,8 +1,12 @@ package redis.clients.jedis.mcf; +import static org.awaitility.Awaitility.await; import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.*; +import java.time.Duration; + +import org.awaitility.Durations; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -25,6 +29,7 @@ class FailbackMechanismIntegrationTest { private HostAndPort endpoint2; private HostAndPort endpoint3; private JedisClientConfig clientConfig; + private static Duration FIFTY_MILLISECONDS = Duration.ofMillis(50); @BeforeEach void setUp() { @@ -73,10 +78,9 @@ void testFailbackDisabledDoesNotPerformFailback() throws InterruptedException { MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint2, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait longer than failback interval - Thread.sleep(200); - // Should still be on cluster1 since failback is disabled - assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + await().atMost(Durations.FIVE_HUNDRED_MILLISECONDS).pollInterval(FIFTY_MILLISECONDS) + .until(() -> provider.getCluster(endpoint1) == provider.getCluster()); } } } @@ -113,10 +117,9 @@ void testFailbackToHigherWeightCluster() throws InterruptedException { MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for failback check interval + some buffer - Thread.sleep(250); - // Should have failed back to cluster1 (higher weight) - assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + await().atMost(Durations.FIVE_HUNDRED_MILLISECONDS).pollInterval(FIFTY_MILLISECONDS) + .until(() -> provider.getCluster(endpoint1) == provider.getCluster()); } } } @@ -156,10 +159,9 @@ void testNoFailbackToLowerWeightCluster() throws InterruptedException { MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for failback check interval - Thread.sleep(250); - // Should still be on cluster2 (no failback to lower weight cluster1) - assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + await().atMost(Durations.FIVE_HUNDRED_MILLISECONDS).pollInterval(FIFTY_MILLISECONDS) + .until(() -> provider.getCluster(endpoint2) == provider.getCluster()); } } } @@ -191,10 +193,9 @@ void testFailbackToHigherWeightClusterImmediately() throws InterruptedException MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for failback check - Thread.sleep(150); - // Should have failed back to cluster1 immediately (higher weight, no stability period required) - assertEquals(provider.getCluster(endpoint1), provider.getCluster()); + await().atMost(Durations.TWO_HUNDRED_MILLISECONDS).pollInterval(FIFTY_MILLISECONDS) + .until(() -> provider.getCluster(endpoint1) == provider.getCluster()); } } } @@ -232,10 +233,9 @@ void testUnhealthyClusterCancelsFailback() throws InterruptedException { MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint1, HealthStatus.HEALTHY, HealthStatus.UNHEALTHY); // Wait past the original failback interval - Thread.sleep(150); - // Should still be on cluster2 (failback was cancelled due to cluster1 becoming unhealthy) - assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + await().atMost(Durations.TWO_HUNDRED_MILLISECONDS).pollInterval(FIFTY_MILLISECONDS) + .until(() -> provider.getCluster(endpoint2) == provider.getCluster()); } } } @@ -271,10 +271,9 @@ void testMultipleClusterFailbackPriority() throws InterruptedException { MultiClusterPooledConnectionProviderHelper.onHealthStatusChange(provider, endpoint3, HealthStatus.UNHEALTHY, HealthStatus.HEALTHY); // Wait for failback - Thread.sleep(250); - // Should fail back to cluster3 (highest weight) - assertEquals(provider.getCluster(endpoint3), provider.getCluster()); + await().atMost(Durations.FIVE_HUNDRED_MILLISECONDS).pollInterval(FIFTY_MILLISECONDS) + .until(() -> provider.getCluster(endpoint3) == provider.getCluster()); } } } @@ -344,16 +343,14 @@ void testGracePeriodReEnablesClusterAfterPeriod() throws InterruptedException { assertEquals(provider.getCluster(endpoint1), provider.getCluster()); // Wait for grace period to expire - Thread.sleep(150); - // Cluster2 should no longer be in grace period - assertFalse(provider.getCluster(endpoint2).isInGracePeriod()); + await().atMost(Durations.FIVE_HUNDRED_MILLISECONDS).pollInterval(FIFTY_MILLISECONDS) + .until(() -> !provider.getCluster(endpoint2).isInGracePeriod()); // Wait for failback check to run - Thread.sleep(100); - // Should now failback to cluster2 (higher weight) since grace period has expired - assertEquals(provider.getCluster(endpoint2), provider.getCluster()); + await().atMost(Durations.FIVE_HUNDRED_MILLISECONDS).pollInterval(FIFTY_MILLISECONDS) + .until(() -> provider.getCluster(endpoint2) == provider.getCluster()); } } }