-
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 all 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,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,14 +100,15 @@ 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 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(); | ||
} | ||
|
||
@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); | ||
} | ||
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 seems to be subject to race conditions to me, although I haven't been able to construct a specific scenario that triggers it. The old implementation computed a result array locally and then returned that; there was no changing of state at the ultimate destination (the old responseTimes map) beyond updating an entry or not doing so. This implementation is doing more than that; we're actually changing the types of elements (this.times changes in size) based on data we receive. It might be okay... but then again I'm also not sure I see a harm in just making this.times be an array of size two from the beginning. 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. As I stated previously, this will race indeed, unless calls to The idea of using an array of size 2 from the beginning could save us one array allocation – why not. |
||
|
||
// 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; | ||
} | ||
} | ||
} |
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); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -82,6 +82,7 @@ public void markReady() { | |
consumer.accept(event); | ||
} | ||
} finally { | ||
recordedEvents.clear(); | ||
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. ReplayingEventFilter works like a buffer. It holds a list (queue) of Events since its state is STARTED, and consumes all of them when its state becomes READY all at once. However, the list of the events is never cleared. They leak strong references for the nodes. |
||
stateLock.writeLock().unlock(); | ||
} | ||
} | ||
|
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.
Not sure if this was discussed already: the new design is undeniably more elegant, but it also allocates one new
NodeResponseRateSample
each time we access the map entry (because the entry value is immutable). In the old design the value was mutable in order to avoid allocations. Are we OK with this change?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.
My expectation is that it wouldn't matter since we're creating (and GCing) pretty small objects and the JVM usually handles that quite well but it's not unreasonable to confirm that.
@SiyaoIsHiding is this something you can test? I suppose if we can demonstrate roughly equivalent memory usage and throughput for a decent size load that would be fine.
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.
I tested with a client app with VisualVM.
Using this branch,
NodeResponseSample
uses up to 5000 live bytes with 170 live objects. The app finished 40000 requests in 154 seconds.Using the current 4.x branch, A lambda in
DefaultLoadBalancingPolicy
uses up to 3600 live bytes with 200 live objects. The only lambda I find there is in theresponseTimes.compute
here. The app finished 40000 requests in 161 seconds.So I think there isn't much performance difference.