in ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java [1248:1292]
CompletableFuture<RaftClientReply> transferLeadershipAsync(TransferLeadershipRequest request)
throws IOException {
if (request.getNewLeader() == null) {
return stepDownLeaderAsync(request);
}
LOG.info("{}: receive transferLeadership {}", getMemberId(), request);
assertLifeCycleState(LifeCycle.States.RUNNING);
assertGroup(getMemberId(), request);
synchronized (this) {
CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
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 in " + conf;
return logAndReturnTransferLeadershipFail(request, msg);
}
return transferLeadership.start(leaderState, request);
}
}