[
https://issues.apache.org/jira/browse/IGNITE-15085?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17432528#comment-17432528
]
Alexey Scherbakov commented on IGNITE-15085:
--------------------------------------------
This patch includes phase 1 of tx support (lock based concurrency control + 2PC
atomic commit + SERIALIZABLE isolation for table API).
Implementation details:
# Implemented the LockManager with deadlock prevention.
# Implemented basic TX coordination: each enlisted partition is mapped to
leaseholder(currenly raft group leader)
# Locking is tighly integrated with raft [1] - before a key is enlisted a lock
must be acquired.
# All raft commands now carry a transaction timestamp.
# Failover of any kind is not supported.
# Cursors are not transactional due to current raft based implementation.
# The current API using _*withTransactions*_ for table enlisting doesn't works
well then async chaning is used. Instead we should change table API to support
current transaction as an argument. I'm intended to do this in separate ticket.
# This patch includes not directly relevant change in RecordView.getAll
semantics. Now it returns a null if a key is not present - this allows O(1)
check is a key was found or not. For example: getAll(List.of(1, 2, 3)) can
return a List(v1, null, v3) if a key=2 is not present
# The patch contains a lot of TODOs without a ticket - I will create tickets
before merging.
[1] org.apache.ignite.raft.client.service.RaftGroupListener#onBeforeApply
> 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)