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-16460: New consumer times out consuming records in multiple consumer_test.py system tests #17777

Merged
merged 1 commit into from
Nov 15, 2024

Conversation

FrankYang0529
Copy link
Member

@FrankYang0529 FrankYang0529 commented Nov 12, 2024

When a consumer receives group heartbeat with revoked partitions. If the consumer enable auto commit, it will send a OffsetCommitRequest before removing revoked partitions from subscription state. There are two potential race conditions in AsyncKafkaConsumer:

  • If there is data in FetchBuffer from previous FetchRequest, users can still get data from poll function. However, the auto commit function doesn't work for these data, because the consumer mark revoked partitions with pendingRevocation=true after we receives OffsetCommitResponse. At that time, there is no further OffsetCommitRequest will be sent by auto commit feature.
  • The FetchRequestManager still can send new FetchRequest before the consumer receives OffsetCommitResponse. The reason is same as above.

To fix this, we have to mark pendingRevocation=true before the consumer sends last OffsetCommitRequest, so FetchCollector can't get revoked partition data from FetchBuffer and FetchRequestManager doesn't send new FetchReqeust with revoked partitions.

I run consumer_test.py.test_broker_failure 10 times without error.

> TC_PATHS="tests/kafkatest/tests/client/consumer_test.py::OffsetValidationTest.test_broker_failure" /bin/bash tests/docker/run_tests.sh
================================================================================
SESSION REPORT (ALL TESTS)
ducktape version: 0.12.0
session_id:       2024-11-15--010
run time:         6 minutes 57.245 seconds
tests run:        12
passed:           12
flaky:            0
failed:           0
ignored:          0
================================================================================

Committer Checklist (excluded from commit message)

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

@kirktrue kirktrue added KIP-848 The Next Generation of the Consumer Rebalance Protocol ctr Consumer Threading Refactor (KIP-848) labels Nov 14, 2024
@FrankYang0529 FrankYang0529 changed the title KAFKA-16460: New consumer times out consuming records in multiple consumer_test.py system tests (WIP) KAFKA-16460: New consumer times out consuming records in multiple consumer_test.py system tests Nov 15, 2024
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.

Nice catch @FrankYang0529 ! This makes sense to me. It was actually the initial intention
https://github.com/lianetm/kafka/blob/e227a9cac887a201eed9105ed47337fb35acb0ae/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java#L872-L886

I guess we lost it with a refactoring and didn't have test coverage for that corner case.

// Mark partitions as pending revocation to stop fetching from the partitions (no new
// fetches sent out, and no in-flight fetches responses processed).
System.out.println("revokedPartitions = " + revokedPartitions);
markPendingRevocationToPauseFetching(revokedPartitions);
Copy link
Member

@lianetm lianetm Nov 15, 2024

Choose a reason for hiding this comment

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

do we still need the other call now that we have this at the beginning of the reconciliation before commits?

markPendingRevocationToPauseFetching(revokedPartitions);

Copy link
Member Author

@FrankYang0529 FrankYang0529 Nov 15, 2024

Choose a reason for hiding this comment

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

I think it's better to keep markPendingRevocationToPauseFetching in AbstractMembershipMember#revokePartitions, because ConsumerMembershipManager#invokeOnPartitionsRevokedOrLostToReleaseAssignment also uses the function.

Although it's redundant for consumer heartbeat response path, it's still required for ConsumerMembershipManager#signalMemberLeavingGroup path.

Copy link
Member

Choose a reason for hiding this comment

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

makes sense.

// Mark partitions as pending revocation to stop fetching from the partitions (no new
// fetches sent out, and no in-flight fetches responses processed).
System.out.println("revokedPartitions = " + revokedPartitions);
markPendingRevocationToPauseFetching(revokedPartitions);
Copy link
Member

Choose a reason for hiding this comment

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

makes sense.

@@ -1456,6 +1457,7 @@ public void testReconcilePartitionsRevokedWithSuccessfulAutoCommitNoCallbacks()

// Complete commit request
commitResult.complete(null);
verify(subscriptionState, times(2)).markPendingRevocation(Set.of(new TopicPartition("topic1", 0)));
Copy link
Member

Choose a reason for hiding this comment

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

could we extend this test to ensure that the markPending happens before the commit is triggered? (Maybe using InOrder like I'm doing here)

InOrder inOrder = inOrder(backgroundEventHandler, membershipManager);
inOrder.verify(backgroundEventHandler).add(any(ErrorEvent.class));
inOrder.verify(membershipManager).onHeartbeatFailure(false);

Copy link
Member Author

Choose a reason for hiding this comment

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

Updated it. Thanks for the suggestion.

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 for the good catch and improved test coverage @FrankYang0529! LGTM.

@lianetm lianetm merged commit 5725a51 into apache:trunk Nov 15, 2024
8 checks passed
@FrankYang0529 FrankYang0529 deleted the KAFKA-16460 branch November 16, 2024 00:47
chiacyu pushed a commit to chiacyu/kafka that referenced this pull request Nov 30, 2024
tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
clients consumer ctr Consumer Threading Refactor (KIP-848) KIP-848 The Next Generation of the Consumer Rebalance Protocol small Small PRs
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants