KAFKA-9582: Do not abort transaction in unclean close#8143
KAFKA-9582: Do not abort transaction in unclean close#8143guozhangwang merged 4 commits intoapache:2.5from
Conversation
0985f56 to
f5fa343
Compare
|
|
||
| assertTrue(producer.transactionAborted()); | ||
| assertFalse(producer.transactionInFlight()); | ||
| assertTrue(producer.transactionInFlight()); |
There was a problem hiding this comment.
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.
|
Thanks @abbccdda ! It LGTM. @guozhangwang , do you mind taking a look to confirm that this is what you had in mind? |
|
Ok to test. |
|
The two test failures are just flaky: Both are failing the following 2 tests: and a fixing PR is in review stage: #8094 |
|
cc @mumrah |
|
ok to test |
guozhangwang
left a comment
There was a problem hiding this comment.
I made a pass on the PR and left a question about simply removing isZombie and always skip calling abort txn.
| } | ||
|
|
||
| private void maybeAbortTransactionAndCloseRecordCollector(final boolean isZombie) { | ||
| if (!isZombie) { |
There was a problem hiding this comment.
I think there are some callers of suspend that would set clean to false while isZombie to false as well, e.g. suspendRunningTasks in this case should we still call abortTxn?
There was a problem hiding this comment.
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.
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 except close().
We could think of doing some safe abort operations for producer internally instead of externally, which could be done by either getting a new API or just change the default behavior of producer.close to do the transaction abortion when time allowed.
There was a problem hiding this comment.
Cool I think I am convinced, let's just ignore the isZombie flag then.
|
Merged to 2.5 |
In order to avoid hitting the fatal exception during unclean close, we should avoid calling the abortTransaction() call.
Committer Checklist (excluded from commit message)