Table of Contents
ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - Before
and After
. It guarantees that every transaction committed Before
also will be committed Before
on every other node participated in the transaction. It means that an Ignite nodes can safely recover themself to the consistent Before
state without any coordination with each other.
...
ConsistentCutStartRecord
and AND weren't included into ConsistentCutFinishRecord#after().
ConsistentCutStartRecord
and ConsistentCutFinishRecord
and AND were included into ConsistentCutFinishRecord#before()
....
Code Block | ||||
---|---|---|---|---|
| ||||
/** */ public class ConsistentCutStartRecord extends WALRecord { /** Consistent Cut ID. */ private final UUID cutId; } /** */ public class ConsistentCutFinishRecord extends WALRecord { /** Consistent Cut ID. */ private final UUID cutId; /** * Collections of transactions committed BEFORE. */ private final Set<GridCacheVersion> before; /** * Collections of transactions committed AFTER. */ private final Set<GridCacheVersion> after; } |
...
Picture bellow illustrates steps of the algorithm on single node:
lastFinishedCutId
lastFinishedCutId
holds previous ConsistentCutId
, or null.DistributedProcess
with special message a DistributedProcess with discovery message SnapshotOperationRequest
that holds new ConsistentCutId
(goal is to notify every node in a cluster about running incremental snapshot). ConsistentCutId
by discoverythe SnapshotOperationRequest#ConsistentCutId
by DiscoverySPI (by the DistributedProcess).ConsistentCutId
by transaction message ( ConsistentCutAwareMessage#ConsistentCutId
by CommunicationSPI (by transaction messages - Prepare, Finish).ConsistentCutId
, every node: it starts local ConsistentCut: ConsistentCut
!= null) or finished (lastFinishedCutId
== id) for this ConsistentCut
future.committingTx
(goal is to track COMMITTING transactions, that aren't part of IgniteTxManager#activeTx
)ConsistentCutId
.IgniteTxManager#activeTx.
Set listeners on those tx#finishFuture.
ConsistentCutStartRecord
to WAL with the received ConsistentCutId
.committingTxs.
Set listeners on those tx#finishFuture.
committingTxs
to null.
DistributedProcess
is running every node is signing output transaction messages:ConsistentCutAwareMessage#topVer
with local node order:ConsistentCut
future. ConsistentCut != null
wraps outgoing messages to ConsistentCutAwareMessage
. It contains info:ConsistentCutId
(to start ConsistentCut ConsistentCutId
(to trigger ConsistentCut
txCutId
on the node that commits first (see below in Signing messages)txCutId
equals to null then transaction starts committing Before
Consistent Cut started, otherwise After
.ConsistentCutAwareMessage
that makes transaction committed (FinishRequest for 2PC, PrepareResponse for 1PC) sets tx#txCutId = message#txCutId
.ConsistentCut
future.removedActiveTxs
(This collection doesn't remove transactions unlike IgniteTxManager#activeTx
does).ConsistentCutStartRecord
to WAL with the received ConsistentCutId
.IgniteTxManager#activeTx
. Set listeners on those tx#finishFuture
.committingTxs
, and marks the transaction with extracted from the message tx#status == ACTIVE
. It's guaranteed that such transactions belongs After side.removedActiveTxs
(contains transactions that are might be cleaned from IgniteTxManager#activeTx
). Set listeners on those tx#finishFuture
.removedActiveTxs
to null. We don't care of txs concurrently added to removedActiveTxs
, they just don't land into "before" or "after" set and will be excluded from recovery.removedActiveTxs
if ConsistentCut != null
and removedActiveTxs != null
:removedActiveTxs
right before it is removed from IgniteTxManager#activeTx
txCutId
equals tx#txCutId
equals to local, then put transaction ConsistentCutFinishRecord
into WAL with ConsistentCut
future.ConsistentCut
future becomes future becomes null....
tx.finalizationStatus
== RECOVERY_FINISH).On the picture below on left side is a diagram of sending transaction messages. Before sending message it checks whether cut is running. If it is then wraps the message, otherwise send ordinary message (PrepareRequest
in example).
Ignite transaction protocol includes multiple messages. But only some of them affects meaningful (relating to the algorithm) that change state of transactions (PREPARED, COMMITTED):
...
Those messages are wrapped in ConsistentCutAwareMessage
that is prepared right before sending message on other node. They used the current ConsistentCutId
.
Code Block | ||||
---|---|---|---|---|
| ||||
class ConsistentCutAwareMessage {
Message msg;
UUID cutId;
} |
Also some messages require to be signed with combine with additional ConsistentCutId
to check it them on primary/backup node.:
GridNearTxFinishRequest / GridDhtTxFinishRequest
GridNearTxPrepareResponse / GridDhtTxPrepareResponse
(for 1PC algorithm).Those messages are wrapped in ConsistentCutAwareTxFinishMessage
filled with txCutId
that is prepared right before transaction starts committing on first committing node. They used the current ConsistentCutId
for this setting. txCutId
can be If current ConsistentCutId
is not null, if transaction starts committing before ConsistentCut starts.then transaction starts committing after ConsistentCut started and it means that this transaction belongs the After
side.
Code Block | ||||
---|---|---|---|---|
| ||||
class ConsistentCutAwareMessage {
/** Original transaction message. */
Message msg;
/** Consistent Cut ID. */
UUID cutId;
/** Consistent Cut ID after which transaction committed. */
@Nullable UUID txCutId;
/** Cluster topology version on which Consistent Cut started. */
long topVer;
} |
A new field added to IgniteInternalTx
Code Block | ||||
---|---|---|---|---|
| ||||
class IgniteInternalTx {
/**
* @param ID of {@link ConsistentCut} AFTER which this transaction was committed, {@code null} if transaction
* committed BEFORE.
*/
public void cutId(@Nullable UUID id);
} |
Code Block | |||||
---|---|---|---|---|---|
| |||||
// Class is responsible for managing all stuff related to Consistent Cut. It's an entrypoint for transaction threads to check running consistent cut.
class ConsistentCutManager extends GridCacheSharedManagerAdapter {
// Current Consistent Cut. All transactions threads wraps outgoing messages if this field is not null. */
volatile @Nullable ConsistentCut cut;
// Entrypoint for handling received new Consistent Cut ID.
void handleConsistentCutId(UUID id);
} |
Code Block | ||||
---|---|---|---|---|
| ||||
class ConsistentCut extends GridFutureAdapter<WALPointer> { Set<GridCacheVersion> beforeCut; Set<GridCacheVersion> afterCut; Set<IgniteInternalFuture<IgniteInternalTx>> removedActiveclass ConsistentCutAwareTransactionFinishMessage extends ConsistentCutAwareMessage { @Nullable UUID txCutId; } |