-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-9582: Do not abort transaction in unclean close #8143
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1317,26 +1317,25 @@ public void shouldNotCloseProducerIfFencedOnCloseDuringCleanCloseWithEosEnabled( | |
| } | ||
|
|
||
| @Test | ||
| public void shouldAbortTransactionAndCloseProducerOnUncleanCloseWithEosEnabled() { | ||
| public void shouldCloseProducerOnUncleanCloseWithEosEnabled() { | ||
| task = createStatelessTask(createConfig(true), StreamsConfig.METRICS_LATEST); | ||
| task.initializeTopology(); | ||
|
|
||
| task.close(false, false); | ||
| task = null; | ||
|
|
||
| assertTrue(producer.transactionAborted()); | ||
| assertFalse(producer.transactionInFlight()); | ||
| // Make sure no method call on the producer during an unclean close (such as abort). | ||
| assertTrue(producer.transactionInFlight()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This one in particular might be worth a comment that we're specifically checking because we don't want to call methods on the producer during an unclean close.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sounds good! |
||
| assertTrue(producer.closed()); | ||
| } | ||
|
|
||
| @Test | ||
| public void shouldAbortTransactionAndCloseProducerOnErrorDuringUncleanCloseWithEosEnabled() { | ||
| public void shouldCloseProducerOnErrorDuringUncleanCloseWithEosEnabled() { | ||
| task = createTaskThatThrowsException(true); | ||
| task.initializeTopology(); | ||
|
|
||
| task.close(false, false); | ||
|
|
||
| assertTrue(producer.transactionAborted()); | ||
| assertTrue(producer.closed()); | ||
| } | ||
|
|
||
|
|
@@ -1553,15 +1552,14 @@ public void shouldOnlyCloseFencedProducerOnUncleanClosedWithEosEnabled() { | |
| } | ||
|
|
||
| @Test | ||
| public void shouldAbortTransactionButNotCloseProducerIfFencedOnCloseDuringUncleanCloseWithEosEnabled() { | ||
| public void shouldNotCloseProducerIfFencedOnCloseDuringUncleanCloseWithEosEnabled() { | ||
| task = createStatelessTask(createConfig(true), StreamsConfig.METRICS_LATEST); | ||
| task.initializeTopology(); | ||
| producer.fenceProducerOnClose(); | ||
|
|
||
| task.close(false, false); | ||
| task = null; | ||
|
|
||
| assertTrue(producer.transactionAborted()); | ||
| assertFalse(producer.closed()); | ||
| } | ||
|
|
||
|
|
||
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.
I think there are some callers of
suspendthat would setcleanto false whileisZombieto false as well, e.g.suspendRunningTasksin this case should we still call abortTxn?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.
That call is inside the catch block of
task.suspend. For simplicity, I think calling abortTxn is not very tempting at this point as well.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.
After another thought, I'm pretty sure the purpose of this PR is not to maintain the caller of abortTxn in either mode to simplify our error handling. The other txn mechanism like InitPid will make sure to cleanup the pending transactions or through the txn timeout. However the risk of calling abortTxn during close is much higher than a normal processing loop, at least for today's producer. If the producer is in
FATAL_ERROR, it becomes a bomb for any caller touching on its APIs exceptclose().We could think of doing some
safe abortoperations for producer internally instead of externally, which could be done by either getting a new API or just change the default behavior ofproducer.closeto do the transaction abortion when time allowed.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.
Cool I think I am convinced, let's just ignore the isZombie flag then.