-
Notifications
You must be signed in to change notification settings - Fork 2.3k
/
RaftState.java
153 lines (136 loc) · 4.38 KB
/
RaftState.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
/*
* Copyright (c) 2002-2016 "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.state;
import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
import org.neo4j.coreedge.raft.log.RaftLog;
import org.neo4j.coreedge.raft.log.RaftStorageException;
import org.neo4j.coreedge.raft.log.ReadableRaftLog;
import org.neo4j.coreedge.raft.membership.RaftMembership;
import org.neo4j.coreedge.raft.outcome.LogCommand;
import org.neo4j.coreedge.raft.outcome.Outcome;
import org.neo4j.coreedge.raft.state.follower.FollowerStates;
import org.neo4j.coreedge.raft.state.term.TermState;
import org.neo4j.coreedge.raft.state.vote.VoteState;
public class RaftState<MEMBER> implements ReadableRaftState<MEMBER>
{
private final MEMBER myself;
private final StateStorage<TermState> termStorage;
private final StateStorage<VoteState<MEMBER>> voteStorage;
private final RaftMembership<MEMBER> membership;
private final TermState termState;
private MEMBER leader;
private long leaderCommit = -1;
private final VoteState<MEMBER> voteState;
private Set<MEMBER> votesForMe = new HashSet<>();
private long lastLogIndexBeforeWeBecameLeader = -1;
private FollowerStates<MEMBER> followerStates = new FollowerStates<>();
private final RaftLog entryLog;
public RaftState( MEMBER myself, StateStorage<TermState> termStorage, RaftMembership<MEMBER> membership,
RaftLog entryLog, StateStorage<VoteState<MEMBER>> voteStorage )
{
this.myself = myself;
this.termStorage = termStorage;
this.voteStorage = voteStorage;
this.termState = termStorage.getInitialState();
this.voteState = voteStorage.getInitialState();
this.membership = membership;
this.entryLog = entryLog;
}
@Override
public MEMBER myself()
{
return myself;
}
@Override
public Set<MEMBER> votingMembers()
{
return membership.votingMembers();
}
@Override
public Set<MEMBER> replicationMembers()
{
return membership.replicationMembers();
}
@Override
public long term()
{
return termState.currentTerm();
}
@Override
public MEMBER leader()
{
return leader;
}
@Override
public long leaderCommit()
{
return leaderCommit;
}
@Override
public MEMBER votedFor()
{
return voteState.votedFor();
}
@Override
public Set<MEMBER> votesForMe()
{
return votesForMe;
}
@Override
public long lastLogIndexBeforeWeBecameLeader()
{
return lastLogIndexBeforeWeBecameLeader;
}
@Override
public FollowerStates<MEMBER> followerStates()
{
return followerStates;
}
@Override
public ReadableRaftLog entryLog()
{
return entryLog;
}
public void update( Outcome<MEMBER> outcome ) throws RaftStorageException
{
termState.update( outcome.getTerm() );
voteState.votedFor( outcome.getVotedFor(), outcome.getTerm() );
try
{
termStorage.persistStoreData( termState );
voteStorage.persistStoreData( voteState );
}
catch ( IOException e )
{
throw new RaftStorageException( e );
}
leader = outcome.getLeader();
leaderCommit = outcome.getLeaderCommit();
votesForMe = outcome.getVotesForMe();
lastLogIndexBeforeWeBecameLeader = outcome.getLastLogIndexBeforeWeBecameLeader();
followerStates = outcome.getFollowerStates();
for ( LogCommand logCommand : outcome.getLogCommands() )
{
logCommand.applyTo( entryLog );
}
}
}