divijvaidya opened a new pull request, #13956: URL: https://github.com/apache/kafka/pull/13956
Sometimes, we see a large number of tests suddenly fail in CI. I encountered at least one case where this could be explained. Test run: https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13953/2 ### why do we have cascading failures? Sequence of events: 1. Some test leaks (does not close properly) the thread `controller-event-thread` 2. This leak is caught at `@AfterAll` method of QuorumTestHarnessScala at https://github.com/apache/kafka/blob/701f924352da1225a881f0f78f19ddf51485030a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala#L446 3. The above leads to an assertion failure but does not kill the threads. 4. Other tests that run after it and before execution check for existing threads all fail due to these leaked threads at https://github.com/apache/kafka/blob/701f924352da1225a881f0f78f19ddf51485030a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala#L438 5. This leads to a cascading failure ### what specifically failed in this scenario In the scenario that I looked into, the following things occurred: 1. A ConsumerBounceTest#testConsumptionWithBrokerFailures() fails for unknown reason. I have created a JIRA for it - https://issues.apache.org/jira/browse/KAFKA-15146 ``` Gradle Test Run :core:integrationTest > Gradle Test Executor 177 > ConsumerBounceTest > testConsumptionWithBrokerFailures() FAILED org.apache.kafka.clients.consumer.CommitFailedException: Offset commit cannot be completed since the consumer is not part of an active group for auto partition assignment; it is likely that the consumer was kicked out of the group. at app//org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.sendOffsetCommitRequest(ConsumerCoordinator.java:1351) at app//org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsSync(ConsumerCoordinator.java:1188) at app//org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1518) at app//org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1417) at app//org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1374) at app//kafka.api.ConsumerBounceTest.consumeWithBrokerFailures(ConsumerBounceTest.scala:109) at app//kafka.api.ConsumerBounceTest.testConsumptionWithBrokerFailures(ConsumerBounceTest.scala:81) ``` 2. This test has a bug which leads to leak of `daemon-bounce-broker` thread. This leaked thread along with ``` Gradle Test Run :core:integrationTest > Gradle Test Executor 177 > ConsumerBounceTest > executionError FAILED org.opentest4j.AssertionFailedError: Found 3 unexpected threads during @AfterAll: `controller-event-thread,daemon-bounce-broker-EventThread,Test worker-EventThread` ==> expected: <true> but was: <false> at app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151) ``` ### what does this PR change? This changes fixes the leak of `daemon-bounce-broker` thread. It **does not** fix the leak of `controller-event-thread` which is the root cause of failure in `@AfterAll`. Hence, although this will not fix the flaky problems, but it will be a step in the right direction to prevent thread leaks from impacting tests. ### Note to reviewers I would suggest to use "hide whitespaces" while reviewing since most of the changes in this PR are simply adding code inside try/finally block. -- 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