-
Notifications
You must be signed in to change notification settings - Fork 13.6k
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-14938: Fixing flaky test testConnectorBoundary #13646
Conversation
…xactlyOnceSourceIntegrationTest#testConnectorBoundary
Thanks @sambhav-jain-16 . This seems to have fixed the test as such going by the build but I am still curious as to why it used to fail expecting 100+ but got 72. How would this fix be any different from that i.e if consume can't get 100 messages within the timeout. Won't it still fail? |
@sagarrao12 |
Yes it will fail, but If it was timing out, the method would have thrown an Exception. |
Thanks @sambhav-jain-16
I'm not sure how this is is possible given that we're waiting for Lines 399 to 410 in c6ad151
SourceTask::commitRecord is called MINIMUM_MESSAGES number of times).
Records are "committed" only after the producer transaction is committed successfully - Lines 302 to 332 in c6ad151
|
Yeah I agree with @yashmayya . Moreover this
is not entirely correct i think. I agree what gets thrown in an AssertionError but thats because the number of sourceRecords returned by |
Actually I tried it and it failed for me that's why I started to look into the |
I think I get what is happening. kafka/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java Line 577 in 512fd6e
However while getting the kafka/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java Line 567 in 512fd6e
the value is coming more than the actual consumed records in the last loop and that's why it is stopping prematurely here. kafka/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java Line 568 in 512fd6e
Even if consumer is lagging, the method was designed to consume atleast IMO we should fix the |
Thanks @sambhav-jain-16 . So, one thing that I note is that the |
Hi @vamossagar12, |
I was able to know the reason why the position reported by Transactional producer produces commit messages into the user topic itself which is marked as a "Control" batch. However when these messages are consumed, the consumer deliberately skips these messages and doesn't add them to the returned kafka/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java Lines 214 to 222 in fa7818d
Now the issue with |
Thanks @sambhav-jain-16 , @kirktrue , just to give more context, there are ITs in connect for EOS which rely upon the position() API to assert some messages ebing read. But they seem to have be failing now. What the tests do is to read the end offsets for Topic Partitions and keep consuming messages. The moment the last consumed offset exceeds the end offset, the test exits. Basically this block of code: kafka/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java Lines 552 to 580 in 512fd6e
In this case, the assertion fails because even though the last consumed offsets zooms past the endOffset read upfront, the number of records aren't what we are expecting it to be (even though we have already ensured that those many number of records are produced successfully). We can go ahead and look at other ways of PS: this is the flaky test in question Line 376 in 512fd6e
|
@sambhav-jain-16 thanks for looking into this. I'm prioritizing a few other KIP-related PRs at the moment but will try to make time for this sometime in the next couple of weeks. |
Hi @C0urante, Did you get a chance to take a look at it? TIA |
hi @sambhav-jain-16 @C0urante , here's what i think may be happening,
The test doesn't do Reason why assumption Say r1, ... rY were sent by producer sucessfully in a transaction Since assumption imo flakiness is just due to source connector running in background continuously producing records. |
Thanks @sudeshwasnik , for the assessment. Continuing the line of thought, if you check the Because of this, the However, as you also rightly pointed out, the Could this make the test flaky since the bound on the |
hi @sagarrao12 , small correction in my comment earlier -
it seems that Also, the reason this PR passes the test now is because, it doesn't validate X records are present in topic doesn't help now because connector is running continously, so there |
Thanks for the detailed analysis, everyone! Root causeI believe @sudeshwasnik's latest theory is correct: the Connect runtime invokes Consume-all implementationI haven't been able to find any issues with Consume vs. consume allWith regards to the change in the PR--the concern with consuming a fixed number of records from the topic is that we can potentially see a gap in sequence numbers if the topic has multiple partitions, since we wouldn't be guaranteed to consume records in the same order they were produced (which is why I implemented and used Could we stick with using // the connector aborts some transactions, which causes records that it has emitted (and for which
// SourceTask::commitRecord has been invoked) to be invisible to consumers; we expect the task to
// emit at most 233 records in total before 100 records have been emitted as part of one or more
// committed transactions
connectorHandle.expectedRecords(233);
connectorHandle.expectedCommits(233); (This would replace the existing code here.) Transaction size loggingAlso, as an aside--it was really helpful to know how many records were in each aborted/committed transaction while investigating this test. I tweaked private void maybeDefineTransactionBoundary(SourceRecord record) {
if (context.transactionContext() == null || seqno != nextTransactionBoundary) {
return;
}
long transactionSize = nextTransactionBoundary - priorTransactionBoundary;
// If the transaction boundary ends on an even-numbered offset, abort it
// Otherwise, commit
boolean abort = nextTransactionBoundary % 2 == 0;
calculateNextBoundary();
if (abort) {
log.info("Aborting transaction of {} records", transactionSize);
context.transactionContext().abortTransaction(record);
} else {
log.info("Committing transaction of {} records", transactionSize);
context.transactionContext().commitTransaction(record);
}
}
} And in fact, if we believe this would be useful for all connectors, we could even add this kind of logging to the |
Thanks @C0urante for the response. I have applied the suggested changes. PTAL
I'll create a ticket for the same and attach it in this PR comments' |
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.
Thanks @sambhav-jain-16! LGTM
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Yash Mayya <yash.mayya@gmail.com>, Sudesh Wasnik <swasnik@confluent.io>, Chris Egerton <chrise@aiven.io>
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Yash Mayya <yash.mayya@gmail.com>, Sudesh Wasnik <swasnik@confluent.io>, Chris Egerton <chrise@aiven.io>
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Yash Mayya <yash.mayya@gmail.com>, Sudesh Wasnik <swasnik@confluent.io>, Chris Egerton <chrise@aiven.io>
Fixing
org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest#testConnectorBoundary
test by usingconsume()
in place ofconsumeAll()
since we can specify the minimum records to consume inconsume()
which is sufficient to assert the test.Ran the test successfully ~100 times
More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.
Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.
Committer Checklist (excluded from commit message)