[ 
https://issues.apache.org/jira/browse/IGNITE-19824?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Alexander Lapin updated IGNITE-19824:
-------------------------------------
    Description: 
h3. Motivation

Currently, all implicit read operations start RW transactions, thus it's 
possible to catch "Failed to acquire a lock due to a conflict" exception. 
Generally speaking, given issue should be resolved by substituting RW with RO 
for all implicit read transactions, however such approach will decrease 
linearizability so it's required to verify it with product management. It's 
still however possible to have special case RO for implicit single-key get 
operation that will set readTimestamp on primary replica instead of transaction 
coordinator and thus provide cluster-wide linearizability even for RO 
transactions (only for single-key implicit get operations). Within this ticket, 
such special RO transactions should be introduced along with their usage switch 
for single-get implicit reads.
h3. Definition of Done
 * Implicit single-get operations use special RO transactions that provide 
cluster-wide linearizability and thus do not throw "Failed to acquire a lock 
due to a conflict" exception.
 * ItAbstractDataStreamerTest#testAutoFlushByTimer adjusted: catch block 
removed.

h3. Implementation Notes

1. Basically, what we need to do here is to start RO transaction instead of RW 
one in case of single-key implicit get, thus we should add
{code:java}
if (tx == null) {
    tx = txManager.begin(true);
}{code}
right in front of
{code:java}
return enlistInTx({code}
Please pay attention, that we wan't to start special case RO transaction that 
should go to primary and only primary, so it's not valit to put aforementioned 
tx = txManager.begin(true); at the very beginning of the method, because in 
that case balancer may return non-primary through 
evaluateReadOnlyRecipientNode. Corresponging comment should be added.

2. Such specifal case RO transcation doesn't require readTimestamp calcualtion 
on tx.start for the evaluation point of view, however it still required it for 
lowWatermark managerment:
{code:java}
readOnlyTxFutureById.compute(new TxIdAndTimestamp(readTimestamp, txId), 
(txIdAndTimestamp, readOnlyTxFuture) -> {
    assert readOnlyTxFuture == null : "previous transaction has not completed 
yet: " + txIdAndTimestamp;

    if (lowWatermark != null && readTimestamp.compareTo(lowWatermark) <= 0) {
        throw new IgniteInternalException(
                TX_READ_ONLY_TOO_OLD_ERR,
                "Timestamp read-only transaction must be greater than the low 
watermark: [txTimestamp={}, lowWatermark={}]",
                readTimestamp, lowWatermark
        );
    }

    return new CompletableFuture<>();
}); {code}
So, seems that it worth to leave readTimestamp generatoin at it's current place.

3. And again in order to have cluster-wide linearizability it's requried to use 
primaryReplica now as readTimestamp instead of the one proposed in 
readOnlyReplicaRequest. Basically that means substitution of
{code:java}
HybridTimestamp readTimestamp = request.readTimestamp(); {code}
with
{code:java}
HybridTimestamp readTimestamp;

if (request.requestType() == RequestType.RO_GET && request.implicit()) {
    readTimestamp = hybridClock.now();
} else {
    readTimestamp = request.readTimestamp();
} {code}
along with
{code:java}
CompletableFuture<Void> safeReadFuture = isPrimaryInTimestamp(isPrimary, 
readTimestamp) ? completedFuture(null)
        : safeTime.waitFor(readTimestamp); {code}
in PartitionReplicaListnener. That on its part required adding implicit() to 
ReadOnlySingleRowReplicaRequest that should be properly set on the client side.

  was:
h3. Motivation

Currently, all implicit read operations start RW transactions, thus it's 
possible to catch "Failed to acquire a lock due to a conflict" exception. 
Generally speaking, given issue should be resolved by substituting RW with RO 
for all implicit read transactions, however such approach will decrease 
linearizability so it's required to verify it with product management. It's 
still however possible to have special case RO for implicit single-key get 
operation that will set readTimestamp on primary replica instead of transaction 
coordinator and thus provide cluster-wide linearizability even for RO 
transactions (only for single-key implicit get operations). Within this ticket, 
such special RO transactions should be introduced along with their usage switch 
for single-get implicit reads.
h3. Definition of Done
 * Implicit single-get operations use special RO transactions that provide 
cluster-wide linearizability and thus do not throw "Failed to acquire a lock 
due to a conflict" exception.
 * ItAbstractDataStreamerTest#testAutoFlushByTimer adjusted: catch block 
removed.

h3. Implementation Notes

1. Basically, what we need to do here is to start RO transaction instead of RW 
one in case of single-key implicit get, thus we should add

 
{code:java}
if (tx == null) {
    tx = txManager.begin(true);
}{code}
right in front of

 
{code:java}
return enlistInTx({code}
Please pay attention, that we wan't to start special case RO transaction that 
should go to primary and only primary, so it's not valit to put aforementioned 
tx = txManager.begin(true); at the very beginning of the method, because in 
that case balancer may return non-primary through 
evaluateReadOnlyRecipientNode. Corresponging comment should be added.

2. Such specifal case RO transcation doesn't require readTimestamp calcualtion 
on tx.start for the evaluation point of view, however it still required it for 
lowWatermark managerment:

 
{code:java}
readOnlyTxFutureById.compute(new TxIdAndTimestamp(readTimestamp, txId), 
(txIdAndTimestamp, readOnlyTxFuture) -> {
    assert readOnlyTxFuture == null : "previous transaction has not completed 
yet: " + txIdAndTimestamp;

    if (lowWatermark != null && readTimestamp.compareTo(lowWatermark) <= 0) {
        throw new IgniteInternalException(
                TX_READ_ONLY_TOO_OLD_ERR,
                "Timestamp read-only transaction must be greater than the low 
watermark: [txTimestamp={}, lowWatermark={}]",
                readTimestamp, lowWatermark
        );
    }

    return new CompletableFuture<>();
}); {code}
So, seems that it worth to leave readTimestamp generatoin at it's current place.

 

3. And again in order to have cluster-wide linearizability it's requried to use 
primaryReplica now as readTimestamp instead of the one proposed in 
readOnlyReplicaRequest. Basically that means substitution of

 
{code:java}
HybridTimestamp readTimestamp = request.readTimestamp(); {code}
with

 

 
{code:java}
HybridTimestamp readTimestamp;

if (request.requestType() == RequestType.RO_GET && request.implicit()) {
    readTimestamp = hybridClock.now();
} else {
    readTimestamp = request.readTimestamp();
} {code}
along with

 

 
{code:java}
CompletableFuture<Void> safeReadFuture = isPrimaryInTimestamp(isPrimary, 
readTimestamp) ? completedFuture(null)
        : safeTime.waitFor(readTimestamp); {code}
in PartitionReplicaListnener. That on its part required adding implicit() to 
ReadOnlySingleRowReplicaRequest that should be properly set on the client side.

 

 

 

 

 


> Implicit RO should be used in implicit single gets
> --------------------------------------------------
>
>                 Key: IGNITE-19824
>                 URL: https://issues.apache.org/jira/browse/IGNITE-19824
>             Project: Ignite
>          Issue Type: Improvement
>            Reporter: Alexander Lapin
>            Priority: Major
>              Labels: ignite-3
>
> h3. Motivation
> Currently, all implicit read operations start RW transactions, thus it's 
> possible to catch "Failed to acquire a lock due to a conflict" exception. 
> Generally speaking, given issue should be resolved by substituting RW with RO 
> for all implicit read transactions, however such approach will decrease 
> linearizability so it's required to verify it with product management. It's 
> still however possible to have special case RO for implicit single-key get 
> operation that will set readTimestamp on primary replica instead of 
> transaction coordinator and thus provide cluster-wide linearizability even 
> for RO transactions (only for single-key implicit get operations). Within 
> this ticket, such special RO transactions should be introduced along with 
> their usage switch for single-get implicit reads.
> h3. Definition of Done
>  * Implicit single-get operations use special RO transactions that provide 
> cluster-wide linearizability and thus do not throw "Failed to acquire a lock 
> due to a conflict" exception.
>  * ItAbstractDataStreamerTest#testAutoFlushByTimer adjusted: catch block 
> removed.
> h3. Implementation Notes
> 1. Basically, what we need to do here is to start RO transaction instead of 
> RW one in case of single-key implicit get, thus we should add
> {code:java}
> if (tx == null) {
>     tx = txManager.begin(true);
> }{code}
> right in front of
> {code:java}
> return enlistInTx({code}
> Please pay attention, that we wan't to start special case RO transaction that 
> should go to primary and only primary, so it's not valit to put 
> aforementioned tx = txManager.begin(true); at the very beginning of the 
> method, because in that case balancer may return non-primary through 
> evaluateReadOnlyRecipientNode. Corresponging comment should be added.
> 2. Such specifal case RO transcation doesn't require readTimestamp 
> calcualtion on tx.start for the evaluation point of view, however it still 
> required it for lowWatermark managerment:
> {code:java}
> readOnlyTxFutureById.compute(new TxIdAndTimestamp(readTimestamp, txId), 
> (txIdAndTimestamp, readOnlyTxFuture) -> {
>     assert readOnlyTxFuture == null : "previous transaction has not completed 
> yet: " + txIdAndTimestamp;
>     if (lowWatermark != null && readTimestamp.compareTo(lowWatermark) <= 0) {
>         throw new IgniteInternalException(
>                 TX_READ_ONLY_TOO_OLD_ERR,
>                 "Timestamp read-only transaction must be greater than the low 
> watermark: [txTimestamp={}, lowWatermark={}]",
>                 readTimestamp, lowWatermark
>         );
>     }
>     return new CompletableFuture<>();
> }); {code}
> So, seems that it worth to leave readTimestamp generatoin at it's current 
> place.
> 3. And again in order to have cluster-wide linearizability it's requried to 
> use primaryReplica now as readTimestamp instead of the one proposed in 
> readOnlyReplicaRequest. Basically that means substitution of
> {code:java}
> HybridTimestamp readTimestamp = request.readTimestamp(); {code}
> with
> {code:java}
> HybridTimestamp readTimestamp;
> if (request.requestType() == RequestType.RO_GET && request.implicit()) {
>     readTimestamp = hybridClock.now();
> } else {
>     readTimestamp = request.readTimestamp();
> } {code}
> along with
> {code:java}
> CompletableFuture<Void> safeReadFuture = isPrimaryInTimestamp(isPrimary, 
> readTimestamp) ? completedFuture(null)
>         : safeTime.waitFor(readTimestamp); {code}
> in PartitionReplicaListnener. That on its part required adding implicit() to 
> ReadOnlySingleRowReplicaRequest that should be properly set on the client 
> side.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to