From 630cb61cc1e51a486785162995eab7b541fcf237 Mon Sep 17 00:00:00 2001 From: Robert Varga Date: Sat, 2 Apr 2016 15:19:22 +0200 Subject: [PATCH] BUG-5626: handle ElectionTimeout message first ElectionTimeout is a singleton, it is not a RaftRPC, hence its reception leads to bail from handleMessage(). Move it to the front of the Candidate/Follower methods so we can simplify the checks performed in those methods. Change-Id: I3dfef4c42e2997d490de776de1c42e61cfacf217 Signed-off-by: Robert Varga --- .../cluster/raft/behaviors/Candidate.java | 35 +++++++++---------- .../cluster/raft/behaviors/Follower.java | 23 ++++++------ 2 files changed, 28 insertions(+), 30 deletions(-) diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java index a5def02b1f..4d51922bc2 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java @@ -123,9 +123,25 @@ public class Candidate extends AbstractRaftActorBehavior { @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { + if (originalMessage instanceof ElectionTimeout) { + LOG.debug("{}: Received ElectionTimeout", logName()); + + if (votesRequired == 0) { + // If there are no peers then we should be a Leader + // We wait for the election timeout to occur before declare + // ourselves the leader. This gives enough time for a leader + // who we do not know about (as a peer) + // to send a message to the candidate - Object message = fromSerializableMessage(originalMessage); + return internalSwitchBehavior(RaftState.Leader); + } + + startNewTerm(); + scheduleElection(electionDuration()); + return this; + } + final Object message = fromSerializableMessage(originalMessage); if (message instanceof RaftRPC) { RaftRPC rpc = (RaftRPC) message; @@ -145,23 +161,6 @@ public class Candidate extends AbstractRaftActorBehavior { } } - if (message instanceof ElectionTimeout) { - LOG.debug("{}: Received ElectionTimeout", logName()); - - if (votesRequired == 0) { - // If there are no peers then we should be a Leader - // We wait for the election timeout to occur before declare - // ourselves the leader. This gives enough time for a leader - // who we do not know about (as a peer) - // to send a message to the candidate - - return internalSwitchBehavior(RaftState.Leader); - } - startNewTerm(); - scheduleElection(electionDuration()); - return this; - } - return super.handleMessage(sender, message); } 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 2541dfab6e..2480a0db8f 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 @@ -347,9 +347,16 @@ public class Follower extends AbstractRaftActorBehavior { @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { + if (originalMessage instanceof ElectionTimeout) { + if (canStartElection()) { + LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); + return internalSwitchBehavior(RaftState.Candidate); + } else { + return this; + } + } - Object message = fromSerializableMessage(originalMessage); - + final Object message = fromSerializableMessage(originalMessage); if (message instanceof RaftRPC) { RaftRPC rpc = (RaftRPC) message; // If RPC request or response contains term T > currentTerm: @@ -363,20 +370,12 @@ public class Follower extends AbstractRaftActorBehavior { } } - if (message instanceof ElectionTimeout) { - if(canStartElection()) { - LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); - return internalSwitchBehavior(RaftState.Candidate); - } else { - return this; - } - - } else if (message instanceof InstallSnapshot) { + if (message instanceof InstallSnapshot) { InstallSnapshot installSnapshot = (InstallSnapshot) message; handleInstallSnapshot(sender, installSnapshot); } - if(message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){ + if (message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){ scheduleElection(electionDuration()); } -- 2.36.6