gharris1727 commented on code in PR #16499:
URL: https://github.com/apache/kafka/pull/16499#discussion_r1679658127


##########
core/src/test/java/kafka/test/junit/DetectThreadLeakTest.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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 kafka.test.junit;
+
+import org.apache.kafka.test.TestUtils;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class DetectThreadLeakTest {

Review Comment:
   This is a test that is testing other tests and IMHO is not necessary. You 
should at least call join() to ensure the thread is not leaked from this test 
:^)



##########
core/src/test/java/kafka/test/junit/ClusterTestExtensions.java:
##########
@@ -82,7 +87,21 @@
  * SomeIntegrationTest will be instantiated, lifecycle methods (before/after) 
will be run, and "someTest" will be invoked.
  *
  */
-public class ClusterTestExtensions implements 
TestTemplateInvocationContextProvider {
+public class ClusterTestExtensions implements 
TestTemplateInvocationContextProvider, BeforeEachCallback, AfterEachCallback {
+    private static final String METRICS_METER_TICK_THREAD_PREFIX = 
"metrics-meter-tick-thread";
+    private static final String SCALA_THREAD_PREFIX = "scala-";
+    private static final String FORK_JOIN_POOL_THREAD_PREFIX = "ForkJoinPool";
+    private static final String JUNIT_THREAD_PREFIX = "junit-";
+    private static final String ATTACH_LISTENER_THREAD_PREFIX = "Attach 
Listener";
+    private static final String PROCESS_REAPER_THREAD_PREFIX = "process 
reaper";
+    private static final String RMI_THREAD_PREFIX = "RMI";
+    private static final Set<String> SKIPPED_THREAD_PREFIX = 
Collections.unmodifiableSet(Stream.of(
+            METRICS_METER_TICK_THREAD_PREFIX, SCALA_THREAD_PREFIX, 
FORK_JOIN_POOL_THREAD_PREFIX, JUNIT_THREAD_PREFIX,
+            ATTACH_LISTENER_THREAD_PREFIX, PROCESS_REAPER_THREAD_PREFIX, 
RMI_THREAD_PREFIX)
+            .collect(Collectors.toSet()));
+
+    private DetectThreadLeak detectThreadLeak;

Review Comment:
   Instance fields are an anti-pattern in junit extensions because they don't 
permit the extension to be used in concurrent scenarios. You should use the 
ExtensionContext to store this object per-test.



##########
core/src/test/java/kafka/test/junit/DetectThreadLeak.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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 kafka.test.junit;
+
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+public interface DetectThreadLeak {

Review Comment:
   I like how concise this interface is, very nice!



##########
core/src/test/java/kafka/test/junit/ClusterTestExtensions.java:
##########
@@ -119,7 +138,21 @@ public Stream<TestTemplateInvocationContext> 
provideTestTemplateInvocationContex
         return generatedContexts.stream();
     }
 
+    @Override
+    public void beforeEach(ExtensionContext context) {
+        detectThreadLeak = DetectThreadLeak.of(thread ->
+                SKIPPED_THREAD_PREFIX.stream().noneMatch(prefix -> 
thread.getName().startsWith(prefix)));
+    }
 
+    @Override
+    public void afterEach(ExtensionContext context) throws 
InterruptedException {
+        if (detectThreadLeak == null) {
+            return;
+        }
+        TestUtils.waitForCondition(() -> 
detectThreadLeak.newThreads().isEmpty(),

Review Comment:
   I think this waitForCondition will cover up certain thread leaks where the 
test started shutdown of the thread, but didn't call join() to ensure the 
thread was completely stopped. I think these are leaks that would be good to 
find and fix.



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to