RATIS-2433. Cancel transaction in case of failure to append#1382
Open
spacemonkd wants to merge 2 commits intoapache:masterfrom
Open
RATIS-2433. Cancel transaction in case of failure to append#1382spacemonkd wants to merge 2 commits intoapache:masterfrom
spacemonkd wants to merge 2 commits intoapache:masterfrom
Conversation
szetszwo
reviewed
Apr 1, 2026
Contributor
szetszwo
left a comment
There was a problem hiding this comment.
@spacemonkd , thanks for working on this!
Please see the comments inlined and also https://issues.apache.org/jira/secure/attachment/13081527/1382_review.patch
Comment on lines
844
to
+864
| final PendingRequests.Permit unsyncedPermit = unsyncedLeaderState.tryAcquirePendingRequest(request.getMessage()); | ||
| if (unsyncedPermit == null) { | ||
| return getResourceUnavailableReply(request, cacheEntry); | ||
| final ResourceUnavailableException e = new ResourceUnavailableException( | ||
| getMemberId() + ": Failed to acquire a pending write request for " + request); | ||
| cancelTransaction(context, e); | ||
| return cacheEntry.failWithException(e); | ||
| } | ||
|
|
||
| final LeaderStateImpl leaderState; | ||
| final PendingRequest pending; | ||
| LeaderStateImpl leaderState = null; | ||
| PendingRequest pending = null; | ||
| CompletableFuture<RaftClientReply> failure = null; | ||
| Exception cancelException = null; |
Contributor
There was a problem hiding this comment.
We should keep returning immediately in case of a failure. This change make the code harder to read.
Contributor
There was a problem hiding this comment.
Please keep this method and call cancelTransaction inside. We should use it for the other ResourceUnavailableException cases.
Comment on lines
+850
to
851
| cancelTransaction(context, nle); | ||
| return RetryCacheImpl.failWithReply(reply, cacheEntry); |
Contributor
There was a problem hiding this comment.
Let's move the failWithReply method to RaftServerImpl:
private CompletableFuture<RaftClientReply> failWithReply(RaftClientReply reply, CacheEntry entry,
TransactionContextImpl context) {
cancelTransaction(context, reply.getException());
if (entry == null) {
return CompletableFuture.completedFuture(reply);
}
entry.failWithReply(reply);
return entry.getReplyFuture();
}| } | ||
|
|
||
| JavaUtils.attemptRepeatedly(() -> { | ||
| Assertions.assertTrue(numCancelTransaction.get() > 0, |
Contributor
There was a problem hiding this comment.
Could we check the exact number instead of "> 0"?
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
What changes were proposed in this pull request?
Currently in RaftServerImpl#writeAsyncImpl() the client request is added to the pending requests asynchronously.
In between if there is any failure/exception in appendTransactions() then we are not cancelling the transaction. The failure is returned to the client/retry-cache, but the statemachine is not notified.
This can cause partial state in the statemachine.
We should handle this such that in case of exceptions the statemachine is notified via cancelTransaction().
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-2433
How was this patch tested?
Patch was tested via unit tests