Skip to content

Commit

Permalink
Fix a bug and checkstyle.
Browse files Browse the repository at this point in the history
  • Loading branch information
szetszwo committed Sep 28, 2024
1 parent 0d2a4dc commit 205c720
Show file tree
Hide file tree
Showing 2 changed files with 29 additions and 32 deletions.
16 changes: 5 additions & 11 deletions ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
import java.util.function.Predicate;

/**
* Simple general resource leak detector using {@link ReferenceQueue} and {@link java.lang.ref.WeakReference} to
Expand Down Expand Up @@ -96,7 +95,7 @@ private String allLeaksString() {
private final LeakTrackerSet allLeaks = new LeakTrackerSet();
private final String name;

public LeakDetector(String name) {
LeakDetector(String name) {
this.name = name + COUNTER.getAndIncrement();
}

Expand Down Expand Up @@ -128,12 +127,12 @@ private void run() {
LOG.warn("Exiting leak detector {}.", name);
}

public Predicate<ReferenceCountedObject<?>> track(Object leakable, Runnable reportLeak) {
Runnable track(Object leakable, Runnable reportLeak) {
// A rate filter can be put here to only track a subset of all objects, e.g. 5%, 10%,
// if we have proofs that leak tracking impacts performance, or a single LeakDetector
// thread can't keep up with the pace of object allocation.
// For now, it looks effective enough and let keep it simple.
return allLeaks.add(leakable, queue, reportLeak)::releaseAndCheckRemove;
return allLeaks.add(leakable, queue, reportLeak)::remove;
}

public void assertNoLeaks() {
Expand All @@ -154,13 +153,8 @@ private static final class LeakTracker extends WeakReference<Object> {
/**
* Called by the tracked resource when releasing the object.
*/
boolean releaseAndCheckRemove(ReferenceCountedObject<?> referenceCountedObject) {
if (referenceCountedObject.release()) {
removeMethod.accept(this);
return true;
} else {
return false;
}
void remove() {
removeMethod.accept(this);
}

void reportLeak() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.Predicate;
import java.util.function.Supplier;

/**
Expand Down Expand Up @@ -143,30 +142,38 @@ public boolean release() {
}

private static class SimpleTracing<T> extends Impl<T> {
final LeakDetector leakDetector;
final Class<?> clazz;
private final LeakDetector leakDetector;
private final Class<?> valueClass;

Predicate<ReferenceCountedObject<?>> releaseAndCheckLeak = null;
private Runnable removeMethod = null;

SimpleTracing(T value, Runnable retainMethod, Consumer<Boolean> releaseMethod, LeakDetector leakDetector) {
super(value, retainMethod, releaseMethod);
this.clazz = value.getClass();
this.valueClass = value.getClass();
this.leakDetector = leakDetector;
}

void logLeakMessage(Class<?> clazz) {
LOG.warn("LEAK: A {} is not released properly", clazz.getName());
}

@Override
public synchronized T retain() {
if (getCount() == 0) {
this.releaseAndCheckLeak = leakDetector.track(this,
() -> LOG.warn("LEAK: A {} is not released properly", clazz.getName()));
this.removeMethod = leakDetector.track(this, () -> logLeakMessage(valueClass));
}
return super.retain();
}

@Override
public synchronized boolean release() {
Preconditions.assertNotNull(releaseAndCheckLeak != null, () -> "Not yet retained: " + clazz);
return releaseAndCheckLeak.test(this);
Preconditions.assertNotNull(removeMethod != null, () -> "Not yet retained: " + valueClass);
if (super.release()) {
removeMethod.run();
return true;
} else {
return false;
}
}
}

Expand All @@ -180,18 +187,14 @@ private static class AdvancedTracing<T> extends SimpleTracing<T> {
}

@Override
public synchronized T retain() {
if (getCount() == 0) {
this.releaseAndCheckLeak = leakDetector.track(this, () ->
LOG.warn("LEAK: A {} is not released properly.\n"
+ " Creation trace: {}\n"
+ " Retain traces({}): {}\n"
+ " Release traces({}): {}",
clazz.getName(), formatStackTrace(createStrace, 3),
retainsTraces.size(), formatStackTraces(retainsTraces, 2),
releaseTraces.size(), formatStackTraces(releaseTraces, 2)));
}
return super.retain();
void logLeakMessage(Class<?> clazz) {
LOG.warn("LEAK: A {} is not released properly.\n"
+ " Creation trace: {}\n"
+ " Retain traces({}): {}\n"
+ " Release traces({}): {}",
clazz.getName(), formatStackTrace(createStrace, 3),
retainsTraces.size(), formatStackTraces(retainsTraces, 2),
releaseTraces.size(), formatStackTraces(releaseTraces, 2));
}

@Override
Expand Down

0 comments on commit 205c720

Please sign in to comment.