LogEntryProto startLeaderState(RaftServerImpl server, RaftProperties properties) { return updateAndGet(leaderState, new LeaderState(server, properties)).start(); }
private CompletableFuture<RaftClientReply> watchAsync(RaftClientRequest request) { return role.getLeaderState() .map(ls -> ls.addWatchReqeust(request)) .orElseGet(() -> CompletableFuture.completedFuture( new RaftClientReply(request, generateNotLeaderException(), getCommitInfos()))); }
private void checkNewPeers() { if (!inStagingState()) { // it is possible that the bootstrapping is done and we still have // remaining STAGINGPROGRESS event to handle. updateLastCommitted(); } else { final long committedIndex = server.getState().getLog() .getLastCommittedIndex(); Collection<BootStrapProgress> reports = checkAllProgress(committedIndex); if (reports.contains(BootStrapProgress.NOPROGRESS)) { LOG.debug("{} fails the setConfiguration request", server.getId()); stagingState.fail(); } else if (!reports.contains(BootStrapProgress.PROGRESSING)) { // all caught up! applyOldNewConf(); senders.forEach(s -> s.getFollower().startAttendVote()); } } }
private void handleEvent(StateUpdateEvent e) throws IOException { if (e == null) { if (inStagingState()) { checkNewPeers(); } } else { if (e.type == STEPDOWN) { server.changeToFollower(e.newTerm, true); } else if (e.type == UPDATECOMMIT) { updateLastCommitted(); } else if (e.type == STAGINGPROGRESS) { checkNewPeers(); } } }
private Collection<BootStrapProgress> checkAllProgress(long committed) { Preconditions.assertTrue(inStagingState()); return senders.stream() .filter(sender -> !sender.getFollower().isAttendingVote()) .map(sender -> checkProgress(sender.getFollower(), committed)) .collect(Collectors.toCollection(ArrayList::new)); }
/** * Start bootstrapping new peers */ PendingRequest startSetConfiguration(SetConfigurationRequest request) { Preconditions.assertTrue(running && !inStagingState()); RaftPeer[] peersInNewConf = request.getPeersInNewConf(); Collection<RaftPeer> peersToBootStrap = RaftConfiguration .computeNewPeers(peersInNewConf, server.getRaftConf()); // add the request to the pending queue final PendingRequest pending = pendingRequests.addConfRequest(request); ConfigurationStagingState stagingState = new ConfigurationStagingState( peersToBootStrap, new PeerConfiguration(Arrays.asList(peersInNewConf))); Collection<RaftPeer> newPeers = stagingState.getNewPeers(); // set the staging state this.stagingState = stagingState; if (newPeers.isEmpty()) { applyOldNewConf(); } else { // update the LeaderState's sender list addSenders(newPeers); } return pending; }
private void updateCommit(long majority, long min) { final long oldLastCommitted = raftLog.getLastCommittedIndex(); if (majority > oldLastCommitted) { // copy the entries out from the raftlog, in order to prevent that // the log gets purged after the statemachine does a snapshot final TermIndex[] entriesToCommit = raftLog.getEntries( oldLastCommitted + 1, majority + 1); if (server.getState().updateStatemachine(majority, currentTerm)) { watchRequests.update(ReplicationLevel.MAJORITY, majority); logMetadata(majority); commitIndexChanged(); } checkAndUpdateConfiguration(entriesToCommit); } watchRequests.update(ReplicationLevel.ALL, min); }
private void checkStaging() { if (!inStagingState()) { // it is possible that the bootstrapping is done. Then, fallback to UPDATE_COMMIT UPDATE_COMMIT_EVENT.execute(); } else { final long committedIndex = server.getState().getLog() .getLastCommittedIndex(); Collection<BootStrapProgress> reports = checkAllProgress(committedIndex); if (reports.contains(BootStrapProgress.NOPROGRESS)) { LOG.debug("{} fails the setConfiguration request", server.getId()); stagingState.fail(); } else if (!reports.contains(BootStrapProgress.PROGRESSING)) { // all caught up! applyOldNewConf(); senders.forEach(s -> s.getFollower().startAttendVote()); } } }
final long[] indicesInNewConf = computeCommittedIndices(followers, includeSelf); final long majorityInNewConf = getMajority(indicesInNewConf); final long majority; final long min; final long[] indicesInOldConf = computeCommittedIndices(oldFollowers, includeSelfInOldConf); final long majorityInOldConf = getMajority(indicesInOldConf); majority = Math.min(majorityInNewConf, majorityInOldConf); min = Math.min(indicesInNewConf[0], indicesInOldConf[0]); oldLastCommitted + 1, majority + 1); server.getState().updateStatemachine(majority, currentTerm); checkAndUpdateConfiguration(entriesToCommit);
leaderState.submitStepDownEvent(); pending = leaderState.addPendingRequest(request, context); leaderState.notifySenders();
public static Stream<LogAppender> getLogAppenders(RaftServerImpl server) { return server.getLeaderState().getLogAppenders(); }
pending = leaderState.addPendingRequest(entryIndex, request, context); leaderState.notifySenders();
/** * Start bootstrapping new peers */ PendingRequest startSetConfiguration(SetConfigurationRequest request) { Preconditions.assertTrue(running && !inStagingState()); RaftPeer[] peersInNewConf = request.getPeersInNewConf(); Collection<RaftPeer> peersToBootStrap = RaftConfiguration .computeNewPeers(peersInNewConf, server.getRaftConf()); // add the request to the pending queue final PendingRequest pending = pendingRequests.addConfRequest(request); ConfigurationStagingState stagingState = new ConfigurationStagingState( peersToBootStrap, new PeerConfiguration(Arrays.asList(peersInNewConf))); Collection<RaftPeer> newPeers = stagingState.getNewPeers(); // set the staging state this.stagingState = stagingState; if (newPeers.isEmpty()) { applyOldNewConf(); } else { // update the LeaderState's sender list addSenders(newPeers); } return pending; }
private Collection<BootStrapProgress> checkAllProgress(long committed) { Preconditions.assertTrue(inStagingState()); return senders.stream() .filter(sender -> !sender.getFollower().isAttendingVote()) .map(sender -> checkProgress(sender.getFollower(), committed)) .collect(Collectors.toCollection(ArrayList::new)); }
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())));
synchronized void changeToLeader() { Preconditions.assertTrue(isCandidate()); shutdownElectionDaemon(); setRole(Role.LEADER, "changeToLeader"); state.becomeLeader(); // start sending AppendEntries RPC to followers leaderState = new LeaderState(this, getProxy().getProperties()); leaderState.start(); }