in ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java [1363:1432]
public CompletableFuture<RaftClientReply> setConfigurationAsync(SetConfigurationRequest request) throws IOException {
LOG.info("{}: receive setConfiguration {}", getMemberId(), request);
assertLifeCycleState(LifeCycle.States.RUNNING);
assertGroup(getMemberId(), request);
CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
if (reply != null) {
return reply;
}
final SetConfigurationRequest.Arguments arguments = request.getArguments();
final PendingRequest pending;
synchronized (this) {
reply = checkLeaderState(request);
if (reply != null) {
return reply;
}
final RaftConfigurationImpl current = getRaftConf();
final LeaderStateImpl leaderState = role.getLeaderStateNonNull();
// make sure there is no other raft reconfiguration in progress
if (!current.isStable() || leaderState.inStagingState() || !state.isConfCommitted()) {
throw new ReconfigurationInProgressException(
"Reconfiguration is already in progress: " + current);
}
final List<RaftPeer> serversInNewConf;
final List<RaftPeer> listenersInNewConf;
if (arguments.getMode() == SetConfigurationRequest.Mode.ADD) {
serversInNewConf = add(RaftPeerRole.FOLLOWER, current, arguments);
listenersInNewConf = add(RaftPeerRole.LISTENER, current, arguments);
} else if (arguments.getMode() == SetConfigurationRequest.Mode.COMPARE_AND_SET) {
final Comparator<RaftPeer> comparator = Comparator.comparing(RaftPeer::getId,
Comparator.comparing(RaftPeerId::toString));
if (CollectionUtils.equalsIgnoreOrder(arguments.getServersInCurrentConf(),
current.getAllPeers(RaftPeerRole.FOLLOWER), comparator)
&& CollectionUtils.equalsIgnoreOrder(arguments.getListenersInCurrentConf(),
current.getAllPeers(RaftPeerRole.LISTENER), comparator)) {
serversInNewConf = arguments.getPeersInNewConf(RaftPeerRole.FOLLOWER);
listenersInNewConf = arguments.getPeersInNewConf(RaftPeerRole.LISTENER);
} else {
throw new SetConfigurationException("Failed to set configuration: current configuration "
+ current + " is different than the request " + request);
}
} else {
serversInNewConf = arguments.getPeersInNewConf(RaftPeerRole.FOLLOWER);
listenersInNewConf = arguments.getPeersInNewConf(RaftPeerRole.LISTENER);
}
// return success with a null message if the new conf is the same as the current
if (current.hasNoChange(serversInNewConf, listenersInNewConf)) {
pending = new PendingRequest(request);
pending.setReply(newSuccessReply(request));
return pending.getFuture();
}
if (current.changeMajority(serversInNewConf)) {
if (!memberMajorityAddEnabled) {
throw new SetConfigurationException("Failed to set configuration: request " + request
+ " changes a majority set of the current configuration " + current);
}
LOG.warn("Try to add/replace a majority of servers in a single setConf: {}", request);
}
getRaftServer().addRaftPeers(serversInNewConf);
getRaftServer().addRaftPeers(listenersInNewConf);
// add staging state into the leaderState
pending = leaderState.startSetConfiguration(request, serversInNewConf);
}
return pending.getFuture();
}