diff --git a/protocols/raft/src/main/java/io/atomix/protocols/raft/roles/CandidateRole.java b/protocols/raft/src/main/java/io/atomix/protocols/raft/roles/CandidateRole.java index 1dbba5c9e9..ced715612b 100644 --- a/protocols/raft/src/main/java/io/atomix/protocols/raft/roles/CandidateRole.java +++ b/protocols/raft/src/main/java/io/atomix/protocols/raft/roles/CandidateRole.java @@ -55,6 +55,13 @@ public RaftServer.Role role() { @Override public synchronized CompletableFuture start() { + if (raft.getCluster().getActiveMemberStates().isEmpty()) { + log.debug("Single member cluster. Transitioning directly to leader."); + raft.setTerm(raft.getTerm() + 1); + raft.setLastVotedFor(raft.getCluster().getMember().memberId()); + raft.transition(RaftServer.Role.LEADER); + return CompletableFuture.completedFuture(this); + } return super.start().thenRun(this::startElection).thenApply(v -> this); } @@ -91,13 +98,6 @@ private void sendVoteRequests() { final AtomicBoolean complete = new AtomicBoolean(); final Set votingMembers = new HashSet<>(raft.getCluster().getActiveMemberStates().stream().map(RaftMemberContext::getMember).collect(Collectors.toList())); - // If there are no other members in the cluster, immediately transition to leader. - if (votingMembers.isEmpty()) { - log.debug("Single member cluster. Transitioning directly to leader.", raft.getCluster().getMember().memberId()); - raft.transition(RaftServer.Role.LEADER); - return; - } - // Send vote requests to all nodes. The vote request that is sent // to this node will be automatically successful. // First check if the quorum is null. If the quorum isn't null then that diff --git a/protocols/raft/src/main/java/io/atomix/protocols/raft/roles/FollowerRole.java b/protocols/raft/src/main/java/io/atomix/protocols/raft/roles/FollowerRole.java index 018049c0bc..75b6dbd2fd 100644 --- a/protocols/raft/src/main/java/io/atomix/protocols/raft/roles/FollowerRole.java +++ b/protocols/raft/src/main/java/io/atomix/protocols/raft/roles/FollowerRole.java @@ -62,6 +62,11 @@ public RaftServer.Role role() { @Override public synchronized CompletableFuture start() { + if (raft.getCluster().getActiveMemberStates().isEmpty()) { + log.debug("Single member cluster. Transitioning directly to candidate."); + raft.transition(RaftServer.Role.CANDIDATE); + return CompletableFuture.completedFuture(this); + } raft.getMembershipService().addListener(clusterListener); return super.start().thenRun(this::resetHeartbeatTimeout).thenApply(v -> this); }