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-5124: autocommit reset earliest fixes race condition #2921
Conversation
Fix can be verified to work and the issue reproduced directly by putting a sleep in the test after the stream thread starts the test like so: private void verifyKTableKTableJoin(final JoinType joinType1,
final JoinType joinType2,
final List<KeyValue<String, String>> expectedResult) throws Exception {
streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, joinType1 + "-" + joinType2 + "-ktable-ktable-join");
streams = prepareTopology(joinType1, joinType2);
streams.start();
TimeUnit.SECONDS.sleep(5L); // makes trunk fail 100% of runs but is green with this fix
final List<KeyValue<String, String>> result = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(
CONSUMER_CONFIG,
OUTPUT,
expectedResult.size());
assertThat(result, equalTo(expectedResult));
} |
Thanks @original-brownbear . I noticed that the default in StreamsConfig.java is also "earliest". : tempConsumerDefaultOverrides.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") I'm wondering why we need to explicitly put it in the config too? Curious to hear your thoughts. Thanks. |
@enothereska I was wondering the same (that's why it took me 1h+ rather than minutes to figure this out) :D it's really simple though :( We use |
Doh! LGTM, thanks! |
cc @mjsax @guozhangwang for checkin |
Can we extend this, and apply to all tests? We have couple of flaky tests... (I actually think, some test to set "earliest" for consumer already -- too bad that we missed to set the config everywhere) |
@mjsax sure on it :) |
Retest this please. |
@mjsax separate PR for the extending or do it here? :) |
Not sure... It's actually good to have a PR per JIRA. However, personally I think, we can do one PR, and mark all other JIRAs we "resolved by"... It's just single line fixes anyway. @enothereska @guozhangwang @dguy WDYT? |
Let's do separate PRs for other tests. |
LGTM. One thing to note that, this is not introduced in this PR but: each time we call I think we can do a follow up PR for fixing the
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Merged to trunk. @original-brownbear could you consider working on the follow-up PR as I mentioned? Thanks!
@guozhangwang sure, will work the above into #2920 tomorrow :) |
…se consumer * Producer and Consumer `close` calls were not handled via `try-with-resources` * `cleanRun` unused field removed * Refactored handling of Consumer configuration in `IntegrationTestUtils` to ensure auto-committing of offsets and starting from `earliest` * As a result reverted #2921 since it's redundant now Author: Armin Braun <me@obrown.io> Reviewers: Matthias J. Sax, Guozhang Wang Closes #2920 from original-brownbear/cleanup-it-utils-closing
Fixes
org.apache.kafka.streams.integration.utils.IntegrationTestUtils#readKeyValues
potentially starting topoll
for stream output after the stream finished sending the test data and hence missing it when working withlatest
offsets.