X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FFollower.java;h=288ce32a64ec21286adda9cf3672f7b53d1bf506;hp=b8d229d84ca3e607cc3cae1d1f7893a45124c543;hb=d3d04140ac3d7a2a90a3b953cc4ea27fca2bfc32;hpb=570a59fc778c77bf8344af5fda9b28b91bec59d9 diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java index b8d229d84c..288ce32a64 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java @@ -18,6 +18,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import java.io.IOException; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Set; @@ -30,6 +31,7 @@ import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; @@ -435,6 +437,11 @@ public class Follower extends AbstractRaftActorBehavior { return this; } + @Override + final ApplyState getApplyStateFor(final ReplicatedLogEntry entry) { + return new ApplyState(null, null, entry); + } + @Override public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) { if (message instanceof ElectionTimeout || message instanceof TimeoutNow) { @@ -454,7 +461,7 @@ public class Follower extends AbstractRaftActorBehavior { // If RPC request or response contains term T > currentTerm: // set currentTerm = T, convert to follower (§5.1) // This applies to all RPC messages and responses - if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) { + if (rpc.getTerm() > context.getTermInformation().getCurrentTerm() && shouldUpdateTerm(rpc)) { log.info("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term", logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); @@ -500,6 +507,10 @@ public class Follower extends AbstractRaftActorBehavior { if (isLeaderAvailabilityKnown() && lastLeaderMessageInterval < maxElectionTimeout) { log.debug("{}: Received ElectionTimeout but leader appears to be available", logName()); scheduleElection(electionDuration()); + } else if (isThisFollowerIsolated()) { + log.debug("{}: this follower is isolated. Do not switch to Candidate for now.", logName()); + setLeaderId(null); + scheduleElection(electionDuration()); } else { log.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); return internalSwitchBehavior(RaftState.Candidate); @@ -569,6 +580,40 @@ public class Follower extends AbstractRaftActorBehavior { return false; } + private boolean isThisFollowerIsolated() { + final Optional maybeCluster = context.getCluster(); + if (!maybeCluster.isPresent()) { + return false; + } + + final Cluster cluster = maybeCluster.get(); + final Member selfMember = cluster.selfMember(); + + final CurrentClusterState state = cluster.state(); + final Set unreachable = state.getUnreachable(); + final Iterable members = state.getMembers(); + + log.debug("{}: Checking if this node is isolated in the cluster unreachable set {}," + + "all members {} self member: {}", logName(), unreachable, members, selfMember); + + // no unreachable peers means we cannot be isolated + if (unreachable.size() == 0) { + return false; + } + + final Set membersToCheck = new HashSet<>(); + members.forEach(membersToCheck::add); + + membersToCheck.removeAll(unreachable); + + // check if the only member not unreachable is us + if (membersToCheck.size() == 1 && membersToCheck.iterator().next().equals(selfMember)) { + return true; + } + + return false; + } + private void handleInstallSnapshot(final ActorRef sender, final InstallSnapshot installSnapshot) { log.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);