szetszwo commented on code in PR #1023: URL: https://github.com/apache/ratis/pull/1023#discussion_r1466792881
########## ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java: ########## @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.util; + +import org.apache.ratis.util.ReferenceCountedObject.ReferenceCountedObjectImpl; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.List; +import java.util.function.Consumer; + +/** + * A utility to detect leaks from @{@link ReferenceCountedObject}. + */ +final class ReferenceCountedLeakDetector { + private static final Logger LOG = LoggerFactory.getLogger(ReferenceCountedLeakDetector.class); + // Leak detection is turned off by default. + + private static LeakDetectionMode leakDetectionFactory = LeakDetectionMode.NONE; + private static LeakDetector leakDetector = null; + + private ReferenceCountedLeakDetector() { + } + + static synchronized void enableLeakDetector(boolean advanced) { + leakDetector = new LeakDetector("ReferenceCountedObject"); + leakDetectionFactory = advanced ? LeakDetectionMode.ADVANCED : LeakDetectionMode.SIMPLE; + } Review Comment: This will create multiple `leadDetector`s when the method is called multiple times. Let's use a `MemoizedSupplier` for `LeakDetector` and a `AtomicReference` for factory? Then, it don't not require `synchronized`. ```java private static final AtomicReference<Mode> FACTORY = new AtomicReference<>(Mode.NONE); private static final Supplier<LeakDetector> SUPPLIER = MemoizedSupplier.valueOf(() -> new LeakDetector(FACTORY.get().name()).start()); static Factory getFactory() { return FACTORY.get(); } static LeakDetector getLeakDetector() { return SUPPLIER.get(); } ``` ########## ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java: ########## @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Simple general resource leak detector using {@link ReferenceQueue} and {@link java.lang.ref.WeakReference} to + * observe resource object life-cycle and assert proper resource closure before they are GCed. + * + * <p> + * Example usage: + * + * <pre> {@code + * class MyResource implements AutoClosable { + * static final LeakDetector LEAK_DETECTOR = new LeakDetector("MyResource"); + * + * private final UncheckedAutoCloseable leakTracker = LEAK_DETECTOR.track(this, () -> { + * // report leaks, don't refer to the original object (MyResource) here. + * System.out.println("MyResource is not closed before being discarded."); + * }); + * + * @Override + * public void close() { + * // proper resources cleanup... + * // inform tracker that this object is closed properly. + * leakTracker.close(); + * } + * } + * + * }</pre> + */ +public class LeakDetector { + private static final Logger LOG = LoggerFactory.getLogger(LeakDetector.class); + private static final AtomicLong COUNTER = new AtomicLong(); + private final ReferenceQueue<Object> queue = new ReferenceQueue<>(); + private final Set<LeakTracker> allLeaks = Collections.newSetFromMap(new ConcurrentHashMap<>()); + private final String name; + + public LeakDetector(String name) { + this.name = name + COUNTER.getAndIncrement(); + start(); Review Comment: It is not a good idea to start a thread inside a constructor; see https://stackoverflow.com/questions/11834173/why-shouldnt-i-use-thread-start-in-the-constructor-of-my-class ########## ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java: ########## @@ -161,52 +162,72 @@ static <V> ReferenceCountedObject<V> wrap(V value, Runnable retainMethod, Consum Objects.requireNonNull(retainMethod, "retainMethod == null"); Objects.requireNonNull(releaseMethod, "releaseMethod == null"); - return new ReferenceCountedObject<V>() { - private final AtomicInteger count = new AtomicInteger(); + return ReferenceCountedLeakDetector.getLeakDetectionFactory().create(value, retainMethod, releaseMethod); + } - @Override - public V get() { - final int previous = count.get(); - if (previous < 0) { - throw new IllegalStateException("Failed to get: object has already been completely released."); - } else if (previous == 0) { - throw new IllegalStateException("Failed to get: object has not yet been retained."); - } - return value; - } + /** The same as wrap(value, retainMethod, ignored -> releaseMethod.run()). */ + static <V> ReferenceCountedObject<V> wrap(V value, Runnable retainMethod, Runnable releaseMethod) { + return wrap(value, retainMethod, ignored -> releaseMethod.run()); + } - @Override - public V retain() { - // n < 0: exception - // n >= 0: n++ - if (count.getAndUpdate(n -> n < 0? n : n + 1) < 0) { - throw new IllegalStateException("Failed to retain: object has already been completely released."); - } + static void enableLeakDetection() { + ReferenceCountedLeakDetector.enableLeakDetector(false); + } - retainMethod.run(); - return value; - } + static void enableAdvancedLeakDetection() { + ReferenceCountedLeakDetector.enableLeakDetector(true); + } - @Override - public boolean release() { - // n <= 0: exception - // n > 1: n-- - // n == 1: n = -1 - final int previous = count.getAndUpdate(n -> n <= 1? -1: n - 1); - if (previous < 0) { - throw new IllegalStateException("Failed to release: object has already been completely released."); - } else if (previous == 0) { - throw new IllegalStateException("Failed to release: object has not yet been retained."); - } - final boolean completedReleased = previous == 1; - releaseMethod.accept(completedReleased); - return completedReleased; + class ReferenceCountedObjectImpl<V> implements ReferenceCountedObject<V> { Review Comment: Move it and make it private to `ReferenceCountedLeakDetector` since it is only used there. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
