Skip to content

KAFKA-20363 : server-common - Fix flaky testIdleTimeCallback in KafkaEventQueueTest#21920

Merged
chia7712 merged 2 commits intoapache:trunkfrom
muralibasani:KAFKA-20363
Apr 10, 2026
Merged

KAFKA-20363 : server-common - Fix flaky testIdleTimeCallback in KafkaEventQueueTest#21920
chia7712 merged 2 commits intoapache:trunkfrom
muralibasani:KAFKA-20363

Conversation

@muralibasani
Copy link
Copy Markdown
Contributor

@muralibasani muralibasani commented Mar 31, 2026

https://issues.apache.org/jira/browse/KAFKA-20363

The test had a race condition between Test thread and Queue thread. We
actually wait until queue thread is sleeping/waiting so it guarantees
the order.

Reviewers: junvelop 151982401+junjunclub@users.noreply.github.com,
TaiJuWu tjwu1217@gmail.com, Chia-Ping Tsai chia7712@gmail.com

@github-actions github-actions Bot added triage PRs from the community core Kafka Broker tests Test fixes (including flaky tests) small Small PRs labels Mar 31, 2026
@muralibasani
Copy link
Copy Markdown
Contributor Author

Closing this pr @chia7712 . Sorry, didn't notice the interest comment from Kim.

@chia7712
Copy link
Copy Markdown
Member

chia7712 commented Apr 1, 2026

@muralibasani excuse me. Do we already have a PR for this flaky?

@chia7712
Copy link
Copy Markdown
Member

chia7712 commented Apr 1, 2026

I've seen the comments. Let's move forward with your PR. Even if there's another similar proposal, I don't see it as a problem.

@muralibasani muralibasani reopened this Apr 1, 2026
@muralibasani
Copy link
Copy Markdown
Contributor Author

I've seen the comments. Let's move forward with your PR. Even if there's another similar proposal, I don't see it as a problem.

@chia7712 thankyou. Re-opened this pr again.

@junjunclub
Copy link
Copy Markdown

LGTM! The race condition analysis is spot on — waiting for the queue thread to reach WAITING state before advancing MockTime ensures startIdleMs is captured correctly. I was actually thinking the exact same thing. Nice fix! 👍

// Wait for the queue thread to enter cond.await() before advancing MockTime.
// This ensures startIdleMs is captured before the time advancement.
TestUtils.waitForCondition(
() -> queueThread.getState() == Thread.State.WAITING,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please remove extra indent

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, I see the 'extra' indent is the existing convention in this file. I'll withdraw my comment

// Test 2: Deferred event
// Wait for the queue thread to enter cond.await() before advancing MockTime.
TestUtils.waitForCondition(
() -> queueThread.getState() == Thread.State.WAITING,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@chia7712 shall I remove the extra indents ? Or it's ok to leave it as is ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is ok to leave it as is :)

@TaiJuWu
Copy link
Copy Markdown
Collaborator

TaiJuWu commented Apr 3, 2026

@muralibasani Could help to you re-trigger CI?

@github-actions github-actions Bot removed the triage PRs from the community label Apr 3, 2026
@muralibasani muralibasani changed the title KAFKA-20363 : Fix flaky testIdleTimeCallback in KafkaEventQueueTest KAFKA-20363 : server-common - Fix flaky testIdleTimeCallback in KafkaEventQueueTest Apr 5, 2026
@muralibasani
Copy link
Copy Markdown
Contributor Author

CI looks good. Ok to merge?

@muralibasani muralibasani requested a review from TaiJuWu April 7, 2026 18:27
@muralibasani
Copy link
Copy Markdown
Contributor Author

@TaiJuWu any further review possible. Would be great. Thanks.


// Test 2: Deferred event
// Wait for the queue thread to enter cond.await() before advancing MockTime.
TestUtils.waitForCondition(
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Could we extract a helper method for this waiting?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Indeed, there were 2 occurances. Thanks. Updated.

@muralibasani muralibasani requested a review from chia7712 April 10, 2026 07:43
@muralibasani
Copy link
Copy Markdown
Contributor Author

@chia7712 @TaiJuWu Hope one last review could be good for this. Thank you.

@muralibasani
Copy link
Copy Markdown
Contributor Author

Or because of code freeze, we must wait probably

@chia7712
Copy link
Copy Markdown
Member

Or because of code freeze, we must wait probably

Only the 4.3 branch is frozen; the door to trunk is still open! :)

@muralibasani
Copy link
Copy Markdown
Contributor Author

Or because of code freeze, we must wait probably

Only the 4.3 branch is frozen; the door to trunk is still open! :)

Nice. So my first pr on 4.4.0.

@chia7712 chia7712 merged commit a0446e3 into apache:trunk Apr 10, 2026
24 checks passed
nileshkumar3 pushed a commit to nileshkumar3/kafka that referenced this pull request Apr 15, 2026
…ache#21920)

https://issues.apache.org/jira/browse/KAFKA-20363

The test had a race condition between Test thread and Queue thread.  We
actually wait until queue thread is sleeping/waiting so it guarantees
the order.

Reviewers: junvelop <151982401+junjunclub@users.noreply.github.com>,
 TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

ci-approved core Kafka Broker small Small PRs tests Test fixes (including flaky tests)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants