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

 Kirill Sizov updated IGNITE-20034:
-----------------------------------
    Description: 
h3. Motivation

In order to implement proper handling of a commit partition primary replica 
change it's required to:
 # Support local txnStateMap, in order to
 # Implement writeIntentResolution coordinator path, in order to
 # Calculate commitTimestamp on txn coordtinator istead of commit partition, 
where we do it now.

First item is covered in IGNITE-20033 Given ticket is about second item only. 
Third one will be covered in a separate ticket.
h3. Definition of Done

>From the "client" side, it's required to adjust current writeIntentResolution 
>logic by checking local txnStateLocal map.
 * If there's no corresponding transaction in given map - fallback to the 
commitPartition path.
 * If the state is COMMITED or ABORTED - use given value as 
writeInentResolution base. Actually, by doing this we will implelent 
writeIntentResolution local path.
 * If the state is FINISHING - wait for outcome and return response with the 
result: commit or abort. One of the most nontrivial steps, because of awaiting 
logic.
 * If the state is PENDING, retrieve txCoordAddr and ask tx coordinator.
 ** If tx coordinator is missing or returns null fallback to the commit 
partition path.

>From the "server" side
 * If the local txn is COMMITED or ABORTED, return the response with the 
outcome: commit or abort.
 * If the local txn is FINISHING ** (waiting for finish state replication), 
wait for outcome and return response with the result: commit or abort
 ** If the result is to commit, additional timestamp check is required: a 
commit timestamp must be <= readTs. If the condition is not held, the result is 
changed to abort.
 * If local txn is PENDING, adjust the txn coordinator node HLC according to 
readTs to make sure the txn commit timestamp is above the read timestamp. The 
read timestamp must be installed before txn is started to commit, so commit 
timestamp is assigned after the read timestamp. This must be achieved by some 
sort of concurrency control, preferably non-blocking. In this case we must 
ignore the write intent, so the outcome is to abort.
 * If txn state is not found in a local cache and txn is not active, return 
NULL.

h3. Implementation Notes

Open questions:
 * Which component is responsible for handling TxStateReq?
 * How to await FINISHING state termination?

  was:
h3. Motivation

In order to implement proper handling of a commit partition primary replica 
change it's required to:
 # Support local txnStateMap, in order to
 # Implement writeIntentResolution coordinator path, in order to
 # Calculate commitTimestamp on txn coordtinator istead of commit partition, 
where we do it now.

First item is covered in https://issues.apache.org/jira/browse/IGNITE-20033 
Given ticket is about second item only. Third one will be covered in a separate 
ticket.
h3. Definition of Done

>From the "client" side, it's required to adjust current writeIntentResolution 
>logic by checking local txnStateLocal map.
 * If there's no corresponding transaction in given map - fallback to the 
commitPartition path.
 * If the state is COMMITED or ABORTED - use given value as 
writeInentResolution base. Actually, by doing this we will implelent 
writeIntentResolution local path.
 * If the state is FINISHING - wait for outcome and return response with the 
result: commit or abort. One of the most nontrivial steps, because of awaiting 
logic.
 * If the state is PENDING, retrieve txCoordAddr and ask tx coordinator.
 ** If tx coordinator is missing or returns null fallback to the commit 
partition path.

>From the "server" side
 * If the local txn is COMMITED or ABORTED, return the response with the 
outcome: commit or abort.
 * If the local txn is FINISHING ** (waiting for finish state replication), 
wait for outcome and return response with the result: commit or abort
 ** If the result is to commit, additional timestamp check is required: a 
commit timestamp must be <= readTs. If the condition is not held, the result is 
changed to abort.
 * If local txn is PENDING, adjust the txn coordinator node HLC according to 
readTs to make sure the txn commit timestamp is above the read timestamp. The 
read timestamp must be installed before txn is started to commit, so commit 
timestamp is assigned after the read timestamp. This must be achieved by some 
sort of concurrency control, preferably non-blocking. In this case we must 
ignore the write intent, so the outcome is to abort.
 * If txn state is not found in a local cache and txn is not active, return 
NULL.

h3. Implementation Notes

Open questions:
 * Which component is responsible for handling TxStateReq?
 * How to await FINISHING state termination?


> Implement writeIntentResolution coordinator path
> ------------------------------------------------
>
>                 Key: IGNITE-20034
>                 URL: https://issues.apache.org/jira/browse/IGNITE-20034
>             Project: Ignite
>          Issue Type: Improvement
>            Reporter: Alexander Lapin
>            Assignee: Denis Chudov
>            Priority: Major
>              Labels: ignite-3, transaction, transaction3_recovery
>
> h3. Motivation
> In order to implement proper handling of a commit partition primary replica 
> change it's required to:
>  # Support local txnStateMap, in order to
>  # Implement writeIntentResolution coordinator path, in order to
>  # Calculate commitTimestamp on txn coordtinator istead of commit partition, 
> where we do it now.
> First item is covered in IGNITE-20033 Given ticket is about second item only. 
> Third one will be covered in a separate ticket.
> h3. Definition of Done
> From the "client" side, it's required to adjust current writeIntentResolution 
> logic by checking local txnStateLocal map.
>  * If there's no corresponding transaction in given map - fallback to the 
> commitPartition path.
>  * If the state is COMMITED or ABORTED - use given value as 
> writeInentResolution base. Actually, by doing this we will implelent 
> writeIntentResolution local path.
>  * If the state is FINISHING - wait for outcome and return response with the 
> result: commit or abort. One of the most nontrivial steps, because of 
> awaiting logic.
>  * If the state is PENDING, retrieve txCoordAddr and ask tx coordinator.
>  ** If tx coordinator is missing or returns null fallback to the commit 
> partition path.
> From the "server" side
>  * If the local txn is COMMITED or ABORTED, return the response with the 
> outcome: commit or abort.
>  * If the local txn is FINISHING ** (waiting for finish state replication), 
> wait for outcome and return response with the result: commit or abort
>  ** If the result is to commit, additional timestamp check is required: a 
> commit timestamp must be <= readTs. If the condition is not held, the result 
> is changed to abort.
>  * If local txn is PENDING, adjust the txn coordinator node HLC according to 
> readTs to make sure the txn commit timestamp is above the read timestamp. The 
> read timestamp must be installed before txn is started to commit, so commit 
> timestamp is assigned after the read timestamp. This must be achieved by some 
> sort of concurrency control, preferably non-blocking. In this case we must 
> ignore the write intent, so the outcome is to abort.
>  * If txn state is not found in a local cache and txn is not active, return 
> NULL.
> h3. Implementation Notes
> Open questions:
>  * Which component is responsible for handling TxStateReq?
>  * How to await FINISHING state termination?



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

Reply via email to