Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-15900, KAFKA-18310: fix flaky test testOutdatedCoordinatorAssignment and AbstractCoordinatorTest #18945

Open
wants to merge 1 commit into
base: trunk
Choose a base branch
from

Conversation

FrankYang0529
Copy link
Member

@FrankYang0529 FrankYang0529 commented Feb 18, 2025

Both https://issues.apache.org/jira/browse/KAFKA-15900 and https://issues.apache.org/jira/browse/KAFKA-18310 have similar race condition. These test cases expect JoinGroupResponse and SyncGroupResponse can be finished in main thread. However, after AbstractCoordinator receives JoinGroupResponse, it enables HeartbeatThread. The HeartbeatThread also does poll, so SyncGroupResponse may be handled in it. This makes test flaky. In this PR, we use MockHeartbeatThread for flaky tests, so there is no race condition.


Previous fix strategy

There are two thread do poll. One is the main thread, another is the AbstractCoordinator#HeartbeatThread.

The heartbeat thread is enabled after AbstractCoordinator receives JoinGroupResponse.In this case, we assume the first and second coordinator.poll(time.timer(0)) don't get an active group, so the assignedCount is 1.

However, if there is race condition, the joinFuture is completed by heartbeat thread. The coordinator.poll(time.timer(0)) gets an active group and AbstractCoordinator#onJoinComplete is called twice, so the assignedCount is 2.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@github-actions github-actions bot added consumer tests Test fixes (including flaky tests) clients small Small PRs labels Feb 18, 2025
@FrankYang0529
Copy link
Member Author

@lianetm I have another idea to fix this and #18665 together. Both cases has race condition in HeartbeatThread. How about we add a heartbeat thread supplier argument to AbstractCoordinator? Both cases are unit test. We can provide a mock heartbeat thread in test cases, so the poll action in heartbeat thread doesn't make cases flaky.

@lianetm
Copy link
Member

lianetm commented Feb 18, 2025

Hey @FrankYang0529, makes sense that the HB thread can mess the expectations of the unit test with a race as it will poll the client on it's side too. Sounds sensible to have a mocked HB thread for unit testing then, good idea, let's try it out. Thanks!

@github-actions github-actions bot removed the small Small PRs label Feb 19, 2025
@FrankYang0529 FrankYang0529 force-pushed the KAFKA-15900 branch 2 times, most recently from 274d939 to ae008e3 Compare February 19, 2025 14:28
…nment and AbstractCoordinatorTest

Signed-off-by: PoAn Yang <payang@apache.org>
@FrankYang0529 FrankYang0529 changed the title KAFKA-15900: fix flaky test testOutdatedCoordinatorAssignment KAFKA-15900, KAFKA-18310: fix flaky test testOutdatedCoordinatorAssignment and AbstractCoordinatorTest Feb 20, 2025
@FrankYang0529
Copy link
Member Author

Hi @lianetm, I update the PR to use MockHeartbeatThread. Could you review it when you have time? Thank you.

Copy link
Member

@lianetm lianetm left a comment

Choose a reason for hiding this comment

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

Thanks @FrankYang0529 ! Just some comments to improve on the same direction you have.

} finally {
log.debug("Heartbeat thread has closed");
synchronized (AbstractCoordinator.this) {
Copy link
Member

Choose a reason for hiding this comment

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

Don't we need to keep this synchronization here?

public abstract class AbstractHeartbeatThread extends KafkaThread implements AutoCloseable {
private final AtomicBoolean enabled = new AtomicBoolean(false);
private final AtomicBoolean closed = new AtomicBoolean(false);
private final AtomicReference<RuntimeException> failed = new AtomicReference<>(null);
Copy link
Member

Choose a reason for hiding this comment

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

failed sounds like a boolean but it's the error/failure, should we rename it?

@@ -230,7 +230,8 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
this.interceptors,
config.getBoolean(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED),
config.getString(ConsumerConfig.CLIENT_RACK_CONFIG),
clientTelemetryReporter);
clientTelemetryReporter,
Optional.empty());
Copy link
Member

Choose a reason for hiding this comment

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

we're introducing this param only for some tests, but end up having to pass it empty in lots of cases (this consumer class mainly, and other test files).

So I wonder if it would be a fair trade off in this case to add a another constructor to the ConsumerCoordinator to take this param, but keep also the existing one that does not take it. Seems that we would reduce the scope of the changes and avoid the noise of this empty param when not needed.

import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;

public abstract class AbstractHeartbeatThread extends KafkaThread implements AutoCloseable {
Copy link
Member

Choose a reason for hiding this comment

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

Should we add a java doc to show that this is a wrapper for a KafkaThread that allows to be enabled/disabled (looks like the main thing right?).

And seeing it like this, is it intentionally abstract? No harm in it really if the intention is to express that we don't want to allow instances of it, but caught my attention (vs. non-abstract ~BaseHeartbeatThread)

*/
package org.apache.kafka.clients.consumer.internals;

public class MockHeartbeatThread extends AbstractHeartbeatThread {
Copy link
Member

Choose a reason for hiding this comment

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

do we really need this class or would it be enough to simply use a mock(AbstractHeartbeatThread)? (and verify the calls to enable()/disable() instead of checking the actual value of the boolean. Would that work?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
clients consumer tests Test fixes (including flaky tests)
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants