Anshul-creator opened a new pull request, #20657: URL: https://github.com/apache/kafka/pull/20657
# [KAFKA-17999] Deflake `DynamicConnectionQuotaTest` by unifying loopback family and stabilizing connection-count waits ### What This PR eliminates nondeterminism in `DynamicConnectionQuotaTest` caused by IPv4/IPv6 resolution differences and timing sensitivity in connection accounting. ### JIRA [KAFKA-17999](https://issues.apache.org/jira/browse/KAFKA-17999) ### Reproduction of flakiness To force the mismatch and make the original test fail locally, you can bias the system toward IPv6 and ensure `"localhost"` resolves to `::1` while the test counts on `127.0.0.1`. **These steps modify `/etc/hosts`; revert instructions are below.** 1. Backup and edit `/etc/hosts` to favor IPv6 localhost ```bash sudo cp /etc/hosts /etc/hosts.bak # Comment out the IPv4 localhost line sudo sed -i 's/^\(127\.0\.0\.1[[:space:]]\+localhost\)/# \1/' /etc/hosts # Ensure the IPv6 localhost line exists grep -qE '^::1[[:space:]]+localhost' /etc/hosts || echo "::1 localhost" | sudo tee -a /etc/hosts ``` 2. Force JVM to prefer IPv6 and disable DNS caching ```bash export JAVA_TOOL_OPTIONS="-Djava.net.preferIPv6Addresses=true -Djava.net.preferIPv4Stack=false -Dsun.net.inetaddr.ttl=0" ``` 3. Run the test repeatedly ```bash pass=0; fail=0 for i in {1..20}; do echo "=== Run $i ===" if (( i % 2 == 0 )); then export JAVA_TOOL_OPTIONS="-Djava.net.preferIPv6Addresses=true -Djava.net.preferIPv4Stack=false -Dsun.net.inetaddr.ttl=0" else export JAVA_TOOL_OPTIONS="-Djava.net.preferIPv4Stack=true -Djava.net.preferIPv6Addresses=false -Dsun.net.inetaddr.ttl=0" fi ./gradlew :core:test \ --tests 'kafka.network.DynamicConnectionQuotaTest.testDynamicConnectionQuota' \ -Pkafka.test.run.flaky=true -PmaxParallelForks=1 -Dorg.gradle.workers.max=1 -Dtest.forkEvery=1 \ --rerun-tasks --no-daemon --no-build-cache \ && pass=$((pass+1)) || fail=$((fail+1)) done echo "Summary: $pass pass / $fail fail" ``` **Observed**: Test fails on every alternate run due to `java.net.SocketException: Broken pipe` when the broker applies the per-IP limit to one literal while the client connects via another, causing unexpected disconnects at the quota boundary. **IMPORTANT (Do at the end, when done verifying the stability of the fix): Revert the host/network changes after reproducing** ```bash # Restore hosts file sudo mv /etc/hosts.bak /etc/hosts # Clear JVM env var unset JAVA_TOOL_OPTIONS # Optional sanity ping -c 1 127.0.0.1 ping6 -c 1 ::1 ``` ### Why this flakes #### Environment-dependent IPv4/IPv6 mismatch The test previously used: - `connectionCount(localAddress)` where `localAddress` was `127.0.0.1` (IPv4), **but** - `connect()` dialed `"localhost"` which may resolve to **either** `127.0.0.1` or `::1` depending on the machine’s `/etc/hosts` and JVM preferences. Because Kafka enforces connection quotas **per literal remote IP**, counting on `127.0.0.1` while connecting via `::1` (or vice versa) breaks the test’s assumptions and leads to intermittent failures ### How (the fix) - Detect the loopback address family the broker actually accepts (`::1` vs `127.0.0.1`) and use that **same literal** everywhere in the test. - Ensure the detection runs after brokers start (lazy value) and is initialized during `setUp()` to avoid first-use side effects before taking connection-count baselines. - Use the detected literal in: - `connect()` (client sockets), - IP-based quota override (`MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG`), - connection accounting (`connectionCount(localAddress)`). ### Verification of stability With the IPv6-favored setup above, the test below now consistently passes because counting, dialing, and per-IP override all use the same detected literal: ```bash pass=0; fail=0 for i in {1..20}; do echo "=== Run $i ===" if (( i % 2 == 0 )); then export JAVA_TOOL_OPTIONS="-Djava.net.preferIPv6Addresses=true -Djava.net.preferIPv4Stack=false -Dsun.net.inetaddr.ttl=0" else export JAVA_TOOL_OPTIONS="-Djava.net.preferIPv4Stack=true -Djava.net.preferIPv6Addresses=false -Dsun.net.inetaddr.ttl=0" fi ./gradlew :core:test \ --tests 'kafka.network.DynamicConnectionQuotaTest.testDynamicConnectionQuota' \ -Pkafka.test.run.flaky=true -PmaxParallelForks=1 -Dorg.gradle.workers.max=1 -Dtest.forkEvery=1 \ --rerun-tasks --no-daemon --no-build-cache \ && pass=$((pass+1)) || fail=$((fail+1)) done echo "Summary: $pass pass / $fail fail" ``` ### Scope - **Test-only change.** - No change to public APIs or runtime behavior. - Preserves the original test semantics and assertions by removing environment dependence and first-use interference. -- 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]
