-
Notifications
You must be signed in to change notification settings - Fork 2.3k
/
InMemoryRaftLog.java
209 lines (187 loc) · 5.72 KB
/
InMemoryRaftLog.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
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
/*
* 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.log;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArraySet;
import org.neo4j.coreedge.raft.replication.ReplicatedContent;
public class InMemoryRaftLog implements RaftLog
{
private final Set<Listener> listeners = new CopyOnWriteArraySet<>();
private final Map<Long, RaftLogEntry> raftLog = new HashMap<>();
private long appendIndex = -1;
private long commitIndex = -1;
private long term = -1;
@Override
public void replay() throws Throwable
{
int index = 0;
for (; index <= commitIndex; index++ )
{
ReplicatedContent content = readEntryContent( index );
for ( Listener listener : listeners )
{
listener.onAppended( content, index );
listener.onCommitted( content, index );
}
}
for (; index <= appendIndex; index++ )
{
ReplicatedContent content = readEntryContent( index );
for ( Listener listener : listeners )
{
listener.onAppended( content, index );
}
}
}
@Override
public void registerListener( Listener listener )
{
listeners.add( listener );
}
@Override
public long append( RaftLogEntry logEntry ) throws RaftStorageException
{
Objects.requireNonNull( logEntry );
if ( logEntry.term() >= term )
{
term = logEntry.term();
}
else
{
throw new RaftStorageException( String.format( "Non-monotonic term %d for in entry %s in term %d",
logEntry.term(), logEntry.toString(), term ) );
}
appendIndex++;
for ( Listener listener : listeners )
{
listener.onAppended( logEntry.content(), appendIndex );
}
raftLog.put( appendIndex, logEntry );
return appendIndex;
}
@Override
public void commit( long commitIndex )
{
if ( commitIndex > appendIndex )
{
commitIndex = appendIndex;
}
while ( this.commitIndex < commitIndex )
{
long nextCommitIndex = this.commitIndex + 1;
RaftLogEntry logEntry = raftLog.get( nextCommitIndex );
for ( Listener listener : listeners )
{
listener.onCommitted( logEntry.content(), nextCommitIndex );
}
this.commitIndex = nextCommitIndex;
}
}
@Override
public long appendIndex()
{
return appendIndex;
}
@Override
public long commitIndex()
{
return commitIndex;
}
@Override
public RaftLogEntry readLogEntry( long logIndex )
{
if ( logIndex < 0 )
{
throw new IllegalArgumentException( "logIndex must not be negative" );
}
if ( logIndex > appendIndex )
{
throw new IllegalArgumentException(
String.format( "cannot read past last appended index (lastAppended=%d, readIndex=%d)",
appendIndex, logIndex ) );
}
return raftLog.get( logIndex );
}
@Override
public ReplicatedContent readEntryContent( long logIndex )
{
return readLogEntry( logIndex ).content();
}
@Override
public long readEntryTerm( long logIndex )
{
if ( logIndex < 0 || logIndex > appendIndex )
{
return -1;
}
return readLogEntry( logIndex ).term();
}
@Override
public synchronized void truncate( long fromIndex )
{
if ( fromIndex <= commitIndex )
{
throw new IllegalArgumentException( "cannot truncate before the commit index" );
}
for ( long i = appendIndex; i >= fromIndex; --i )
{
raftLog.remove( i );
}
if ( appendIndex >= fromIndex )
{
appendIndex = fromIndex - 1;
for ( Listener listener : listeners )
{
listener.onTruncated( fromIndex );
}
}
term = readEntryTerm( appendIndex );
}
@Override
public boolean entryExists( long logIndex )
{
return raftLog.containsKey( logIndex );
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
InMemoryRaftLog that = (InMemoryRaftLog) o;
return Objects.equals( appendIndex, that.appendIndex ) &&
Objects.equals( commitIndex, that.commitIndex ) &&
Objects.equals( term, that.term ) &&
Objects.equals( raftLog, that.raftLog );
}
@Override
public int hashCode()
{
return Objects.hash( raftLog, appendIndex, commitIndex, term );
}
}