IGNITE-27947 Fix rollback for client piggyback tx#7844
IGNITE-27947 Fix rollback for client piggyback tx#7844tmgodinho wants to merge 11 commits intoapache:mainfrom
Conversation
** Code should be very similar to ClientTable
* The first request is tracked for direct transactions. * The first request id is passed on the Rollback message using the negative range of the resourceId field.
|
This should be a simpler implementation than the one on the previous approach. |
| metrics.requestsFailedIncrement(); | ||
| } | ||
|
|
||
| firstReqToTxResMap.remove(requestId); |
There was a problem hiding this comment.
Here we remove the mapping once the response is sent, but the client might have requested cancellation just before it received our response:
client: send request
server: firstReqToTxResMap.add
server: send response, `firstReqToTxResMap.remove`
client: cancel operation
server: returns an error?
Can we remove the firstReqToTxResMap later? For example, when the tx is cleaned up (removed from the resource registry)?
This should simplify client-side logic too, we can always rollback using the first request id.
There was a problem hiding this comment.
Hi Pavel,
Yeah, you are right.
The rollback message using the firstReqId may already be waiting to be processed when we remove here.
In this scenario, the server would respond with an error, but we are not interested in that response.
That's why we also attach to the actual transaction future from the server and execute the rollback using the normal method if necessary. If we received the first request response concurrently, we just resend the rollback using the normal way, even if the first rollback was already sent.
But I forgot to add this scenario to the tests. I had it in the first version but forgot to port it to this one.
Delaying the removal of the mapping is also possible, at the expense of slightly more complexity on the server-side. I thought about it during the impl. On the client-side it would probably be simpler as you said, since we would rely on the server response for the first rollback message.
I'll add the test and implement this to see how it looks.
** Postpone removal of firstReqToTx resource to after the transaction is commited/rolled back. ** Added test
.../java/org/apache/ignite/client/handler/requests/table/DirectTransactionWithFirstRequest.java
Outdated
Show resolved
Hide resolved
...ient-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
Outdated
Show resolved
Hide resolved
| tx.recordOperationFailure(err); | ||
| } | ||
|
|
||
| // We should reconcile this code with ClientTable. Should be the same. |
There was a problem hiding this comment.
Can we do that as part of current PR?
There was a problem hiding this comment.
Yes, I also think it would be best.
| public CompletableFuture<Void> rollbackAsync() { | ||
| var tx0 = tx; | ||
|
|
||
| // This is really fishy. It will probably let you reuse a transaction after calling a rollback :( |
There was a problem hiding this comment.
Good catch. Let's create a ticket to address this.
| if (!tx0.isDone() && cancelled.compareAndSet(false, true)) { | ||
| return requestInfoFuture | ||
| .thenCompose(reqInfo -> | ||
| reqInfo.ch.serviceAsync(ClientOp.TX_ROLLBACK, w -> w.out().packLong(-reqInfo.firstReqId), r -> null) |
There was a problem hiding this comment.
Protocol compat: what will happen with an older server that does not understand negative resource ids?
There was a problem hiding this comment.
Essentially, it will experience the old "blocking" behaviour.
The request will be sent to the server, the server will panic at:
https://github.com/apache/ignite-3/pull/7844/changes/BASE..f649f764f18e572f830208127faddbc566fc8a6f#diff-6e5ede08174517c940134dafdcb9270727ed3738dcb475582b48c017344896f2L59
The client will pick the error, and try to rollback using the old method once the tx information is known.
I assumed it was ok just to fix it for newer versions without breaking the previous ones.
PS: The error is something like, resource not found, nothing very relevant.
There was a problem hiding this comment.
I think we need a protocol feature flag for this. Avoid sending the request if the server does not understand it.
** Fixed type ** Updated javadoc comment on ClientInboundMessageHandler ** Added IGNITE-28405 ticket mention
…andling logic. (For the most part anyway) ** The code is still very hard, its just a little less duplicated.
# Conflicts: # modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ProtocolBitmaskFeature.java # modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java # modules/client/src/main/java/org/apache/ignite/internal/client/TcpClientChannel.java
| if (ch.protocolContext().isFeatureSupported(TX_ROLLBACK_USING_FIRST_REQUEST)) { | ||
| return ch.serviceAsync(ClientOp.TX_ROLLBACK, w -> w.out().packLong(-reqInfo.firstReqId), r -> null); | ||
| } else { | ||
| return NOT_SUPPORTED_FUTURE; |
There was a problem hiding this comment.
What does this mean for the user?
| } | ||
|
|
||
| /** | ||
| * If the current request is the first request of a direct translation, add a listener to the {@link PayloadWriter}. |
There was a problem hiding this comment.
| * If the current request is the first request of a direct translation, add a listener to the {@link PayloadWriter}. | |
| * If the current request is the first request of a direct tx, add a listener to the {@link PayloadWriter}. |
| kvView.put(tx1, key, val); | ||
| @ParameterizedTest | ||
| @MethodSource("org.apache.ignite.internal.client.ItThinClientTransactionsTest#killTestContextFactory") | ||
| public void testRollbackDoesNotBlock(KillTestContext ctx) throws InterruptedException { |
There was a problem hiding this comment.
All those new tests pass on the main branch, looks like we don't test the new functionality.
Opened a new commit following #7730 with a much simpler approach.
https://issues.apache.org/jira/browse/IGNITE-27947
What was done:
** TX_ROLLBACK accpets a request id of the first request of a direct mapped TX. Request Id is encoded in the negative range of resourceId.
** First Request Ids per TX are tracked using a mapping from reqId to resourceId. Mappings are created when the TX is created and removed on rollback and on response sent.
** Update all the operations. RO ops have the same parameters just for consistency.
** Allow multiple onSent callbacks on the payload output object.
** Added information to ClientLazyTransaction about the first request in the TX. Updated via PayloadOutputChannel on successful request.
** Implemented TX_ROLLBACK based on firstReqId