/
RaftServerImpl.java
1851 lines (1638 loc) · 75.7 KB
/
RaftServerImpl.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
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.ratis.server.impl;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.ratis.client.RaftClient;
import org.apache.ratis.client.impl.ClientProtoUtils;
import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.proto.RaftProtos.*;
import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase;
import org.apache.ratis.protocol.*;
import org.apache.ratis.protocol.exceptions.GroupMismatchException;
import org.apache.ratis.protocol.exceptions.LeaderNotReadyException;
import org.apache.ratis.protocol.exceptions.LeaderSteppingDownException;
import org.apache.ratis.protocol.exceptions.NotLeaderException;
import org.apache.ratis.protocol.exceptions.RaftException;
import org.apache.ratis.protocol.exceptions.ReconfigurationInProgressException;
import org.apache.ratis.protocol.exceptions.ResourceUnavailableException;
import org.apache.ratis.protocol.exceptions.ServerNotReadyException;
import org.apache.ratis.protocol.exceptions.StaleReadException;
import org.apache.ratis.protocol.exceptions.StateMachineException;
import org.apache.ratis.protocol.exceptions.TransferLeadershipException;
import org.apache.ratis.server.DataStreamMap;
import org.apache.ratis.server.DivisionInfo;
import org.apache.ratis.server.DivisionProperties;
import org.apache.ratis.server.leader.FollowerInfo;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.RaftServerConfigKeys;
import org.apache.ratis.server.RaftServerMXBean;
import org.apache.ratis.server.RaftServerRpc;
import org.apache.ratis.server.impl.LeaderElection.Phase;
import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry;
import org.apache.ratis.server.leader.LeaderState;
import org.apache.ratis.server.leader.LogAppender;
import org.apache.ratis.server.metrics.LeaderElectionMetrics;
import org.apache.ratis.server.metrics.RaftServerMetricsImpl;
import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol;
import org.apache.ratis.server.protocol.RaftServerProtocol;
import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.raftlog.LogProtoUtils;
import org.apache.ratis.server.raftlog.RaftLog;
import org.apache.ratis.server.raftlog.RaftLogIOException;
import org.apache.ratis.server.storage.RaftStorage;
import org.apache.ratis.server.storage.RaftStorageDirectory;
import org.apache.ratis.server.util.ServerStringUtils;
import org.apache.ratis.statemachine.SnapshotInfo;
import org.apache.ratis.statemachine.StateMachine;
import org.apache.ratis.statemachine.TransactionContext;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.util.*;
import javax.management.ObjectName;
import java.io.File;
import java.io.IOException;
import java.nio.file.NoSuchFileException;
import java.util.*;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import static org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.INCONSISTENCY;
import static org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.NOT_LEADER;
import static org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.SUCCESS;
import static org.apache.ratis.util.LifeCycle.State.EXCEPTION;
import static org.apache.ratis.util.LifeCycle.State.NEW;
import static org.apache.ratis.util.LifeCycle.State.PAUSED;
import static org.apache.ratis.util.LifeCycle.State.PAUSING;
import static org.apache.ratis.util.LifeCycle.State.RUNNING;
import static org.apache.ratis.util.LifeCycle.State.STARTING;
import com.codahale.metrics.Timer;
class RaftServerImpl implements RaftServer.Division,
RaftServerProtocol, RaftServerAsynchronousProtocol,
RaftClientProtocol, RaftClientAsynchronousProtocol{
private static final String CLASS_NAME = JavaUtils.getClassSimpleName(RaftServerImpl.class);
static final String REQUEST_VOTE = CLASS_NAME + ".requestVote";
static final String APPEND_ENTRIES = CLASS_NAME + ".appendEntries";
static final String INSTALL_SNAPSHOT = CLASS_NAME + ".installSnapshot";
static final String LOG_SYNC = APPEND_ENTRIES + ".logComplete";
static final String START_LEADER_ELECTION = CLASS_NAME + ".startLeaderElection";
class Info implements DivisionInfo {
@Override
public RaftPeerRole getCurrentRole() {
return getRole().getCurrentRole();
}
@Override
public boolean isLeaderReady() {
return isLeader() && getRole().isLeaderReady();
}
@Override
public LifeCycle.State getLifeCycleState() {
return lifeCycle.getCurrentState();
}
@Override
public RoleInfoProto getRoleInfoProto() {
return RaftServerImpl.this.getRoleInfoProto();
}
@Override
public long getCurrentTerm() {
return getState().getCurrentTerm();
}
@Override
public long getLastAppliedIndex() {
return getState().getLastAppliedIndex();
}
@Override
public long[] getFollowerNextIndices() {
return role.getLeaderState()
.filter(leader -> isLeader())
.map(LeaderStateImpl::getFollowerNextIndices)
.orElse(null);
}
}
private final RaftServerProxy proxy;
private final StateMachine stateMachine;
private final Info info = new Info();
private final DivisionProperties divisionProperties;
private final int maxTimeoutMs;
private final TimeDuration leaderStepDownWaitTime;
private final TimeDuration sleepDeviationThreshold;
private final boolean installSnapshotEnabled;
private final LifeCycle lifeCycle;
private final ServerState state;
private final RoleInfo role;
private final DataStreamMap dataStreamMap;
private final MemoizedSupplier<RaftClient> raftClient;
private final RetryCacheImpl retryCache;
private final CommitInfoCache commitInfoCache = new CommitInfoCache();
private final RaftServerJmxAdapter jmxAdapter;
private final LeaderElectionMetrics leaderElectionMetrics;
private final RaftServerMetricsImpl raftServerMetrics;
private final AtomicLong inProgressInstallSnapshotRequest;
private final AtomicLong installedSnapshotIndex;
private final AtomicBoolean isSnapshotNull;
// To avoid append entry before complete start() method
// For example, if thread1 start(), but before thread1 startAsFollower(), thread2 receive append entry
// request, and change state to RUNNING by lifeCycle.compareAndTransition(STARTING, RUNNING),
// then thread1 execute lifeCycle.transition(RUNNING) in startAsFollower(),
// So happens IllegalStateException: ILLEGAL TRANSITION: RUNNING -> RUNNING,
private final AtomicBoolean startComplete;
private final TransferLeadership transferLeadership;
private final SnapshotManagementRequestHandler snapshotRequestHandler;
RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy) throws IOException {
final RaftPeerId id = proxy.getId();
LOG.info("{}: new RaftServerImpl for {} with {}", id, group, stateMachine);
this.lifeCycle = new LifeCycle(id);
this.stateMachine = stateMachine;
this.role = new RoleInfo(id);
final RaftProperties properties = proxy.getProperties();
this.divisionProperties = new DivisionPropertiesImpl(properties);
maxTimeoutMs = properties().maxRpcTimeoutMs();
leaderStepDownWaitTime = RaftServerConfigKeys.LeaderElection.leaderStepDownWaitTime(properties);
this.sleepDeviationThreshold = RaftServerConfigKeys.sleepDeviationThreshold(properties);
installSnapshotEnabled = RaftServerConfigKeys.Log.Appender.installSnapshotEnabled(properties);
this.proxy = proxy;
this.state = new ServerState(id, group, properties, this, stateMachine);
this.retryCache = new RetryCacheImpl(properties);
this.inProgressInstallSnapshotRequest = new AtomicLong();
this.installedSnapshotIndex = new AtomicLong();
this.isSnapshotNull = new AtomicBoolean(false);
this.dataStreamMap = new DataStreamMapImpl(id);
this.jmxAdapter = new RaftServerJmxAdapter();
this.leaderElectionMetrics = LeaderElectionMetrics.getLeaderElectionMetrics(
getMemberId(), state::getLastLeaderElapsedTimeMs);
this.raftServerMetrics = RaftServerMetricsImpl.computeIfAbsentRaftServerMetrics(
getMemberId(), () -> commitInfoCache::get, retryCache::getStatistics);
this.startComplete = new AtomicBoolean(false);
this.raftClient = JavaUtils.memoize(() -> {
RaftClient client = RaftClient.newBuilder()
.setRaftGroup(group)
.setProperties(getRaftServer().getProperties())
.build();
return client;
});
this.transferLeadership = new TransferLeadership(this);
this.snapshotRequestHandler = new SnapshotManagementRequestHandler(this);
}
@Override
public DivisionProperties properties() {
return divisionProperties;
}
LogAppender newLogAppender(LeaderState leaderState, FollowerInfo f) {
return getRaftServer().getFactory().newLogAppender(this, leaderState, f);
}
int getMaxTimeoutMs() {
return maxTimeoutMs;
}
TimeDuration getRandomElectionTimeout() {
final int min = properties().minRpcTimeoutMs();
final int millis = min + ThreadLocalRandom.current().nextInt(properties().maxRpcTimeoutMs() - min + 1);
return TimeDuration.valueOf(millis, TimeUnit.MILLISECONDS);
}
TimeDuration getLeaderStepDownWaitTime() {
return leaderStepDownWaitTime;
}
TimeDuration getSleepDeviationThreshold() {
return sleepDeviationThreshold;
}
@Override
public StateMachine getStateMachine() {
return stateMachine;
}
@Override
public RaftLog getRaftLog() {
return getState().getLog();
}
@Override
public RaftStorage getRaftStorage() {
return getState().getStorage();
}
@Override
public DataStreamMap getDataStreamMap() {
return dataStreamMap;
}
@Override
public RaftClient getRaftClient() {
return raftClient.get();
}
@Override
public RetryCacheImpl getRetryCache() {
return retryCache;
}
@Override
public RaftServerProxy getRaftServer() {
return proxy;
}
RaftServerRpc getServerRpc() {
return proxy.getServerRpc();
}
private void setRole(RaftPeerRole newRole, Object reason) {
LOG.info("{}: changes role from {} to {} at term {} for {}",
getMemberId(), this.role, newRole, state.getCurrentTerm(), reason);
this.role.transitionRole(newRole);
}
boolean start() {
if (!lifeCycle.compareAndTransition(NEW, STARTING)) {
return false;
}
final RaftConfigurationImpl conf = getRaftConf();
if (conf != null && conf.containsInBothConfs(getId())) {
LOG.info("{}: start as a follower, conf={}", getMemberId(), conf);
startAsFollower();
} else {
LOG.info("{}: start with initializing state, conf={}", getMemberId(), conf);
startInitializing();
}
registerMBean(getId(), getMemberId().getGroupId(), jmxAdapter, jmxAdapter);
state.start();
startComplete.compareAndSet(false, true);
return true;
}
static boolean registerMBean(
RaftPeerId id, RaftGroupId groupdId, RaftServerMXBean mBean, JmxRegister jmx) {
final String prefix = "Ratis:service=RaftServer,group=" + groupdId + ",id=";
final String registered = jmx.register(mBean, Arrays.asList(
() -> prefix + id,
() -> prefix + ObjectName.quote(id.toString())));
return registered != null;
}
/**
* The peer belongs to the current configuration, should start as a follower
*/
private void startAsFollower() {
setRole(RaftPeerRole.FOLLOWER, "startAsFollower");
role.startFollowerState(this, "startAsFollower");
lifeCycle.transition(RUNNING);
}
/**
* The peer does not have any configuration (maybe it will later be included
* in some configuration). Start still as a follower but will not vote or
* start election.
*/
private void startInitializing() {
setRole(RaftPeerRole.FOLLOWER, "startInitializing");
// do not start FollowerState
}
ServerState getState() {
return state;
}
@Override
public RaftGroupMemberId getMemberId() {
return getState().getMemberId();
}
@Override
public DivisionInfo getInfo() {
return info;
}
RoleInfo getRole() {
return role;
}
@Override
public RaftConfigurationImpl getRaftConf() {
return getState().getRaftConf();
}
/**
* This removes the group from the server.
* If the deleteDirectory flag is set to false, and renameDirectory
* the directory is moved to
* {@link RaftServerConfigKeys#REMOVED_GROUPS_DIR_KEY} location.
* If the deleteDirectory flag is true, the group is permanently deleted.
*/
void groupRemove(boolean deleteDirectory, boolean renameDirectory) {
final RaftStorageDirectory dir = state.getStorage().getStorageDir();
/* Shutdown is triggered here inorder to avoid any locked files. */
close();
getStateMachine().event().notifyGroupRemove();
if (deleteDirectory) {
for (int i = 0; i < FileUtils.NUM_ATTEMPTS; i ++) {
try {
FileUtils.deleteFully(dir.getRoot());
LOG.info("{}: Succeed to remove RaftStorageDirectory {}", getMemberId(), dir);
break;
} catch (NoSuchFileException e) {
LOG.warn("{}: Some file does not exist {}", getMemberId(), dir, e);
} catch (Exception ignored) {
LOG.error("{}: Failed to remove RaftStorageDirectory {}", getMemberId(), dir, ignored);
break;
}
}
} else if(renameDirectory) {
try {
/* Create path with current group in REMOVED_GROUPS_DIR_KEY location */
File toBeRemovedGroupFolder = new File(RaftServerConfigKeys
.removedGroupsDir(proxy.getProperties()),
dir.getRoot().getName());
FileUtils.moveDirectory(dir.getRoot().toPath(),
toBeRemovedGroupFolder.toPath());
LOG.info("{}: Group {} is renamed successfully", getMemberId(), getGroup());
} catch (IOException e) {
LOG.warn("{}: Failed to remove group {}", getMemberId(),
dir.getRoot().getName(), e);
}
}
}
@Override
public void close() {
lifeCycle.checkStateAndClose(() -> {
LOG.info("{}: shutdown", getMemberId());
try {
jmxAdapter.unregister();
} catch (Exception ignored) {
LOG.warn("{}: Failed to un-register RaftServer JMX bean", getMemberId(), ignored);
}
try {
role.shutdownFollowerState();
} catch (Exception ignored) {
LOG.warn("{}: Failed to shutdown FollowerState", getMemberId(), ignored);
}
try{
role.shutdownLeaderElection();
} catch (Exception ignored) {
LOG.warn("{}: Failed to shutdown LeaderElection", getMemberId(), ignored);
}
try{
role.shutdownLeaderState(true);
} catch (Exception ignored) {
LOG.warn("{}: Failed to shutdown LeaderState monitor", getMemberId(), ignored);
}
try{
state.close();
} catch (Exception ignored) {
LOG.warn("{}: Failed to close state", getMemberId(), ignored);
}
try {
leaderElectionMetrics.unregister();
raftServerMetrics.unregister();
RaftServerMetricsImpl.removeRaftServerMetrics(getMemberId());
} catch (Exception ignored) {
LOG.warn("{}: Failed to unregister metric", getMemberId(), ignored);
}
try {
if (raftClient.isInitialized()) {
raftClient.get().close();
}
} catch (Exception ignored) {
LOG.warn("{}: Failed to close raft client", getMemberId(), ignored);
}
});
}
/**
* Change the server state to Follower if this server is in a different role or force is true.
* @param newTerm The new term.
* @param force Force to start a new {@link FollowerState} even if this server is already a follower.
* @return if the term/votedFor should be updated to the new term
*/
private synchronized boolean changeToFollower(long newTerm, boolean force, Object reason) {
final RaftPeerRole old = role.getCurrentRole();
final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
if (old != RaftPeerRole.FOLLOWER || force) {
setRole(RaftPeerRole.FOLLOWER, reason);
if (old == RaftPeerRole.LEADER) {
role.shutdownLeaderState(false);
} else if (old == RaftPeerRole.CANDIDATE) {
role.shutdownLeaderElection();
} else if (old == RaftPeerRole.FOLLOWER) {
role.shutdownFollowerState();
}
role.startFollowerState(this, reason);
}
return metadataUpdated;
}
synchronized void changeToFollowerAndPersistMetadata(long newTerm, Object reason) throws IOException {
if (changeToFollower(newTerm, false, reason)) {
state.persistMetadata();
}
}
synchronized void changeToLeader() {
Preconditions.assertTrue(getInfo().isCandidate());
role.shutdownLeaderElection();
setRole(RaftPeerRole.LEADER, "changeToLeader");
state.becomeLeader();
// start sending AppendEntries RPC to followers
final LogEntryProto e = role.startLeaderState(this);
getState().setRaftConf(e);
}
@Override
public Collection<CommitInfoProto> getCommitInfos() {
final List<CommitInfoProto> infos = new ArrayList<>();
// add the commit info of this server
infos.add(updateCommitInfoCache());
// add the commit infos of other servers
if (getInfo().isLeader()) {
role.getLeaderState().ifPresent(
leader -> leader.updateFollowerCommitInfos(commitInfoCache, infos));
} else {
getRaftConf().getAllPeers().stream()
.map(RaftPeer::getId)
.filter(id -> !id.equals(getId()))
.map(commitInfoCache::get)
.filter(Objects::nonNull)
.forEach(infos::add);
}
return infos;
}
GroupInfoReply getGroupInfo(GroupInfoRequest request) {
return new GroupInfoReply(request, getCommitInfos(),
getGroup(), getRoleInfoProto(), state.getStorage().getStorageDir().isHealthy());
}
private RoleInfoProto getRoleInfoProto(RaftPeer leaderPeerInfo) {
RaftPeerRole currentRole = role.getCurrentRole();
RoleInfoProto.Builder roleInfo = RoleInfoProto.newBuilder()
.setSelf(getPeer().getRaftPeerProto())
.setRole(currentRole)
.setRoleElapsedTimeMs(role.getRoleElapsedTimeMs());
final Optional<FollowerState> fs = role.getFollowerState();
final ServerRpcProto leaderInfo =
ServerProtoUtils.toServerRpcProto(leaderPeerInfo,
fs.map(FollowerState::getLastRpcTime).map(Timestamp::elapsedTimeMs).orElse(0L));
roleInfo.setFollowerInfo(FollowerInfoProto.newBuilder().setLeaderInfo(leaderInfo)
.setOutstandingOp(fs.map(FollowerState::getOutstandingOp).orElse(0)));
return roleInfo.build();
}
RoleInfoProto getRoleInfoProto() {
RaftPeerRole currentRole = role.getCurrentRole();
RoleInfoProto.Builder roleInfo = RoleInfoProto.newBuilder()
.setSelf(getPeer().getRaftPeerProto())
.setRole(currentRole)
.setRoleElapsedTimeMs(role.getRoleElapsedTimeMs());
switch (currentRole) {
case CANDIDATE:
CandidateInfoProto.Builder candidate = CandidateInfoProto.newBuilder()
.setLastLeaderElapsedTimeMs(state.getLastLeaderElapsedTimeMs());
roleInfo.setCandidateInfo(candidate);
break;
case FOLLOWER:
final Optional<FollowerState> fs = role.getFollowerState();
final ServerRpcProto leaderInfo = ServerProtoUtils.toServerRpcProto(
getRaftConf().getPeer(state.getLeaderId()),
fs.map(FollowerState::getLastRpcTime).map(Timestamp::elapsedTimeMs).orElse(0L));
// FollowerState can be null while adding a new peer as it is not
// a voting member yet
roleInfo.setFollowerInfo(FollowerInfoProto.newBuilder()
.setLeaderInfo(leaderInfo)
.setOutstandingOp(fs.map(FollowerState::getOutstandingOp).orElse(0)));
break;
case LEADER:
role.getLeaderState().ifPresent(ls -> {
final LeaderInfoProto.Builder leader = LeaderInfoProto.newBuilder();
ls.getLogAppenders().map(LogAppender::getFollower).forEach(f ->
leader.addFollowerInfo(ServerProtoUtils.toServerRpcProto(
f.getPeer(), f.getLastRpcResponseTime().elapsedTimeMs())));
leader.setTerm(ls.getCurrentTerm());
roleInfo.setLeaderInfo(leader);
});
break;
default:
throw new IllegalStateException("incorrect role of server " + currentRole);
}
return roleInfo.build();
}
synchronized void changeToCandidate(boolean forceStartLeaderElection) {
Preconditions.assertTrue(getInfo().isFollower());
role.shutdownFollowerState();
setRole(RaftPeerRole.CANDIDATE, "changeToCandidate");
if (state.shouldNotifyExtendedNoLeader()) {
stateMachine.followerEvent().notifyExtendedNoLeader(getRoleInfoProto());
}
// start election
role.startLeaderElection(this, forceStartLeaderElection);
}
@Override
public String toString() {
return role + " " + state + " " + lifeCycle.getCurrentState();
}
RaftClientReply.Builder newReplyBuilder(RaftClientRequest request) {
return RaftClientReply.newBuilder()
.setRequest(request)
.setCommitInfos(getCommitInfos());
}
private RaftClientReply.Builder newReplyBuilder(ClientInvocationId invocationId, long logIndex) {
return RaftClientReply.newBuilder()
.setClientInvocationId(invocationId)
.setLogIndex(logIndex)
.setServerId(getMemberId())
.setCommitInfos(getCommitInfos());
}
RaftClientReply newSuccessReply(RaftClientRequest request) {
return newReplyBuilder(request)
.setSuccess()
.build();
}
RaftClientReply newSuccessReply(RaftClientRequest request, long logIndex) {
return newReplyBuilder(request)
.setSuccess()
.setLogIndex(logIndex)
.build();
}
RaftClientReply newExceptionReply(RaftClientRequest request, RaftException exception) {
return newReplyBuilder(request)
.setException(exception)
.build();
}
/**
* @return null if the server is in leader state.
*/
private CompletableFuture<RaftClientReply> checkLeaderState(RaftClientRequest request, CacheEntry entry,
boolean isWrite) {
try {
assertGroup(request.getRequestorId(), request.getRaftGroupId());
} catch (GroupMismatchException e) {
return RetryCacheImpl.failWithException(e, entry);
}
if (!getInfo().isLeader()) {
NotLeaderException exception = generateNotLeaderException();
final RaftClientReply reply = newExceptionReply(request, exception);
return RetryCacheImpl.failWithReply(reply, entry);
}
if (!getInfo().isLeaderReady()) {
final CacheEntry cacheEntry = retryCache.getIfPresent(ClientInvocationId.valueOf(request));
if (cacheEntry != null && cacheEntry.isCompletedNormally()) {
return cacheEntry.getReplyFuture();
}
final LeaderNotReadyException lnre = new LeaderNotReadyException(getMemberId());
final RaftClientReply reply = newExceptionReply(request, lnre);
return RetryCacheImpl.failWithReply(reply, entry);
}
if (isWrite && isSteppingDown()) {
final LeaderSteppingDownException lsde = new LeaderSteppingDownException(getMemberId() + " is stepping down");
final RaftClientReply reply = newExceptionReply(request, lsde);
return RetryCacheImpl.failWithReply(reply, entry);
}
return null;
}
NotLeaderException generateNotLeaderException() {
if (lifeCycle.getCurrentState() != RUNNING) {
return new NotLeaderException(getMemberId(), null, null);
}
RaftPeerId leaderId = state.getLeaderId();
if (leaderId == null || leaderId.equals(getId())) {
// No idea about who is the current leader. Or the peer is the current
// leader, but it is about to step down. set the suggested leader as null.
leaderId = null;
}
final RaftConfigurationImpl conf = getRaftConf();
Collection<RaftPeer> peers = conf.getAllPeers();
return new NotLeaderException(getMemberId(), conf.getPeer(leaderId), peers);
}
private LifeCycle.State assertLifeCycleState(Set<LifeCycle.State> expected) throws ServerNotReadyException {
return lifeCycle.assertCurrentState((n, c) -> new ServerNotReadyException(
getMemberId() + " is not in " + expected + ": current state is " + c),
expected);
}
void assertGroup(Object requestorId, RaftGroupId requestorGroupId) throws GroupMismatchException {
final RaftGroupId groupId = getMemberId().getGroupId();
if (!groupId.equals(requestorGroupId)) {
throw new GroupMismatchException(getMemberId()
+ ": The group (" + requestorGroupId + ") of " + requestorId
+ " does not match the group (" + groupId + ") of the server " + getId());
}
}
/**
* Handle a normal update request from client.
*/
private CompletableFuture<RaftClientReply> appendTransaction(
RaftClientRequest request, TransactionContext context, CacheEntry cacheEntry) throws IOException {
assertLifeCycleState(LifeCycle.States.RUNNING);
CompletableFuture<RaftClientReply> reply;
final PendingRequest pending;
synchronized (this) {
reply = checkLeaderState(request, cacheEntry, true);
if (reply != null) {
return reply;
}
// append the message to its local log
final LeaderStateImpl leaderState = role.getLeaderStateNonNull();
final PendingRequests.Permit permit = leaderState.tryAcquirePendingRequest(request.getMessage());
if (permit == null) {
cacheEntry.failWithException(new ResourceUnavailableException(
getMemberId() + ": Failed to acquire a pending write request for " + request));
return cacheEntry.getReplyFuture();
}
try {
state.appendLog(context);
} catch (StateMachineException e) {
// the StateMachineException is thrown by the SM in the preAppend stage.
// Return the exception in a RaftClientReply.
RaftClientReply exceptionReply = newExceptionReply(request, e);
cacheEntry.failWithReply(exceptionReply);
// leader will step down here
if (e.leaderShouldStepDown() && getInfo().isLeader()) {
leaderState.submitStepDownEvent(LeaderState.StepDownReason.STATE_MACHINE_EXCEPTION);
}
return CompletableFuture.completedFuture(exceptionReply);
}
// put the request into the pending queue
pending = leaderState.addPendingRequest(permit, request, context);
if (pending == null) {
cacheEntry.failWithException(new ResourceUnavailableException(
getMemberId() + ": Failed to add a pending write request for " + request));
return cacheEntry.getReplyFuture();
}
leaderState.notifySenders();
}
return pending.getFuture();
}
void stepDownOnJvmPause() {
role.getLeaderState().ifPresent(leader -> leader.submitStepDownEvent(LeaderState.StepDownReason.JVM_PAUSE));
}
private RaftClientRequest filterDataStreamRaftClientRequest(RaftClientRequest request)
throws InvalidProtocolBufferException {
return !request.is(TypeCase.FORWARD) ? request : ClientProtoUtils.toRaftClientRequest(
RaftClientRequestProto.parseFrom(
request.getMessage().getContent().asReadOnlyByteBuffer()));
}
@Override
public CompletableFuture<RaftClientReply> submitClientRequestAsync(
RaftClientRequest request) throws IOException {
assertLifeCycleState(LifeCycle.States.RUNNING);
LOG.debug("{}: receive client request({})", getMemberId(), request);
final Optional<Timer> timer = Optional.ofNullable(raftServerMetrics.getClientRequestTimer(request.getType()));
final CompletableFuture<RaftClientReply> replyFuture;
if (request.is(TypeCase.STALEREAD)) {
replyFuture = staleReadAsync(request);
} else {
// first check the server's leader state
CompletableFuture<RaftClientReply> reply = checkLeaderState(request, null,
!request.is(TypeCase.READ) && !request.is(TypeCase.WATCH));
if (reply != null) {
return reply;
}
// let the state machine handle read-only request from client
RaftClientRequest.Type type = request.getType();
if (type.is(TypeCase.MESSAGESTREAM)) {
if (type.getMessageStream().getEndOfRequest()) {
final CompletableFuture<RaftClientRequest> f = streamEndOfRequestAsync(request);
if (f.isCompletedExceptionally()) {
return f.thenApply(r -> null);
}
request = f.join();
type = request.getType();
}
}
if (type.is(TypeCase.READ)) {
// TODO: We might not be the leader anymore by the time this completes.
// See the RAFT paper section 8 (last part)
replyFuture = processQueryFuture(stateMachine.query(request.getMessage()), request);
} else if (type.is(TypeCase.WATCH)) {
replyFuture = watchAsync(request);
} else if (type.is(TypeCase.MESSAGESTREAM)) {
replyFuture = streamAsync(request);
} else {
// query the retry cache
final RetryCacheImpl.CacheQueryResult queryResult = retryCache.queryCache(ClientInvocationId.valueOf(request));
final CacheEntry cacheEntry = queryResult.getEntry();
if (queryResult.isRetry()) {
// if the previous attempt is still pending or it succeeded, return its
// future
replyFuture = cacheEntry.getReplyFuture();
} else {
// TODO: this client request will not be added to pending requests until
// later which means that any failure in between will leave partial state in
// the state machine. We should call cancelTransaction() for failed requests
TransactionContext context = stateMachine.startTransaction(filterDataStreamRaftClientRequest(request));
if (context.getException() != null) {
final StateMachineException e = new StateMachineException(getMemberId(), context.getException());
final RaftClientReply exceptionReply = newExceptionReply(request, e);
cacheEntry.failWithReply(exceptionReply);
replyFuture = CompletableFuture.completedFuture(exceptionReply);
} else {
replyFuture = appendTransaction(request, context, cacheEntry);
}
}
}
}
final RaftClientRequest.Type type = request.getType();
replyFuture.whenComplete((clientReply, exception) -> {
if (clientReply.isSuccess()) {
timer.map(Timer::time).ifPresent(Timer.Context::stop);
}
if (exception != null || clientReply.getException() != null) {
raftServerMetrics.incFailedRequestCount(type);
}
});
return replyFuture;
}
private CompletableFuture<RaftClientReply> watchAsync(RaftClientRequest request) {
return role.getLeaderState()
.map(ls -> ls.addWatchReqeust(request))
.orElseGet(() -> CompletableFuture.completedFuture(
newExceptionReply(request, generateNotLeaderException())));
}
private CompletableFuture<RaftClientReply> staleReadAsync(RaftClientRequest request) {
final long minIndex = request.getType().getStaleRead().getMinIndex();
final long commitIndex = state.getLog().getLastCommittedIndex();
LOG.debug("{}: minIndex={}, commitIndex={}", getMemberId(), minIndex, commitIndex);
if (commitIndex < minIndex) {
final StaleReadException e = new StaleReadException(
"Unable to serve stale-read due to server commit index = " + commitIndex + " < min = " + minIndex);
return CompletableFuture.completedFuture(
newExceptionReply(request, new StateMachineException(getMemberId(), e)));
}
return processQueryFuture(stateMachine.queryStale(request.getMessage(), minIndex), request);
}
private CompletableFuture<RaftClientReply> streamAsync(RaftClientRequest request) {
return role.getLeaderState()
.map(ls -> ls.streamAsync(request))
.orElseGet(() -> CompletableFuture.completedFuture(
newExceptionReply(request, generateNotLeaderException())));
}
private CompletableFuture<RaftClientRequest> streamEndOfRequestAsync(RaftClientRequest request) {
return role.getLeaderState()
.map(ls -> ls.streamEndOfRequestAsync(request))
.orElse(null);
}
CompletableFuture<RaftClientReply> processQueryFuture(
CompletableFuture<Message> queryFuture, RaftClientRequest request) {
return queryFuture.thenApply(r -> newReplyBuilder(request).setSuccess().setMessage(r).build())
.exceptionally(e -> {
e = JavaUtils.unwrapCompletionException(e);
if (e instanceof StateMachineException) {
return newExceptionReply(request, (StateMachineException)e);
}
throw new CompletionException(e);
});
}
@Override
public RaftClientReply submitClientRequest(RaftClientRequest request)
throws IOException {
return waitForReply(request, submitClientRequestAsync(request));
}
RaftClientReply waitForReply(RaftClientRequest request, CompletableFuture<RaftClientReply> future)
throws IOException {
return waitForReply(getMemberId(), request, future, e -> newExceptionReply(request, e));
}
static <REPLY extends RaftClientReply> REPLY waitForReply(
Object id, RaftClientRequest request, CompletableFuture<REPLY> future,
Function<RaftException, REPLY> exceptionReply)
throws IOException {
try {
return future.get();
} catch (InterruptedException e) {
final String s = id + ": Interrupted when waiting for reply, request=" + request;
LOG.info(s, e);
Thread.currentThread().interrupt();
throw IOUtils.toInterruptedIOException(s, e);
} catch (ExecutionException e) {
final Throwable cause = e.getCause();
if (cause == null) {
throw new IOException(e);
}
if (cause instanceof NotLeaderException ||
cause instanceof StateMachineException) {
final REPLY reply = exceptionReply.apply((RaftException) cause);
if (reply != null) {
return reply;
}
}
throw IOUtils.asIOException(cause);
}
}
public RaftClientReply transferLeadership(TransferLeadershipRequest request) throws IOException {
return waitForReply(request, transferLeadershipAsync(request));
}
private CompletableFuture<RaftClientReply> logAndReturnTransferLeadershipFail(
TransferLeadershipRequest request, String msg) {
LOG.warn(msg);
return CompletableFuture.completedFuture(
newExceptionReply(request, new TransferLeadershipException(msg)));
}
boolean isSteppingDown() {
return transferLeadership.isSteppingDown();
}
void finishTransferLeadership() {
transferLeadership.finish(state.getLeaderId(), false);
}
public CompletableFuture<RaftClientReply> transferLeadershipAsync(TransferLeadershipRequest request)
throws IOException {
LOG.info("{}: receive transferLeadership {}", getMemberId(), request);
assertLifeCycleState(LifeCycle.States.RUNNING);
assertGroup(request.getRequestorId(), request.getRaftGroupId());
synchronized (this) {
CompletableFuture<RaftClientReply> reply = checkLeaderState(request, null, false);
if (reply != null) {
return reply;
}
if (getId().equals(request.getNewLeader())) {
return CompletableFuture.completedFuture(newSuccessReply(request));
}
final RaftConfigurationImpl conf = getRaftConf();
final LeaderStateImpl leaderState = role.getLeaderStateNonNull();
// make sure there is no raft reconfiguration in progress
if (!conf.isStable() || leaderState.inStagingState() || !state.isConfCommitted()) {
String msg = getMemberId() + " refused to transfer leadership to peer " + request.getNewLeader() +
" when raft reconfiguration in progress.";
return logAndReturnTransferLeadershipFail(request, msg);
}
if (!conf.containsInConf(request.getNewLeader())) {
String msg = getMemberId() + " refused to transfer leadership to peer " + request.getNewLeader() +
" as it is not in " + conf;
return logAndReturnTransferLeadershipFail(request, msg);
}
if (!conf.isHighestPriority(request.getNewLeader())) {
String msg = getMemberId() + " refused to transfer leadership to peer " + request.getNewLeader() +
" as it does not has highest priority " + conf;
return logAndReturnTransferLeadershipFail(request, msg);
}
return transferLeadership.start(request);
}
}
public RaftClientReply takeSnapshot(SnapshotManagementRequest request) throws IOException {
return waitForReply(request, takeSnapshotAsync(request));
}
CompletableFuture<RaftClientReply> takeSnapshotAsync(SnapshotManagementRequest request) throws IOException {
LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request);
assertLifeCycleState(LifeCycle.States.RUNNING);
assertGroup(request.getRequestorId(), request.getRaftGroupId());
//TODO(liuyaolong): get the gap value from shell command
long minGapValue = RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties());
final long lastSnapshotIndex = Optional.ofNullable(stateMachine.getLatestSnapshot())
.map(SnapshotInfo::getIndex)
.orElse(0L);
if (state.getLastAppliedIndex() - lastSnapshotIndex < minGapValue) {
return CompletableFuture.completedFuture(newSuccessReply(request, lastSnapshotIndex));
}
synchronized (this) {
long installSnapshot = inProgressInstallSnapshotRequest.get();
// check snapshot install/load
if (installSnapshot != 0) {
String msg = String.format("%s: Failed do snapshot as snapshot (%s) installation is in progress",
getMemberId(), installSnapshot);