-
Notifications
You must be signed in to change notification settings - Fork 2.3k
/
RaftMessages.java
682 lines (584 loc) · 18.7 KB
/
RaftMessages.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
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
/*
* 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;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import org.neo4j.coreedge.network.Message;
import org.neo4j.coreedge.raft.log.RaftLogEntry;
import org.neo4j.coreedge.raft.replication.ReplicatedContent;
import static java.lang.String.format;
public interface RaftMessages
{
enum Type
{
VOTE_REQUEST,
VOTE_RESPONSE,
APPEND_ENTRIES_REQUEST,
APPEND_ENTRIES_RESPONSE,
HEARTBEAT,
LOG_COMPACTION_INFO,
// Timeouts
ELECTION_TIMEOUT,
HEARTBEAT_TIMEOUT,
// TODO: Refactor, these are client-facing messages / api. Perhaps not public and instantiated through an api
// TODO: method instead?
NEW_ENTRY_REQUEST,
NEW_BATCH_REQUEST,
NEW_MEMBERSHIP_TARGET,
}
interface RaftMessage<MEMBER> extends Message
{
MEMBER from();
Type type();
}
class Directed<MEMBER>
{
MEMBER to;
RaftMessage<MEMBER> message;
public Directed( MEMBER to, RaftMessage<MEMBER> message )
{
this.to = to;
this.message = message;
}
public MEMBER to()
{
return to;
}
public RaftMessage<MEMBER> message()
{
return message;
}
@Override
public String toString()
{
return format( "Directed{to=%s, message=%s}", to, message );
}
}
interface Vote
{
class Request<MEMBER> extends BaseMessage<MEMBER>
{
private long term;
private MEMBER candidate;
private long lastLogIndex;
private long lastLogTerm;
public Request( MEMBER from, long term, MEMBER candidate, long lastLogIndex, long lastLogTerm )
{
super( from, Type.VOTE_REQUEST );
this.term = term;
this.candidate = candidate;
this.lastLogIndex = lastLogIndex;
this.lastLogTerm = lastLogTerm;
}
public long term()
{
return term;
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
Request request = (Request) o;
return lastLogIndex == request.lastLogIndex &&
lastLogTerm == request.lastLogTerm &&
term == request.term &&
candidate.equals( request.candidate );
}
@Override
public int hashCode()
{
int result = (int) term;
result = 31 * result + candidate.hashCode();
result = 31 * result + (int) (lastLogIndex ^ (lastLogIndex >>> 32));
result = 31 * result + (int) (lastLogTerm ^ (lastLogTerm >>> 32));
return result;
}
@Override
public String toString()
{
return format( "Vote.Request from %s {term=%d, candidate=%s, lastAppended=%d, lastLogTerm=%d}",
from, term, candidate, lastLogIndex, lastLogTerm );
}
public long lastLogTerm()
{
return lastLogTerm;
}
public long lastLogIndex()
{
return lastLogIndex;
}
public MEMBER candidate()
{
return candidate;
}
}
class Response<MEMBER> extends BaseMessage<MEMBER>
{
private long term;
private boolean voteGranted;
public Response( MEMBER from, long term, boolean voteGranted )
{
super( from, Type.VOTE_RESPONSE );
this.term = term;
this.voteGranted = voteGranted;
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
Response response = (Response) o;
return term == response.term && voteGranted == response.voteGranted;
}
@Override
public int hashCode()
{
int result = (int) term;
result = 31 * result + (voteGranted ? 1 : 0);
return result;
}
@Override
public String toString()
{
return format( "Vote.Response from %s {term=%d, voteGranted=%s}", from, term, voteGranted );
}
public long term()
{
return term;
}
public boolean voteGranted()
{
return voteGranted;
}
}
}
interface AppendEntries
{
class Request<MEMBER> extends BaseMessage<MEMBER>
{
private long leaderTerm;
private long prevLogIndex;
private long prevLogTerm;
private RaftLogEntry[] entries;
private long leaderCommit;
public Request( MEMBER from, long leaderTerm, long prevLogIndex, long prevLogTerm,
RaftLogEntry[] entries, long leaderCommit )
{
super( from, Type.APPEND_ENTRIES_REQUEST );
Objects.requireNonNull( entries );
assert !((prevLogIndex == -1 && prevLogTerm != -1) || (prevLogTerm == -1 && prevLogIndex != -1)) :
format( "prevLogIndex was %d and prevLogTerm was %d", prevLogIndex, prevLogTerm );
this.entries = entries;
this.leaderTerm = leaderTerm;
this.prevLogIndex = prevLogIndex;
this.prevLogTerm = prevLogTerm;
this.leaderCommit = leaderCommit;
}
public long leaderTerm()
{
return leaderTerm;
}
public long prevLogIndex()
{
return prevLogIndex;
}
public long prevLogTerm()
{
return prevLogTerm;
}
public RaftLogEntry[] entries()
{
return entries;
}
public long leaderCommit()
{
return leaderCommit;
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
Request<?> request = (Request<?>) o;
return Objects.equals( leaderTerm, request.leaderTerm ) &&
Objects.equals( prevLogIndex, request.prevLogIndex ) &&
Objects.equals( prevLogTerm, request.prevLogTerm ) &&
Objects.equals( leaderCommit, request.leaderCommit ) &&
Arrays.equals( entries, request.entries );
}
@Override
public int hashCode()
{
return Objects.hash( leaderTerm, prevLogIndex, prevLogTerm, entries, leaderCommit );
}
@Override
public String toString()
{
return format( "AppendEntries.Request from %s {leaderTerm=%d, prevLogIndex=%d, " +
"prevLogTerm=%d, entry=%s, leaderCommit=%d}",
from, leaderTerm, prevLogIndex, prevLogTerm, Arrays.toString( entries ), leaderCommit );
}
}
class Response<MEMBER> extends BaseMessage<MEMBER>
{
private long term;
private boolean success;
private long matchIndex;
private long appendIndex;
public Response( MEMBER from, long term, boolean success, long matchIndex, long appendIndex )
{
super( from, Type.APPEND_ENTRIES_RESPONSE );
this.term = term;
this.success = success;
this.matchIndex = matchIndex;
this.appendIndex = appendIndex;
}
public long term()
{
return term;
}
public boolean success()
{
return success;
}
public long matchIndex()
{
return matchIndex;
}
public long appendIndex()
{
return appendIndex;
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
if ( !super.equals( o ) )
{
return false;
}
Response<?> response = (Response<?>) o;
return term == response.term &&
success == response.success &&
matchIndex == response.matchIndex &&
appendIndex == response.appendIndex;
}
@Override
public int hashCode()
{
return Objects.hash( super.hashCode(), term, success, matchIndex, appendIndex );
}
@Override
public String toString()
{
return format( "AppendEntries.Response from %s {term=%d, success=%s, matchIndex=%d, appendIndex=%d}",
from, term, success, matchIndex, appendIndex );
}
}
}
class Heartbeat<MEMBER> extends BaseMessage<MEMBER>
{
private long leaderTerm;
private long commitIndex;
private long commitIndexTerm;
public Heartbeat( MEMBER from, long leaderTerm, long commitIndex, long commitIndexTerm )
{
super( from, Type.HEARTBEAT );
this.leaderTerm = leaderTerm;
this.commitIndex = commitIndex;
this.commitIndexTerm = commitIndexTerm;
}
public long leaderTerm()
{
return leaderTerm;
}
public long commitIndex()
{
return commitIndex;
}
public long commitIndexTerm()
{
return commitIndexTerm;
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
if ( !super.equals( o ) )
{
return false;
}
Heartbeat<?> heartbeat = (Heartbeat<?>) o;
return leaderTerm == heartbeat.leaderTerm &&
commitIndex == heartbeat.commitIndex &&
commitIndexTerm == heartbeat.commitIndexTerm;
}
@Override
public int hashCode()
{
int result = super.hashCode();
result = 31 * result + (int) (leaderTerm ^ (leaderTerm >>> 32));
result = 31 * result + (int) (commitIndex ^ (commitIndex >>> 32));
result = 31 * result + (int) (commitIndexTerm ^ (commitIndexTerm >>> 32));
return result;
}
@Override
public String toString()
{
return format( "Heartbeat from %s {leaderTerm=%d, commitIndex=%d, commitIndexTerm=%d}", from, leaderTerm,
commitIndex, commitIndexTerm );
}
}
class LogCompactionInfo<MEMBER> extends BaseMessage<MEMBER>
{
private long leaderTerm;
private long prevIndex;
public LogCompactionInfo( MEMBER from, long leaderTerm, long prevIndex )
{
super( from, Type.LOG_COMPACTION_INFO );
this.leaderTerm = leaderTerm;
this.prevIndex = prevIndex;
}
public long leaderTerm()
{
return leaderTerm;
}
public long prevIndex()
{
return prevIndex;
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
if ( !super.equals( o ) )
{
return false;
}
LogCompactionInfo<?> other = (LogCompactionInfo<?>) o;
return leaderTerm == other.leaderTerm &&
prevIndex == other.prevIndex;
}
@Override
public int hashCode()
{
int result = super.hashCode();
result = 31 * result + (int) (leaderTerm ^ (leaderTerm >>> 32));
result = 31 * result + (int) (prevIndex ^ (prevIndex >>> 32));
return result;
}
@Override
public String toString()
{
return format( "Log compaction from %s {leaderTerm=%d, prevIndex=%d}", from, leaderTerm, prevIndex );
}
}
interface Timeout
{
class Election<MEMBER> extends BaseMessage<MEMBER>
{
public Election( MEMBER from )
{
super( from, Type.ELECTION_TIMEOUT );
}
@Override
public String toString()
{
return "Timeout.Election{}";
}
}
class Heartbeat<MEMBER> extends BaseMessage<MEMBER>
{
public Heartbeat( MEMBER from )
{
super( from, Type.HEARTBEAT_TIMEOUT );
}
@Override
public String toString()
{
return "Timeout.Heartbeat{}";
}
}
}
interface NewEntry
{
class Request<MEMBER> extends BaseMessage<MEMBER>
{
private ReplicatedContent content;
public Request( MEMBER from, ReplicatedContent content )
{
super( from, Type.NEW_ENTRY_REQUEST );
this.content = content;
}
@Override
public String toString()
{
return format( "NewEntry.Request from %s {content=%s}", from, content );
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
Request request = (Request) o;
return !(content != null ? !content.equals( request.content ) : request.content != null);
}
@Override
public int hashCode()
{
return content != null ? content.hashCode() : 0;
}
public ReplicatedContent content()
{
return content;
}
}
class Batch<MEMBER> extends BaseMessage<MEMBER>
{
private List<ReplicatedContent> list;
public Batch( int batchSize )
{
super( null, Type.NEW_BATCH_REQUEST );
list = new ArrayList<>( batchSize );
}
public void add( ReplicatedContent content )
{
list.add( content );
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{ return true; }
if ( o == null || getClass() != o.getClass() )
{ return false; }
if ( !super.equals( o ) )
{ return false; }
Batch<?> batch = (Batch<?>) o;
return Objects.equals( list, batch.list );
}
@Override
public int hashCode()
{
return Objects.hash( super.hashCode(), list );
}
@Override
public String toString()
{
return "Batch{" +
"list=" + list +
'}';
}
public List<ReplicatedContent> contents()
{
return Collections.unmodifiableList( list );
}
}
}
abstract class BaseMessage<MEMBER> implements RaftMessage<MEMBER>
{
protected MEMBER from;
private Type type;
public BaseMessage( MEMBER from, Type type )
{
this.from = from;
this.type = type;
}
@Override
public MEMBER from()
{
return from;
}
@Override
public Type type()
{
return type;
}
@Override
public boolean equals( Object o )
{
if ( this == o )
{
return true;
}
if ( o == null || getClass() != o.getClass() )
{
return false;
}
BaseMessage<?> that = (BaseMessage<?>) o;
return Objects.equals( from, that.from ) &&
Objects.equals( type, that.type );
}
@Override
public int hashCode()
{
return Objects.hash( from, type );
}
}
}