[ https://issues.apache.org/jira/browse/IGNITE-20148?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Alexander Lapin updated IGNITE-20148: ------------------------------------- Description: h3. Motivation Currently, locks are released on primary when cleanup is replicated over majority {code:java} private CompletableFuture<Void> processTxCleanupAction(TxCleanupReplicaRequest request) { ... return raftClient .run(txCleanupCmd) .thenCompose(ignored -> allOffFuturesExceptionIgnored(txReadFutures, request) .thenRun(() -> releaseTxLocks(request.txId()))); ... } {code} That is actually incorrect, because it's possible that primary won't be a part of majority, meaning that we will release lock still having writeIntent locally. Generally speaking that should be resolved by implementing writeIntent resolution for RW transactions However given ticket is not yet implemented. Anyway, it is worth to clean up writeIntents on primary replica explicitly for a sense of performance in order to eliminate excessive writeIntent resolutions. h3. Definition of Done * Explicit writeIntent cleanup on primary replica prior to locks release is implemented. was: h3. Motivation Currently, locks are released on primary when cleanup is replicated over majority {code:java} private CompletableFuture<Void> processTxCleanupAction(TxCleanupReplicaRequest request) { ... return raftClient .run(txCleanupCmd) .thenCompose(ignored -> allOffFuturesExceptionIgnored(txReadFutures, request) .thenRun(() -> releaseTxLocks(request.txId()))); ... } {code} That is actually incorrect, because it's possible that primary won't be a part of majority, meaning that we will release lock still having writeIntent locally. Generally speaking that should be resolved by implementing writeIntent resolution for RW transactions However given ticket is not yet implemented. Anyway, it is worth to clean up writeIntents on primary replica explicitly for a sense of performance in order to eliminate excessive writeIntent resolutions. h3. Definition of Done > Explicit writeIntent cleanup on primary replica > ----------------------------------------------- > > Key: IGNITE-20148 > URL: https://issues.apache.org/jira/browse/IGNITE-20148 > Project: Ignite > Issue Type: Bug > Reporter: Alexander Lapin > Priority: Major > Time Spent: 10m > Remaining Estimate: 0h > > h3. Motivation > Currently, locks are released on primary when cleanup is replicated over > majority > {code:java} > private CompletableFuture<Void> > processTxCleanupAction(TxCleanupReplicaRequest request) { > ... > return raftClient > .run(txCleanupCmd) > .thenCompose(ignored -> > allOffFuturesExceptionIgnored(txReadFutures, request) > .thenRun(() -> releaseTxLocks(request.txId()))); > ... > } {code} > That is actually incorrect, because it's possible that primary won't be a > part of majority, meaning that we will release lock still having writeIntent > locally. Generally speaking that should be resolved by implementing > writeIntent resolution for RW transactions However given ticket is not yet > implemented. Anyway, it is worth to clean up writeIntents on primary replica > explicitly for a sense of performance in order to eliminate excessive > writeIntent resolutions. > h3. Definition of Done > * Explicit writeIntent cleanup on primary replica prior to locks release is > implemented. > -- This message was sent by Atlassian Jira (v8.20.10#820010)