-
Notifications
You must be signed in to change notification settings - Fork 2.3k
/
ReplicatedTransactionStateMachine.java
177 lines (158 loc) · 7.28 KB
/
ReplicatedTransactionStateMachine.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
/*
* Copyright (c) 2002-2015 "Neo Technology,"
* Network Engine for Objects in Lund AB [http://neotechnology.com]
*
* This file is part of Neo4j.
*
* Neo4j is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License as
* published by the Free Software Foundation, either version 3 of the
* License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU Affero General Public License for more details.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
package org.neo4j.coreedge.raft.replication.tx;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Future;
import org.neo4j.concurrent.CompletableFuture;
import org.neo4j.coreedge.raft.NewLeaderBarrier;
import org.neo4j.coreedge.raft.replication.ReplicatedContent;
import org.neo4j.coreedge.raft.replication.Replicator;
import org.neo4j.coreedge.raft.replication.session.GlobalSession;
import org.neo4j.coreedge.raft.replication.session.GlobalSessionTracker;
import org.neo4j.coreedge.raft.replication.session.LocalOperationId;
import org.neo4j.graphdb.TransientFailureException;
import org.neo4j.graphdb.TransientTransactionFailureException;
import org.neo4j.kernel.api.exceptions.TransactionFailureException;
import org.neo4j.kernel.impl.api.TransactionApplicationMode;
import org.neo4j.kernel.impl.api.TransactionCommitProcess;
import org.neo4j.kernel.impl.api.TransactionToApply;
import org.neo4j.kernel.impl.transaction.TransactionRepresentation;
import org.neo4j.kernel.impl.transaction.log.PhysicalTransactionRepresentation;
import org.neo4j.kernel.impl.transaction.tracing.CommitEvent;
import org.neo4j.kernel.impl.util.Dependencies;
import static org.neo4j.coreedge.raft.replication.tx.LogIndexTxHeaderEncoding.encodeLogIndexAsTxHeader;
public class ReplicatedTransactionStateMachine implements Replicator.ReplicatedContentListener
{
private final GlobalSessionTracker sessionTracker;
private final GlobalSession myGlobalSession;
private final Dependencies dependencies;
private final TransactionCommitProcess commitProcess;
private final Map<LocalOperationId, FutureTxId> outstanding = new ConcurrentHashMap<>();
private long lastCommittedIndex = -1;
private long lastCommittedTxId; // Maintains the last committed tx id, used to set the next field
private long reignStartTxId; // Maintains the last txid committed under the previous service assignment
public ReplicatedTransactionStateMachine( TransactionCommitProcess commitProcess,
GlobalSession myGlobalSession, Dependencies dependencies )
{
this.commitProcess = commitProcess;
this.myGlobalSession = myGlobalSession;
this.dependencies = dependencies;
this.sessionTracker = new GlobalSessionTracker();
}
public Future<Long> getFutureTxId( LocalOperationId localOperationId )
{
final FutureTxId future = new FutureTxId( localOperationId );
outstanding.put( localOperationId, future );
return future;
}
@Override
public synchronized void onReplicated( ReplicatedContent content, long logIndex )
{
if ( content instanceof ReplicatedTransaction )
{
handleTransaction( (ReplicatedTransaction) content, logIndex );
}
if ( content instanceof NewLeaderBarrier )
{
try
{
reignStartTxId = appendBarrierTx( logIndex );
}
catch ( TransactionFailureException e )
{
throw new RuntimeException( e );
}
}
}
private long appendBarrierTx( long logIndex ) throws TransactionFailureException
{
PhysicalTransactionRepresentation dummyTx = new PhysicalTransactionRepresentation( Collections.emptyList() );
// TODO we need to set the "-1"'s below to useful values
dummyTx.setHeader( encodeLogIndexAsTxHeader( logIndex ), -1, -1, -1, -1, -1, -1 );
return commitProcess.commit( new TransactionToApply( dummyTx ), CommitEvent.NULL, TransactionApplicationMode.EXTERNAL );
}
private void handleTransaction( ReplicatedTransaction replicatedTx, long logIndex )
{
if ( !sessionTracker.validateAndTrackOperation( replicatedTx.globalSession(), replicatedTx.localOperationId() )
|| logIndex <= lastCommittedIndex )
{
return;
}
try
{
byte[] extraHeader = encodeLogIndexAsTxHeader( logIndex );
TransactionRepresentation tx = ReplicatedTransactionFactory.extractTransactionRepresentation(
replicatedTx, extraHeader );
// A missing future means the transaction does not belong to this instance
Optional<CompletableFuture<Long>> future = replicatedTx.globalSession().equals( myGlobalSession ) ?
Optional.ofNullable( outstanding.remove( replicatedTx.localOperationId() ) ) :
Optional.<CompletableFuture<Long>>empty();
if ( tx.getLatestCommittedTxWhenStarted() < reignStartTxId )
{
future.ifPresent( txFuture -> txFuture.completeExceptionally( new TransientTransactionFailureException(
"Attempt to commit transaction that was started on a different leader term. " +
"Please retry the transaction." ) ) );
return;
}
try
{
lastCommittedTxId = commitProcess.commit( new TransactionToApply( tx ), CommitEvent.NULL,
TransactionApplicationMode.EXTERNAL );
future.ifPresent( txFuture -> txFuture.complete( lastCommittedTxId ) );
}
catch ( TransientFailureException e )
{
future.ifPresent( txFuture -> txFuture.completeExceptionally( e ) );
}
}
catch ( TransactionFailureException | IOException e )
{
throw new IllegalStateException( "Failed to locally commit a transaction that has already been " +
"committed to the RAFT log. This server cannot process later transactions and needs to be " +
"restarted once the underlying cause has been addressed.", e );
}
}
public void setLastCommittedIndex( long lastCommittedIndex )
{
this.lastCommittedIndex = lastCommittedIndex;
}
private class FutureTxId extends CompletableFuture<Long>
{
private final LocalOperationId localOperationId;
FutureTxId( LocalOperationId localOperationId )
{
this.localOperationId = localOperationId;
}
@Override
public boolean cancel( boolean mayInterruptIfRunning )
{
if ( !super.cancel( mayInterruptIfRunning ) )
{
return false;
}
outstanding.remove( localOperationId );
return true;
}
}
}