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

Alexey Scherbakov updated IGNITE-15085:
---------------------------------------
    Description: 
This ticket implies the implementation of tx coordination, as described in [1].
 This includes:
 * TxManager - top-level manager for tx state and coordination
 * pre-writes - each tx write is pre-written to partition store in special 
format
 * replicated tx state - tx state is stored in the partition's raft group
 * integration with lock manager (see *precaution* chapter in [1])

The example of single key tx:
{noformat}
Tx client               TxCoordinator                                           
    Partition leaseholder.    
tx.start
            --------->  
                        assign timestamp (id)
                        txstate = PENDING
            <---------                                 
table.put(k,v)   
            --------->   
                        enlist(partition(k));
                        lh = getLeaseholder(partition(k))
                        send UpsertCommand(k) to lh
                                                                      
------------>
                                                                                
     replicate txstate = PENDING
                                                                                
     lockManager.tryAcquire(k,timestamp);
                                                                                
     wait for completion async
                                                                                
     prewrite(k, v) -- replicate to all replicas
                        repeat for each enlisted partition...
            <---------
tx.finish - commit or rollback
            --------->  
                        send finish request to all remote enlisted nodes
                                                                      
------------>
                                                                                
     replicate txstate = COMMITTED/ABORTED
                        txState = COMMITTED/ABORTED                             
     lockManager.tryRelease(k, timestamp)
                                                                      
<------------ 
                                        
                        when all leasholders are replied,
                        reply to initiator
            <--------   
{noformat}
[1] [https://github.com/apache/ignite-3/tree/main/modules/transactions]

  was:
This ticket implies the implementation of tx coordination, as described in [1].
 This includes:
 * TxManager - top-level manager for tx state and coordination
 * pre-writes - each tx write is pre-written to partition store in special 
format
 * replicated tx state - tx state is stored in the partition's raft group
 * integration with lock manager (see *precaution* chapter in [1])

The example of single key tx:
{noformat}
Tx client               TxCoordinator                                           
    Partition leaseholder.    
tx.start
            --------->  
                        assign timestamp (id)
                        txstate = PENDING
            <---------                                 
table.put(k,v)   
            --------->   
                        enlist(partition(k));
                        lh = getLeaseholder(partition(k))
                                                                      
------------>
                                                                                
     write txstate = PENDING
                                                                                
     fut=lockManager.tryAcquire(k,timestamp);
                                                                                
     wait for completion async
                                                                                
     prewrite(k, v)
            <---------
tx.commit
            --------->
                                                                    
------------>
                                                                                
     write txstate = COMMITTED
                                                                                
     lockManager.tryRelease(k, timestamp)
                                                                    
<----------- 
                        lh.txmap.state = COMMITTED              
                        when all leasholders are replied,
                        reply to initiator
            <--------   
{noformat}
[1] [https://github.com/apache/ignite-3/tree/main/modules/transactions]


> Implement tx coordination
> -------------------------
>
>                 Key: IGNITE-15085
>                 URL: https://issues.apache.org/jira/browse/IGNITE-15085
>             Project: Ignite
>          Issue Type: Task
>            Reporter: Alexey Scherbakov
>            Priority: Major
>              Labels: iep-61, ignite-3
>
> This ticket implies the implementation of tx coordination, as described in 
> [1].
>  This includes:
>  * TxManager - top-level manager for tx state and coordination
>  * pre-writes - each tx write is pre-written to partition store in special 
> format
>  * replicated tx state - tx state is stored in the partition's raft group
>  * integration with lock manager (see *precaution* chapter in [1])
> The example of single key tx:
> {noformat}
> Tx client               TxCoordinator                                         
>       Partition leaseholder.    
> tx.start
>             --------->  
>                       assign timestamp (id)
>                       txstate = PENDING
>             <---------                                       
> table.put(k,v)   
>             --------->   
>                         enlist(partition(k));
>                       lh = getLeaseholder(partition(k))
>                         send UpsertCommand(k) to lh
>                                                                     
> ------------>
>                                                                               
>        replicate txstate = PENDING
>                                                                               
>        lockManager.tryAcquire(k,timestamp);
>                                                                               
>        wait for completion async
>                                                                               
>        prewrite(k, v) -- replicate to all replicas
>                         repeat for each enlisted partition...
>             <---------
> tx.finish - commit or rollback
>             --------->  
>                         send finish request to all remote enlisted nodes
>                                                                       
> ------------>
>                                                                               
>        replicate txstate = COMMITTED/ABORTED
>                         txState = COMMITTED/ABORTED                           
>        lockManager.tryRelease(k, timestamp)
>                                                                     
> <------------ 
>                                       
>                       when all leasholders are replied,
>                       reply to initiator
>           <--------   
> {noformat}
> [1] [https://github.com/apache/ignite-3/tree/main/modules/transactions]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to