diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/control/ControlConnection.java b/core/src/main/java/com/datastax/oss/driver/internal/core/control/ControlConnection.java index 5ee9c6e7810..5c29a9b704b 100644 --- a/core/src/main/java/com/datastax/oss/driver/internal/core/control/ControlConnection.java +++ b/core/src/main/java/com/datastax/oss/driver/internal/core/control/ControlConnection.java @@ -253,8 +253,8 @@ private class SingleThreaded { private final Reconnection reconnection; private DriverChannelOptions channelOptions; // The last events received for each node - private final Map lastDistanceEvents = new WeakHashMap<>(); - private final Map lastStateEvents = new WeakHashMap<>(); + private final Map lastNodeDistance = new WeakHashMap<>(); + private final Map lastNodeState = new WeakHashMap<>(); private SingleThreaded(InternalDriverContext context) { this.context = context; @@ -366,8 +366,8 @@ private void connect( .whenCompleteAsync( (channel, error) -> { try { - DistanceEvent lastDistanceEvent = lastDistanceEvents.get(node); - NodeStateEvent lastStateEvent = lastStateEvents.get(node); + NodeDistance lastDistance = lastNodeDistance.get(node); + NodeState lastState = lastNodeState.get(node); if (error != null) { if (closeWasCalled || initFuture.isCancelled()) { onSuccess.run(); // abort, we don't really care about the result @@ -406,8 +406,7 @@ private void connect( channel); channel.forceClose(); onSuccess.run(); - } else if (lastDistanceEvent != null - && lastDistanceEvent.distance == NodeDistance.IGNORED) { + } else if (lastDistance == NodeDistance.IGNORED) { LOG.debug( "[{}] New channel opened ({}) but node became ignored, " + "closing and trying next node", @@ -415,9 +414,9 @@ private void connect( channel); channel.forceClose(); connect(nodes, errors, onSuccess, onFailure); - } else if (lastStateEvent != null - && (lastStateEvent.newState == null /*(removed)*/ - || lastStateEvent.newState == NodeState.FORCED_DOWN)) { + } else if (lastNodeState.containsKey(node) + && (lastState == null /*(removed)*/ + || lastState == NodeState.FORCED_DOWN)) { LOG.debug( "[{}] New channel opened ({}) but node was removed or forced down, " + "closing and trying next node", @@ -534,7 +533,7 @@ private void reconnectNow() { private void onDistanceEvent(DistanceEvent event) { assert adminExecutor.inEventLoop(); - this.lastDistanceEvents.put(event.node, event); + this.lastNodeDistance.put(event.node, event.distance); if (event.distance == NodeDistance.IGNORED && channel != null && !channel.closeFuture().isDone() @@ -549,7 +548,7 @@ private void onDistanceEvent(DistanceEvent event) { private void onStateEvent(NodeStateEvent event) { assert adminExecutor.inEventLoop(); - this.lastStateEvents.put(event.node, event); + this.lastNodeState.put(event.node, event.newState); if ((event.newState == null /*(removed)*/ || event.newState == NodeState.FORCED_DOWN) && channel != null && !channel.closeFuture().isDone() diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicy.java b/core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicy.java index 47edcdfe53e..0f03cbb3643 100644 --- a/core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicy.java +++ b/core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicy.java @@ -33,15 +33,19 @@ import com.datastax.oss.driver.internal.core.util.ArrayUtils; import com.datastax.oss.driver.internal.core.util.collection.QueryPlan; import com.datastax.oss.driver.internal.core.util.collection.SimpleQueryPlan; +import com.datastax.oss.driver.shaded.guava.common.annotations.VisibleForTesting; +import com.datastax.oss.driver.shaded.guava.common.collect.MapMaker; import edu.umd.cs.findbugs.annotations.NonNull; import edu.umd.cs.findbugs.annotations.Nullable; import java.util.BitSet; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Queue; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLongArray; import net.jcip.annotations.ThreadSafe; @@ -96,7 +100,7 @@ public class DefaultLoadBalancingPolicy extends BasicLoadBalancingPolicy impleme private static final int MAX_IN_FLIGHT_THRESHOLD = 10; private static final long RESPONSE_COUNT_RESET_INTERVAL_NANOS = MILLISECONDS.toNanos(200); - protected final Map responseTimes = new ConcurrentHashMap<>(); + protected final ConcurrentMap responseTimes; protected final Map upTimes = new ConcurrentHashMap<>(); private final boolean avoidSlowReplicas; @@ -104,6 +108,7 @@ public DefaultLoadBalancingPolicy(@NonNull DriverContext context, @NonNull Strin super(context, profileName); this.avoidSlowReplicas = profile.getBoolean(DefaultDriverOption.LOAD_BALANCING_POLICY_SLOW_AVOIDANCE, true); + this.responseTimes = new MapMaker().weakKeys().makeMap(); } @NonNull @@ -274,40 +279,19 @@ protected boolean isBusy(@NonNull Node node, @NonNull Session session) { } protected boolean isResponseRateInsufficient(@NonNull Node node, long now) { - // response rate is considered insufficient when less than 2 responses were obtained in - // the past interval delimited by RESPONSE_COUNT_RESET_INTERVAL_NANOS. - if (responseTimes.containsKey(node)) { - AtomicLongArray array = responseTimes.get(node); - if (array.length() == 2) { - long threshold = now - RESPONSE_COUNT_RESET_INTERVAL_NANOS; - long leastRecent = array.get(0); - return leastRecent - threshold < 0; - } - } - return true; + NodeResponseRateSample sample = responseTimes.get(node); + return !(sample == null || sample.hasSufficientResponses(now)); } + /** + * Synchronously updates the response times for the given node. It is synchronous because the + * {@link #DefaultLoadBalancingPolicy(com.datastax.oss.driver.api.core.context.DriverContext, + * java.lang.String) CacheLoader.load} assigned is synchronous. + * + * @param node The node to update. + */ protected void updateResponseTimes(@NonNull Node node) { - responseTimes.compute( - node, - (n, array) -> { - // The array stores at most two timestamps, since we don't need more; - // the first one is always the least recent one, and hence the one to inspect. - long now = nanoTime(); - if (array == null) { - array = new AtomicLongArray(1); - array.set(0, now); - } else if (array.length() == 1) { - long previous = array.get(0); - array = new AtomicLongArray(2); - array.set(0, previous); - array.set(1, now); - } else { - array.set(0, array.get(1)); - array.set(1, now); - } - return array; - }); + this.responseTimes.compute(node, (k, v) -> v == null ? new NodeResponseRateSample() : v.next()); } protected int getInFlight(@NonNull Node node, @NonNull Session session) { @@ -318,4 +302,52 @@ protected int getInFlight(@NonNull Node node, @NonNull Session session) { // processing them). return (pool == null) ? 0 : pool.getInFlight(); } + + protected class NodeResponseRateSample { + + @VisibleForTesting protected final long oldest; + @VisibleForTesting protected final OptionalLong newest; + + private NodeResponseRateSample() { + long now = nanoTime(); + this.oldest = now; + this.newest = OptionalLong.empty(); + } + + private NodeResponseRateSample(long oldestSample) { + this(oldestSample, nanoTime()); + } + + private NodeResponseRateSample(long oldestSample, long newestSample) { + this.oldest = oldestSample; + this.newest = OptionalLong.of(newestSample); + } + + @VisibleForTesting + protected NodeResponseRateSample(AtomicLongArray times) { + assert times.length() >= 1; + this.oldest = times.get(0); + this.newest = (times.length() > 1) ? OptionalLong.of(times.get(1)) : OptionalLong.empty(); + } + + // Our newest sample becomes the oldest in the next generation + private NodeResponseRateSample next() { + return new NodeResponseRateSample(this.getNewestValidSample(), nanoTime()); + } + + // If we have a pair of values return the newest, otherwise we have just one value... so just + // return it + private long getNewestValidSample() { + return this.newest.orElse(this.oldest); + } + + // response rate is considered insufficient when less than 2 responses were obtained in + // the past interval delimited by RESPONSE_COUNT_RESET_INTERVAL_NANOS. + private boolean hasSufficientResponses(long now) { + // If we only have one sample it's an automatic failure + if (!this.newest.isPresent()) return true; + long threshold = now - RESPONSE_COUNT_RESET_INTERVAL_NANOS; + return this.oldest - threshold >= 0; + } + } } diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/metadata/LoadBalancingPolicyWrapper.java b/core/src/main/java/com/datastax/oss/driver/internal/core/metadata/LoadBalancingPolicyWrapper.java index 20d045d4e72..5c8473a3b67 100644 --- a/core/src/main/java/com/datastax/oss/driver/internal/core/metadata/LoadBalancingPolicyWrapper.java +++ b/core/src/main/java/com/datastax/oss/driver/internal/core/metadata/LoadBalancingPolicyWrapper.java @@ -38,6 +38,7 @@ import java.util.Map; import java.util.Queue; import java.util.Set; +import java.util.WeakHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; @@ -105,7 +106,7 @@ public LoadBalancingPolicyWrapper( // Just an alias to make the rest of the code more readable this.policies = reporters.keySet(); - this.distances = new HashMap<>(); + this.distances = new WeakHashMap<>(); this.logPrefix = context.getSessionName(); context.getEventBus().register(NodeStateEvent.class, this::onNodeStateEvent); @@ -172,6 +173,7 @@ private void onNodeStateEvent(NodeStateEvent event) { // once it has gone through the filter private void processNodeStateEvent(NodeStateEvent event) { + DefaultNode node = event.node; switch (stateRef.get()) { case BEFORE_INIT: case DURING_INIT: @@ -181,13 +183,13 @@ private void processNodeStateEvent(NodeStateEvent event) { case RUNNING: for (LoadBalancingPolicy policy : policies) { if (event.newState == NodeState.UP) { - policy.onUp(event.node); + policy.onUp(node); } else if (event.newState == NodeState.DOWN || event.newState == NodeState.FORCED_DOWN) { - policy.onDown(event.node); + policy.onDown(node); } else if (event.newState == NodeState.UNKNOWN) { - policy.onAdd(event.node); + policy.onAdd(node); } else if (event.newState == null) { - policy.onRemove(event.node); + policy.onRemove(node); } else { LOG.warn("[{}] Unsupported event: {}", logPrefix, event); } diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/metrics/AbstractMetricUpdater.java b/core/src/main/java/com/datastax/oss/driver/internal/core/metrics/AbstractMetricUpdater.java index 5e2392a2e7f..3d7dc50a7c0 100644 --- a/core/src/main/java/com/datastax/oss/driver/internal/core/metrics/AbstractMetricUpdater.java +++ b/core/src/main/java/com/datastax/oss/driver/internal/core/metrics/AbstractMetricUpdater.java @@ -173,9 +173,8 @@ protected Timeout newTimeout() { .getTimer() .newTimeout( t -> { - if (t.isExpired()) { - clearMetrics(); - } + clearMetrics(); + cancelMetricsExpirationTimeout(); }, expireAfter.toNanos(), TimeUnit.NANOSECONDS); diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/session/DefaultSession.java b/core/src/main/java/com/datastax/oss/driver/internal/core/session/DefaultSession.java index 6f063ae9a50..b795c30fce7 100644 --- a/core/src/main/java/com/datastax/oss/driver/internal/core/session/DefaultSession.java +++ b/core/src/main/java/com/datastax/oss/driver/internal/core/session/DefaultSession.java @@ -527,14 +527,18 @@ private void notifyListeners() { private void onNodeStateChanged(NodeStateEvent event) { assert adminExecutor.inEventLoop(); - if (event.newState == null) { - context.getNodeStateListener().onRemove(event.node); + DefaultNode node = event.node; + if (node == null) { + LOG.debug( + "[{}] Node for this event was removed, ignoring state change: {}", logPrefix, event); + } else if (event.newState == null) { + context.getNodeStateListener().onRemove(node); } else if (event.oldState == null && event.newState == NodeState.UNKNOWN) { - context.getNodeStateListener().onAdd(event.node); + context.getNodeStateListener().onAdd(node); } else if (event.newState == NodeState.UP) { - context.getNodeStateListener().onUp(event.node); + context.getNodeStateListener().onUp(node); } else if (event.newState == NodeState.DOWN || event.newState == NodeState.FORCED_DOWN) { - context.getNodeStateListener().onDown(event.node); + context.getNodeStateListener().onDown(node); } } diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/util/concurrent/ReplayingEventFilter.java b/core/src/main/java/com/datastax/oss/driver/internal/core/util/concurrent/ReplayingEventFilter.java index 12679db7ff0..27ca1b6ff42 100644 --- a/core/src/main/java/com/datastax/oss/driver/internal/core/util/concurrent/ReplayingEventFilter.java +++ b/core/src/main/java/com/datastax/oss/driver/internal/core/util/concurrent/ReplayingEventFilter.java @@ -82,6 +82,7 @@ public void markReady() { consumer.accept(event); } } finally { + recordedEvents.clear(); stateLock.writeLock().unlock(); } } diff --git a/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyQueryPlanTest.java b/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyQueryPlanTest.java index 6098653bc2e..fff86a1b750 100644 --- a/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyQueryPlanTest.java +++ b/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyQueryPlanTest.java @@ -203,7 +203,10 @@ public void should_prioritize_and_shuffle_3_or_more_replicas_when_first_unhealth given(pool3.getInFlight()).willReturn(0); given(pool5.getInFlight()).willReturn(0); - dsePolicy.responseTimes.put(node1, new AtomicLongArray(new long[] {T0, T0})); // unhealthy + dsePolicy.responseTimes.put( + node1, + dsePolicy + .new NodeResponseRateSample(new AtomicLongArray(new long[] {T0, T0}))); // unhealthy // When Queue plan1 = dsePolicy.newQueryPlan(request, session); @@ -232,7 +235,9 @@ public void should_prioritize_and_shuffle_3_or_more_replicas_when_first_unhealth given(pool3.getInFlight()).willReturn(0); given(pool5.getInFlight()).willReturn(0); - dsePolicy.responseTimes.put(node1, new AtomicLongArray(new long[] {T1, T1})); // healthy + dsePolicy.responseTimes.put( + node1, + dsePolicy.new NodeResponseRateSample(new AtomicLongArray(new long[] {T1, T1}))); // healthy // When Queue plan1 = dsePolicy.newQueryPlan(request, session); diff --git a/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyRequestTrackerTest.java b/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyRequestTrackerTest.java index bcc6439a2a5..757af43ef67 100644 --- a/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyRequestTrackerTest.java +++ b/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyRequestTrackerTest.java @@ -69,11 +69,11 @@ public void should_record_first_response_time_on_node_success() { // Then assertThat(policy.responseTimes) - .hasEntrySatisfying(node1, value -> assertThat(value.get(0)).isEqualTo(123L)) + .hasEntrySatisfying(node1, value -> assertThat(value.oldest).isEqualTo(123L)) .doesNotContainKeys(node2, node3); - assertThat(policy.isResponseRateInsufficient(node1, nextNanoTime)).isTrue(); - assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isTrue(); - assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isTrue(); + assertThat(policy.isResponseRateInsufficient(node1, nextNanoTime)).isFalse(); + assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isFalse(); + assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isFalse(); } @Test @@ -91,13 +91,13 @@ public void should_record_second_response_time_on_node_success() { node1, value -> { // oldest value first - assertThat(value.get(0)).isEqualTo(123); - assertThat(value.get(1)).isEqualTo(456); + assertThat(value.oldest).isEqualTo(123); + assertThat(value.newest.getAsLong()).isEqualTo(456); }) .doesNotContainKeys(node2, node3); assertThat(policy.isResponseRateInsufficient(node1, nextNanoTime)).isFalse(); - assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isTrue(); - assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isTrue(); + assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isFalse(); + assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isFalse(); } @Test @@ -116,14 +116,14 @@ public void should_record_further_response_times_on_node_success() { node1, value -> { // values should rotate left (bubble up) - assertThat(value.get(0)).isEqualTo(456); - assertThat(value.get(1)).isEqualTo(789); + assertThat(value.oldest).isEqualTo(456); + assertThat(value.newest.getAsLong()).isEqualTo(789); }) - .hasEntrySatisfying(node2, value -> assertThat(value.get(0)).isEqualTo(789)) + .hasEntrySatisfying(node2, value -> assertThat(value.oldest).isEqualTo(789)) .doesNotContainKey(node3); assertThat(policy.isResponseRateInsufficient(node1, nextNanoTime)).isFalse(); - assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isTrue(); - assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isTrue(); + assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isFalse(); + assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isFalse(); } @Test @@ -137,11 +137,11 @@ public void should_record_first_response_time_on_node_error() { // Then assertThat(policy.responseTimes) - .hasEntrySatisfying(node1, value -> assertThat(value.get(0)).isEqualTo(123L)) + .hasEntrySatisfying(node1, value -> assertThat(value.oldest).isEqualTo(123L)) .doesNotContainKeys(node2, node3); - assertThat(policy.isResponseRateInsufficient(node1, nextNanoTime)).isTrue(); - assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isTrue(); - assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isTrue(); + assertThat(policy.isResponseRateInsufficient(node1, nextNanoTime)).isFalse(); + assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isFalse(); + assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isFalse(); } @Test @@ -160,13 +160,13 @@ public void should_record_second_response_time_on_node_error() { node1, value -> { // oldest value first - assertThat(value.get(0)).isEqualTo(123); - assertThat(value.get(1)).isEqualTo(456); + assertThat(value.oldest).isEqualTo(123); + assertThat(value.newest.getAsLong()).isEqualTo(456); }) .doesNotContainKeys(node2, node3); assertThat(policy.isResponseRateInsufficient(node1, nextNanoTime)).isFalse(); - assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isTrue(); - assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isTrue(); + assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isFalse(); + assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isFalse(); } @Test @@ -186,13 +186,13 @@ public void should_record_further_response_times_on_node_error() { node1, value -> { // values should rotate left (bubble up) - assertThat(value.get(0)).isEqualTo(456); - assertThat(value.get(1)).isEqualTo(789); + assertThat(value.oldest).isEqualTo(456); + assertThat(value.newest.getAsLong()).isEqualTo(789); }) - .hasEntrySatisfying(node2, value -> assertThat(value.get(0)).isEqualTo(789)) + .hasEntrySatisfying(node2, value -> assertThat(value.oldest).isEqualTo(789)) .doesNotContainKey(node3); assertThat(policy.isResponseRateInsufficient(node1, nextNanoTime)).isFalse(); - assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isTrue(); - assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isTrue(); + assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isFalse(); + assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isFalse(); } }