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-15913: Migrate async consumer tests to mocks #14930

Merged
merged 24 commits into from
Dec 18, 2023

Conversation

lucasbru
Copy link
Member

@lucasbru lucasbru commented Dec 5, 2023

Use mocks to test the AsyncKafkaConsumer

  • Eliminate the use of ConsumerTestBuilder
  • Mock all resources that were previously retrieved via leaking
    the background thread with mockito spys
  • Always use the default constructor of AsyncKafkaConsumer as
    much as possible, inject mocks via factories.
  • Timeouts are mocked directly by timeout exceptions instead
    of waiting for futures to time out.

I did not port the autocommit mocking code, because it was mostly
testing the integration of foreground and background thread (or
making the spy's work which broke during the autocommit on close)
and is currently being reimplemented anyway.

New test runs 10x faster.

Committer Checklist (excluded from commit message)

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

@lucasbru lucasbru changed the title Draft: Migrate async consumer tests to mocks KAFKA-15913: Migrate async consumer tests to mocks Dec 11, 2023
@lucasbru lucasbru self-assigned this Dec 11, 2023
@lucasbru lucasbru requested a review from cadonna December 11, 2023 11:49
@lucasbru lucasbru marked this pull request as ready for review December 11, 2023 11:57
@cadonna cadonna added ctr Consumer Threading Refactor (KIP-848) tests Test fixes (including flaky tests) labels Dec 11, 2023
Copy link
Contributor

@cadonna cadonna 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 PR, @lucasbru !

Here my feedback.

(short) 1,
Errors.NONE).responseBody());
}
private void mockCommitApplicationEventException(Exception ex) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I think calling this method (and its siblings) something like completeCommitApplicationEventExceptionally() (completeCommitApplicationEventSuccessfully() for the good case) would make the code better readable.

assertDoesNotThrow(() -> consumer.commitAsync());
future.completeExceptionally(Errors.FENCED_INSTANCE_ID.exception());
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Isn't here something missing? The next call to consumer.commitAsync() should throw, right? Shouldn't we verify that?

Copy link
Member Author

Choose a reason for hiding this comment

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

Good point. There were a couple of poor tests and I didn't feel like fixing it all, but this one is a good one to fix. Done

verify(applicationEventHandler).add(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class));
}
assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000)));
verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any());
Copy link
Contributor

Choose a reason for hiding this comment

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

I am missing the verification of the return value of committed().

Copy link
Member Author

Choose a reason for hiding this comment

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

Also didn't have it before but I fixed it.

@@ -482,6 +497,47 @@ private void process(final GroupMetadataUpdateEvent event) {
requestManagersSupplier);
}

// auxiliary interface for testing
interface ApplicationEventHandlerSupplier {
Copy link
Contributor

Choose a reason for hiding this comment

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

It is nor clear to me why we need a supplier here. Could you elaborate?

Copy link
Member Author

Choose a reason for hiding this comment

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

I want to inject a custom implementation of the applicationEventHandler. It needs to work for both prod and testing so that we can actually test the code we are running. The instantiation for prod depends on arguments that are created inside the constructor of AsyncKafkaConsumer. If I do not define this supplier, I cannot inject the constructor of the "real" ApplicationEventHandler for prod.

Comment on lines +1594 to +1597
// Visible for testing
SubscriptionState subscriptions() {
return subscriptions;
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Is there a way to avoid this?

Copy link
Member Author

Choose a reason for hiding this comment

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

  • I can add callback to the constructor that registers the SubscriptionState with the calling context (the test)
  • I can add another factory to inject our own instance of SubscriptionState

Copy link
Contributor

@cadonna cadonna left a comment

Choose a reason for hiding this comment

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

LGTM!

We can work on the open test-specific questions in a follow-up PR. Now it is important to get a first version of the spy-free unit tests merged.

Thanks, @lucasbru !

@lucasbru lucasbru merged commit 0cb7d74 into apache:trunk Dec 18, 2023
1 check failed
lucasbru added a commit to lucasbru/kafka that referenced this pull request Dec 19, 2023
Use mocks to test the AsyncKafkaConsumer

Eliminate the use of ConsumerTestBuilder
Mock all resources that were previously retrieved via leaking
the background thread with mockito spys
Always use the default constructor of AsyncKafkaConsumer as
much as possible, inject mocks via factories.
Timeouts are mocked directly by timeout exceptions instead
of waiting for futures to time out.
I did not port the autocommit mocking code, because it was mostly
testing the integration of foreground and background thread (or
making the spy's work which broke during the autocommit on close)
and is currently being reimplemented anyway.

New test runs 10x faster.

Reviewers: Bruno Cadonna <[email protected]>
lucasbru added a commit that referenced this pull request Dec 20, 2023
Use mocks to test the AsyncKafkaConsumer

Eliminate the use of ConsumerTestBuilder
Mock all resources that were previously retrieved via leaking
the background thread with mockito spys
Always use the default constructor of AsyncKafkaConsumer as
much as possible, inject mocks via factories.
Timeouts are mocked directly by timeout exceptions instead
of waiting for futures to time out.
I did not port the autocommit mocking code, because it was mostly
testing the integration of foreground and background thread (or
making the spy's work which broke during the autocommit on close)
and is currently being reimplemented anyway.

New test runs 10x faster.

Reviewers: Bruno Cadonna <[email protected]>
gaurav-narula pushed a commit to gaurav-narula/kafka that referenced this pull request Jan 24, 2024
Use mocks to test the AsyncKafkaConsumer

Eliminate the use of ConsumerTestBuilder
Mock all resources that were previously retrieved via leaking
the background thread with mockito spys
Always use the default constructor of AsyncKafkaConsumer as
much as possible, inject mocks via factories.
Timeouts are mocked directly by timeout exceptions instead
of waiting for futures to time out.
I did not port the autocommit mocking code, because it was mostly
testing the integration of foreground and background thread (or
making the spy's work which broke during the autocommit on close)
and is currently being reimplemented anyway.

New test runs 10x faster.

Reviewers: Bruno Cadonna <[email protected]>
yyu1993 pushed a commit to yyu1993/kafka that referenced this pull request Feb 15, 2024
Use mocks to test the AsyncKafkaConsumer

Eliminate the use of ConsumerTestBuilder
Mock all resources that were previously retrieved via leaking
the background thread with mockito spys
Always use the default constructor of AsyncKafkaConsumer as
much as possible, inject mocks via factories.
Timeouts are mocked directly by timeout exceptions instead
of waiting for futures to time out.
I did not port the autocommit mocking code, because it was mostly
testing the integration of foreground and background thread (or
making the spy's work which broke during the autocommit on close)
and is currently being reimplemented anyway.

New test runs 10x faster.

Reviewers: Bruno Cadonna <[email protected]>
AnatolyPopov pushed a commit to aiven/kafka that referenced this pull request Feb 16, 2024
Use mocks to test the AsyncKafkaConsumer

Eliminate the use of ConsumerTestBuilder
Mock all resources that were previously retrieved via leaking
the background thread with mockito spys
Always use the default constructor of AsyncKafkaConsumer as
much as possible, inject mocks via factories.
Timeouts are mocked directly by timeout exceptions instead
of waiting for futures to time out.
I did not port the autocommit mocking code, because it was mostly
testing the integration of foreground and background thread (or
making the spy's work which broke during the autocommit on close)
and is currently being reimplemented anyway.

New test runs 10x faster.

Reviewers: Bruno Cadonna <[email protected]>
clolov pushed a commit to clolov/kafka that referenced this pull request Apr 5, 2024
Use mocks to test the AsyncKafkaConsumer

Eliminate the use of ConsumerTestBuilder
Mock all resources that were previously retrieved via leaking
the background thread with mockito spys
Always use the default constructor of AsyncKafkaConsumer as
much as possible, inject mocks via factories.
Timeouts are mocked directly by timeout exceptions instead
of waiting for futures to time out.
I did not port the autocommit mocking code, because it was mostly
testing the integration of foreground and background thread (or
making the spy's work which broke during the autocommit on close)
and is currently being reimplemented anyway.

New test runs 10x faster.

Reviewers: Bruno Cadonna <[email protected]>
chia7712 pushed a commit that referenced this pull request Aug 5, 2024
The purpose of this PR is to remove ConsumerTestBuilder.java since it is no longer needed. The following PRs have eliminated the use of ConsumerTestBuilder:
#14930
#16140
#16200
#16312

Reviewers: Chia-Ping Tsai <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
consumer ctr Consumer Threading Refactor (KIP-848) tests Test fixes (including flaky tests)
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants