-
Notifications
You must be signed in to change notification settings - Fork 874
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
JAVA-3051: Memory leak #1743
base: 4.x
Are you sure you want to change the base?
JAVA-3051: Memory leak #1743
Changes from 15 commits
1deb2f3
cb61eb8
4f9d779
7042177
2f23cae
6fa4cb2
e443882
1588ebd
35eab99
6ed271a
ddd6877
39eb4ac
08ad473
ce108d7
7684201
9a4779a
99c5c5b
86b0e1c
5363a73
36d8dd5
7f15b93
d616d06
cde9bbb
4c2ca83
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -33,6 +33,9 @@ | |
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.cache.CacheBuilder; | ||
import com.datastax.oss.driver.shaded.guava.common.cache.CacheLoader; | ||
import com.datastax.oss.driver.shaded.guava.common.cache.LoadingCache; | ||
import edu.umd.cs.findbugs.annotations.NonNull; | ||
import edu.umd.cs.findbugs.annotations.Nullable; | ||
import java.util.BitSet; | ||
|
@@ -96,14 +99,38 @@ 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<Node, AtomicLongArray> responseTimes = new ConcurrentHashMap<>(); | ||
protected final LoadingCache<Node, AtomicLongArray> responseTimes; | ||
protected final Map<Node, Long> upTimes = new ConcurrentHashMap<>(); | ||
private final boolean avoidSlowReplicas; | ||
|
||
public DefaultLoadBalancingPolicy(@NonNull DriverContext context, @NonNull String profileName) { | ||
super(context, profileName); | ||
this.avoidSlowReplicas = | ||
profile.getBoolean(DefaultDriverOption.LOAD_BALANCING_POLICY_SLOW_AVOIDANCE, true); | ||
CacheLoader<Node, AtomicLongArray> cacheLoader = | ||
new CacheLoader<Node, AtomicLongArray>() { | ||
@Override | ||
public AtomicLongArray load(Node key) throws Exception { | ||
// 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(); | ||
AtomicLongArray array = responseTimes.getIfPresent(key); | ||
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); | ||
SiyaoIsHiding marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} else { | ||
array.set(0, array.get(1)); | ||
array.set(1, now); | ||
} | ||
return array; | ||
} | ||
}; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You actually don't need a LoadingCache here; a simple Cache will do. A LoadingCache is useful when you need to create an entry for a key that's been requested but isn't in the cache yet. That's not what's going on here; your CacheLoader is loading keys from the map, creating a new version if it isn't present and then updating it. You're not doing something to create the NodeResponseRateSample instance beyond calling the constructor if necessary. You can accomplish the exact same thing using just a regular Cache by changing your update logic just a bit: protected void updateResponseTimes(@NonNull Node node) {
try {
responseTimes.get(node, NodeResponseRateSample::new).update();
}
catch (ExecutionException ee) {
LOG.info("[{}] Exception updating node response times: {}", logPrefix, ee);
}
} I note that this approach is even recommended in the Guava Javadoc. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Isn't your version subject to race conditions though. In the loading cache version, There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Honestly I've gone round and round about this one over the past few days. I agree that my impl isn't right but a LoadingCache (or at least a LoadingCache used in the way it's used here) still feels wrong to me. I vastly prefer the approach of the original code which just leverages ConcurrentHashMap.compute() to do the right thing. I think part of the mismatch for me here is that CacheLoader.get() doesn't provide the current value (if any) as an arg so you wind up having to do this weird refresh() dance. I'm assuming part of the reason the key wasn't provided is because that's not really the intended use case for a CacheLoader but I digress. As mentioned ConcurrentHashMap seems to model this case more effectively, and while that class doesn't support weak references directly we can get there indirectly with Guava'a MapMaker: There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To make this more concrete: I got the relevant test case (DefaultLoadBalancingPolicyRequestTrackerTest) with the following changes: protected final ConcurrentMap<Node, NodeResponseRateSample> responseTimes;
protected final Map<Node, Long> upTimes = new ConcurrentHashMap<>();
private final boolean avoidSlowReplicas;
public DefaultLoadBalancingPolicy(@NonNull DriverContext context, @NonNull String profileName) {
super(context, profileName);
this.avoidSlowReplicas =
profile.getBoolean(DefaultDriverOption.LOAD_BALANCING_POLICY_SLOW_AVOIDANCE, true);
this.responseTimes =
new MapMaker().weakKeys().makeMap();
}
...
protected void updateResponseTimes(@NonNull Node node) {
this.responseTimes.compute(node, (k,v) -> v == null ? new NodeResponseRateSample() : v.next());
}
...
// Exposed as protected for unit tests
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 false;
long threshold = now - RESPONSE_COUNT_RESET_INTERVAL_NANOS;
return this.oldest - threshold >= 0;
}
} |
||
this.responseTimes = CacheBuilder.newBuilder().weakKeys().build(cacheLoader); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we should add a RemovalListener here. If a GC happens and response times for a Node are purged, then we'll end up treating that as "insufficient responses" in There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you for your review. Would you please explain more about this? this.responseTimes = CacheBuilder.newBuilder().weakKeys().removalListener(
(RemovalListener<Node, AtomicLongArray>) notification ->
LOG.trace("[{}] Evicting response times for {}: {}",
logPrefix, notification.getKey(), notification.getCause()))
.build(cacheLoader); There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @aratno Hi Abe, thank you for your review. Is there any update? |
||
} | ||
|
||
@NonNull | ||
|
@@ -276,38 +303,23 @@ 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; | ||
AtomicLongArray array = responseTimes.getIfPresent(node); | ||
if (array != null && array.length() == 2) { | ||
long threshold = now - RESPONSE_COUNT_RESET_INTERVAL_NANOS; | ||
long leastRecent = array.get(0); | ||
return leastRecent - threshold < 0; | ||
} else return true; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Style nit: Invert the condition and use an early-return if response rate is insufficient, so you don't have |
||
} | ||
|
||
/** | ||
* 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; | ||
}); | ||
responseTimes.refresh(node); | ||
SiyaoIsHiding marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
protected int getInFlight(@NonNull Node node, @NonNull Session session) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,7 +18,9 @@ | |
package com.datastax.oss.driver.internal.core.metadata; | ||
|
||
import com.datastax.oss.driver.api.core.loadbalancing.NodeDistance; | ||
import java.lang.ref.WeakReference; | ||
import java.util.Objects; | ||
import javax.annotation.Nullable; | ||
import net.jcip.annotations.Immutable; | ||
|
||
/** | ||
|
@@ -29,11 +31,16 @@ | |
@Immutable | ||
public class DistanceEvent { | ||
public final NodeDistance distance; | ||
public final DefaultNode node; | ||
private final WeakReference<DefaultNode> node; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same as discussed in NodeStateEvent; I don't think we need a WeakReference here. |
||
|
||
public DistanceEvent(NodeDistance distance, DefaultNode node) { | ||
this.distance = distance; | ||
this.node = node; | ||
this.node = new WeakReference<>(node); | ||
} | ||
|
||
@Nullable | ||
public DefaultNode getNode() { | ||
return node.get(); | ||
} | ||
|
||
@Override | ||
|
@@ -42,19 +49,19 @@ public boolean equals(Object other) { | |
return true; | ||
} else if (other instanceof DistanceEvent) { | ||
DistanceEvent that = (DistanceEvent) other; | ||
return this.distance == that.distance && Objects.equals(this.node, that.node); | ||
return this.distance == that.distance && Objects.equals(this.getNode(), that.getNode()); | ||
} else { | ||
return false; | ||
} | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(this.distance, this.node); | ||
return Objects.hash(this.distance, this.getNode()); | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return "DistanceEvent(" + distance + ", " + node + ")"; | ||
return "DistanceEvent(" + distance + ", " + this.getNode() + ")"; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,7 +19,9 @@ | |
|
||
import com.datastax.oss.driver.api.core.metadata.NodeState; | ||
import com.datastax.oss.driver.shaded.guava.common.base.Preconditions; | ||
import java.lang.ref.WeakReference; | ||
import java.util.Objects; | ||
import javax.annotation.Nullable; | ||
import net.jcip.annotations.Immutable; | ||
|
||
/** | ||
|
@@ -53,14 +55,19 @@ public static NodeStateEvent removed(DefaultNode node) { | |
*/ | ||
public final NodeState newState; | ||
|
||
public final DefaultNode node; | ||
private final WeakReference<DefaultNode> node; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think we need this change. Events of this type are very short-lived; they exist to communicate information between driver components that don't necessarily know about each other. They aren't stockpiled or stored in any meaningful way. You address the problem that was originally reported by changing the distances map in LoadBalancingPolicyWrapper to use weak refs... it's not at all clear to me that making the events use weak references buys you much on top of that. Perhaps more importantly this change has the potential to make events a lot less useful. The driver uses events to notify components about changes in nodes, but if the actual node affected might not be present what use is the notification? Components that receive events without node information have no choice but to ignore them which means (a) every receiver has to check for null node information and (b) if you just ignore events without node data (which all these receivers appear to do) you'll get a lot more indeterminate behaviour since the presence or absence of node data in events is basically a random value (since it's determined by GC pressure which is essentially random from the perspective of the driver). I note that the null checks referenced in (a) above are a big chunk of what's actually in this PR. If not for this constraint the change set would be a lot smaller. That wouldn't be the worse thing as the presence of all those null checks obscures the more significant changes in at least a moderate way. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @absurdfarce I will try to revert the changes and see whether it still leaks, but: private final Map<Node, DistanceEvent> lastDistanceEvents = new WeakHashMap<>();
private final Map<Node, NodeStateEvent> lastStateEvents = new WeakHashMap<>(); However, both DistanceEvent and NodeStateEvent hold strong references to the associated node
In this case There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Tested out. If There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Confirmed that if we put There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What about this: private final Map<Node, NodeDistance> lastDistanceEvents = new WeakHashMap<>();
private final Map<Node, NodeState> lastStateEvents = new WeakHashMap<>(); |
||
|
||
private NodeStateEvent(NodeState oldState, NodeState newState, DefaultNode node) { | ||
this.node = node; | ||
this.node = new WeakReference<>(node); | ||
this.oldState = oldState; | ||
this.newState = newState; | ||
} | ||
|
||
@Nullable | ||
public DefaultNode getNode() { | ||
return node.get(); | ||
} | ||
|
||
@Override | ||
public boolean equals(Object other) { | ||
if (other == this) { | ||
|
@@ -69,19 +76,19 @@ public boolean equals(Object other) { | |
NodeStateEvent that = (NodeStateEvent) other; | ||
return this.oldState == that.oldState | ||
&& this.newState == that.newState | ||
&& Objects.equals(this.node, that.node); | ||
&& Objects.equals(this.getNode(), that.getNode()); | ||
} else { | ||
return false; | ||
} | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(oldState, newState, node); | ||
return Objects.hash(oldState, newState, this.getNode()); | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return "NodeStateEvent(" + oldState + "=>" + newState + ", " + node + ")"; | ||
return "NodeStateEvent(" + oldState + "=>" + newState + ", " + this.getNode() + ")"; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -173,9 +173,8 @@ protected Timeout newTimeout() { | |
.getTimer() | ||
.newTimeout( | ||
t -> { | ||
if (t.isExpired()) { | ||
clearMetrics(); | ||
} | ||
clearMetrics(); | ||
cancelMetricsExpirationTimeout(); | ||
Comment on lines
+176
to
+177
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What's the reasoning for this change? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is a lambda for timeout. Even after the timeout and lambda triggered, the Timer object is not collected and it still holds a reference to this, until it's canceled. |
||
}, | ||
expireAfter.toNanos(), | ||
TimeUnit.NANOSECONDS); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Style nit: use a separate class for the cache value here, rather than using AtomicLongArray as a generic container. Seems like it can be something like
NodeResponseRateSample
, with methods likeboolean hasSufficientResponses
. I see this was present in the previous implementation, so not a required change for this PR, just something I noticed.