X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FAbstractRaftActorBehavior.java;h=b1560a5648b283e028ae0dc96e4267d92c6f438f;hb=2a31c2cacb9ad8f015a49708261ea93d256f0f60;hp=167082711d82b46c61e90e8311473d051e06461c;hpb=a0c5aba42aa36337ff1c6760175918b786897c9e;p=controller.git diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java index 167082711d..45671ea31e 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java @@ -10,20 +10,23 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; import akka.actor.Cancellable; +import java.util.Random; +import java.util.concurrent.TimeUnit; +import org.opendaylight.controller.cluster.raft.ClientRequestTracker; 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.internal.messages.ElectionTimeout; +import org.opendaylight.controller.cluster.raft.SerializationUtils; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; +import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; -import org.opendaylight.controller.cluster.raft.messages.RaftRPC; import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; +import org.slf4j.Logger; import scala.concurrent.duration.FiniteDuration; -import java.util.Random; -import java.util.concurrent.TimeUnit; - /** * Abstract class that represents the behavior of a RaftActor *

@@ -36,41 +39,59 @@ import java.util.concurrent.TimeUnit; */ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { + protected static final ElectionTimeout ELECTION_TIMEOUT = new ElectionTimeout(); + /** * Information about the RaftActor whose behavior this class represents */ protected final RaftActorContext context; /** - * The maximum election time variance - */ - private static final int ELECTION_TIME_MAX_VARIANCE = 100; - - /** - * The interval at which a heart beat message will be sent to the remote - * RaftActor - *

- * Since this is set to 100 milliseconds the Election timeout should be - * at least 200 milliseconds + * */ - protected static final FiniteDuration HEART_BEAT_INTERVAL = - new FiniteDuration(100, TimeUnit.MILLISECONDS); + protected final Logger LOG; /** - * The interval in which a new election would get triggered if no leader is found + * */ - private static final long ELECTION_TIME_INTERVAL = - HEART_BEAT_INTERVAL.toMillis() * 2; + private Cancellable electionCancel = null; /** * */ + protected String leaderId = null; - private Cancellable electionCancel = null; + private long replicatedToAllIndex = -1; + private final String logName; - protected AbstractRaftActorBehavior(RaftActorContext context) { + private final RaftState state; + + protected AbstractRaftActorBehavior(RaftActorContext context, RaftState state) { this.context = context; + this.state = state; + this.LOG = context.getLogger(); + + logName = String.format("%s (%s)", context.getId(), state); + } + + @Override + public RaftState state() { + return state; + } + + public String logName() { + return logName; + } + + @Override + public void setReplicatedToAllIndex(long replicatedToAllIndex) { + this.replicatedToAllIndex = replicatedToAllIndex; + } + + @Override + public long getReplicatedToAllIndex() { + return replicatedToAllIndex; } /** @@ -83,77 +104,39 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { * * @param sender The actor that sent this message * @param appendEntries The AppendEntries message - * @param suggestedState The state that the RaftActor should be in based - * on the base class's processing of the AppendEntries - * message * @return */ - protected abstract RaftState handleAppendEntries(ActorRef sender, - AppendEntries appendEntries, RaftState suggestedState); + protected abstract RaftActorBehavior handleAppendEntries(ActorRef sender, + AppendEntries appendEntries); - protected RaftState appendEntries(ActorRef sender, - AppendEntries appendEntries, RaftState raftState){ + /** + * appendEntries first processes the AppendEntries message and then + * delegates handling to a specific behavior + * + * @param sender + * @param appendEntries + * @return + */ + protected RaftActorBehavior appendEntries(ActorRef sender, + AppendEntries appendEntries) { // 1. Reply false if term < currentTerm (§5.1) - if(appendEntries.getTerm() < currentTerm()){ - sender.tell(new AppendEntriesReply(currentTerm(), false), actor()); - return state(); - } - - // 2. Reply false if log doesn’t contain an entry at prevLogIndex - // whose term matches prevLogTerm (§5.3) - ReplicatedLogEntry previousEntry = context.getReplicatedLog() - .get(appendEntries.getPrevLogIndex()); - - if(previousEntry == null || previousEntry.getTerm() != appendEntries.getPrevLogTerm()){ - sender.tell(new AppendEntriesReply(currentTerm(), false), actor()); - return state(); - } - - if(appendEntries.getEntries() != null) { - // 3. If an existing entry conflicts with a new one (same index - // but different terms), delete the existing entry and all that - // follow it (§5.3) - int addEntriesFrom = 0; - for (int i = 0; - i < appendEntries.getEntries().size(); i++, addEntriesFrom++) { - ReplicatedLogEntry newEntry = context.getReplicatedLog() - .get(i + 1); - - if (newEntry != null && newEntry.getTerm() == appendEntries.getEntries().get(i).getTerm()){ - break; - } - if (newEntry != null && newEntry.getTerm() != appendEntries - .getEntries().get(i).getTerm()) { - context.getReplicatedLog().removeFrom(i + 1); - break; - } - } - - // 4. Append any new entries not already in the log - for (int i = addEntriesFrom; - i < appendEntries.getEntries().size(); i++) { - context.getReplicatedLog() - .append(appendEntries.getEntries().get(i)); + if (appendEntries.getTerm() < currentTerm()) { + if(LOG.isDebugEnabled()) { + LOG.debug("{}: Cannot append entries because sender term {} is less than {}", + logName(), appendEntries.getTerm(), currentTerm()); } - } - - // 5. If leaderCommit > commitIndex, set commitIndex = - // min(leaderCommit, index of last new entry) - context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), - context.getReplicatedLog().last().getIndex())); - - // If commitIndex > lastApplied: increment lastApplied, apply - // log[lastApplied] to state machine (§5.3) - if (appendEntries.getLeaderCommit() > context.getLastApplied()) { - applyLogToStateMachine(appendEntries.getLeaderCommit()); + sender.tell( + new AppendEntriesReply(context.getId(), currentTerm(), false, + lastIndex(), lastTerm()), actor() + ); + return this; } - sender.tell(new AppendEntriesReply(currentTerm(), true), actor()); - return handleAppendEntries(sender, appendEntries, raftState); + return handleAppendEntries(sender, appendEntries); } /** @@ -166,17 +149,22 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { * * @param sender The actor that sent this message * @param appendEntriesReply The AppendEntriesReply message - * @param suggestedState The state that the RaftActor should be in based - * on the base class's processing of the - * AppendEntriesReply message * @return */ + protected abstract RaftActorBehavior handleAppendEntriesReply(ActorRef sender, + AppendEntriesReply appendEntriesReply); - protected abstract RaftState handleAppendEntriesReply(ActorRef sender, - AppendEntriesReply appendEntriesReply, RaftState suggestedState); + /** + * requestVote handles the RequestVote message. This logic is common + * for all behaviors + * + * @param sender + * @param requestVote + * @return + */ + protected RaftActorBehavior requestVote(ActorRef sender, RequestVote requestVote) { - protected RaftState requestVote(ActorRef sender, - RequestVote requestVote, RaftState suggestedState) { + LOG.debug("{}: In requestVote: {}", logName(), requestVote); boolean grantVote = false; @@ -201,20 +189,24 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { if (requestVote.getLastLogTerm() > lastTerm()) { candidateLatest = true; } else if ((requestVote.getLastLogTerm() == lastTerm()) - && requestVote.getLastLogIndex() >= lastTerm()) { + && requestVote.getLastLogIndex() >= lastIndex()) { candidateLatest = true; } if (candidateLatest) { grantVote = true; - context.getTermInformation().update(requestVote.getTerm(), + context.getTermInformation().updateAndPersist(requestVote.getTerm(), requestVote.getCandidateId()); } } - sender.tell(new RequestVoteReply(currentTerm(), grantVote), actor()); + RequestVoteReply reply = new RequestVoteReply(currentTerm(), grantVote); + + LOG.debug("{}: requestVote returning: {}", logName(), reply); - return suggestedState; + sender.tell(reply, actor()); + + return this; } /** @@ -227,102 +219,279 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { * * @param sender The actor that sent this message * @param requestVoteReply The RequestVoteReply message - * @param suggestedState The state that the RaftActor should be in based - * on the base class's processing of the RequestVote - * message * @return */ - - protected abstract RaftState handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply, RaftState suggestedState); + protected abstract RaftActorBehavior handleRequestVoteReply(ActorRef sender, + RequestVoteReply requestVoteReply); /** - * @return The derived class should return the state that corresponds to - * it's behavior + * Creates a random election duration + * + * @return */ - protected abstract RaftState state(); - protected FiniteDuration electionDuration() { - long variance = new Random().nextInt(ELECTION_TIME_MAX_VARIANCE); - return new FiniteDuration(ELECTION_TIME_INTERVAL + variance, - TimeUnit.MILLISECONDS); + long variance = new Random().nextInt(context.getConfigParams().getElectionTimeVariance()); + return context.getConfigParams().getElectionTimeOutInterval().$plus( + new FiniteDuration(variance, TimeUnit.MILLISECONDS)); } - protected void scheduleElection(FiniteDuration interval) { - + /** + * stop the scheduled election + */ + protected void stopElection() { if (electionCancel != null && !electionCancel.isCancelled()) { electionCancel.cancel(); } + } + + /** + * schedule a new election + * + * @param interval + */ + protected void scheduleElection(FiniteDuration interval) { + stopElection(); // Schedule an election. When the scheduler triggers an ElectionTimeout // message is sent to itself electionCancel = context.getActorSystem().scheduler().scheduleOnce(interval, - context.getActor(), new ElectionTimeout(), + context.getActor(), ELECTION_TIMEOUT, context.getActorSystem().dispatcher(), context.getActor()); } + /** + * Get the current term + * @return + */ protected long currentTerm() { return context.getTermInformation().getCurrentTerm(); } + /** + * Get the candidate for whom we voted in the current term + * @return + */ protected String votedFor() { return context.getTermInformation().getVotedFor(); } + /** + * Get the actor associated with this behavior + * @return + */ protected ActorRef actor() { return context.getActor(); } + /** + * Get the term from the last entry in the log + * + * @return + */ protected long lastTerm() { - return context.getReplicatedLog().last().getTerm(); + return context.getReplicatedLog().lastTerm(); } + /** + * Get the index from the last entry in the log + * + * @return + */ protected long lastIndex() { - return context.getReplicatedLog().last().getIndex(); + return context.getReplicatedLog().lastIndex(); } + /** + * Find the client request tracker for a specific logIndex + * + * @param logIndex + * @return + */ + protected ClientRequestTracker findClientRequestTracker(long logIndex) { + return null; + } - @Override - public RaftState handleMessage(ActorRef sender, Object message) { - RaftState raftState = state(); - if (message instanceof RaftRPC) { - raftState = applyTerm((RaftRPC) message); + /** + * Find the client request tracker for a specific logIndex + * + * @param logIndex + * @return + */ + protected ClientRequestTracker removeClientRequestTracker(long logIndex) { + return null; + } + + + /** + * Find the log index from the previous to last entry in the log + * + * @return + */ + protected long prevLogIndex(long index){ + ReplicatedLogEntry prevEntry = + context.getReplicatedLog().get(index - 1); + if (prevEntry != null) { + return prevEntry.getIndex(); } + return -1; + } + + /** + * Find the log term from the previous to last entry in the log + * @return + */ + protected long prevLogTerm(long index){ + ReplicatedLogEntry prevEntry = + context.getReplicatedLog().get(index - 1); + if (prevEntry != null) { + return prevEntry.getTerm(); + } + return -1; + } + + /** + * Apply the provided index to the state machine + * + * @param index a log index that is known to be committed + */ + protected void applyLogToStateMachine(final long index) { + long newLastApplied = context.getLastApplied(); + // Now maybe we apply to the state machine + for (long i = context.getLastApplied() + 1; + i < index + 1; i++) { + ActorRef clientActor = null; + String identifier = null; + ClientRequestTracker tracker = removeClientRequestTracker(i); + + if (tracker != null) { + clientActor = tracker.getClientActor(); + identifier = tracker.getIdentifier(); + } + ReplicatedLogEntry replicatedLogEntry = + context.getReplicatedLog().get(i); + + if (replicatedLogEntry != null) { + // Send a local message to the local RaftActor (it's derived class to be + // specific to apply the log to it's index) + actor().tell(new ApplyState(clientActor, identifier, + replicatedLogEntry), actor()); + newLastApplied = i; + } else { + //if one index is not present in the log, no point in looping + // around as the rest wont be present either + LOG.warn( + "{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}", + logName(), i, i, index); + break; + } + } + if(LOG.isDebugEnabled()) { + LOG.debug("{}: Setting last applied to {}", logName(), newLastApplied); + } + context.setLastApplied(newLastApplied); + + // send a message to persist a ApplyLogEntries marker message into akka's persistent journal + // will be used during recovery + //in case if the above code throws an error and this message is not sent, it would be fine + // as the append entries received later would initiate add this message to the journal + actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor()); + } + + protected Object fromSerializableMessage(Object serializable){ + return SerializationUtils.fromSerializable(serializable); + } + + @Override + public RaftActorBehavior handleMessage(ActorRef sender, Object message) { if (message instanceof AppendEntries) { - raftState = appendEntries(sender, (AppendEntries) message, - raftState); + return appendEntries(sender, (AppendEntries) message); } else if (message instanceof AppendEntriesReply) { - raftState = - handleAppendEntriesReply(sender, (AppendEntriesReply) message, - raftState); + return handleAppendEntriesReply(sender, (AppendEntriesReply) message); } else if (message instanceof RequestVote) { - raftState = - requestVote(sender, (RequestVote) message, raftState); + return requestVote(sender, (RequestVote) message); } else if (message instanceof RequestVoteReply) { - raftState = - handleRequestVoteReply(sender, (RequestVoteReply) message, - raftState); + return handleRequestVoteReply(sender, (RequestVoteReply) message); + } + return this; + } + + @Override public String getLeaderId() { + return leaderId; + } + + protected RaftActorBehavior switchBehavior(RaftActorBehavior behavior) { + LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), behavior.state()); + try { + close(); + } catch (Exception e) { + LOG.error("{}: Failed to close behavior : {}", logName(), this.state(), e); } - return raftState; + + return behavior; } - private RaftState applyTerm(RaftRPC rpc) { - // 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()) { - context.getTermInformation().update(rpc.getTerm(), null); - return RaftState.Follower; + protected int getMajorityVoteCount(int numPeers) { + // Votes are required from a majority of the peers including self. + // The numMajority field therefore stores a calculated value + // of the number of votes required for this candidate to win an + // election based on it's known peers. + // If a peer was added during normal operation and raft replicas + // came to know about them then the new peer would also need to be + // taken into consideration when calculating this value. + // Here are some examples for what the numMajority would be for n + // peers + // 0 peers = 1 numMajority -: (0 + 1) / 2 + 1 = 1 + // 2 peers = 2 numMajority -: (2 + 1) / 2 + 1 = 2 + // 4 peers = 3 numMajority -: (4 + 1) / 2 + 1 = 3 + + int numMajority = 0; + if (numPeers > 0) { + int self = 1; + numMajority = (numPeers + self) / 2 + 1; } - return state(); + return numMajority; + } - private void applyLogToStateMachine(long index) { - // Send a local message to the local RaftActor (it's derived class to be - // specific to apply the log to it's index) - context.setLastApplied(index); + + /** + * Performs a snapshot with no capture on the replicated log. + * It clears the log from the supplied index or last-applied-1 which ever is minimum. + * + * @param snapshotCapturedIndex + */ + protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) { + // we would want to keep the lastApplied as its used while capturing snapshots + long lastApplied = context.getLastApplied(); + long tempMin = Math.min(snapshotCapturedIndex, (lastApplied > -1 ? lastApplied - 1 : -1)); + + if(LOG.isTraceEnabled()) { + LOG.trace("{}: performSnapshotWithoutCapture: snapshotCapturedIndex: {}, lastApplied: {}, tempMin: {}", + logName, snapshotCapturedIndex, lastApplied, tempMin); + } + + if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin)) { + LOG.debug("{}: fakeSnapshot purging log to {} for term {}", logName(), tempMin, + context.getTermInformation().getCurrentTerm()); + + //use the term of the temp-min, since we check for isPresent, entry will not be null + ReplicatedLogEntry entry = context.getReplicatedLog().get(tempMin); + context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm()); + context.getReplicatedLog().snapshotCommit(); + setReplicatedToAllIndex(tempMin); + } else if(tempMin > getReplicatedToAllIndex()) { + // It's possible a follower was lagging and an install snapshot advanced its match index past + // the current replicatedToAllIndex. Since the follower is now caught up we should advance the + // replicatedToAllIndex (to tempMin). The fact that tempMin wasn't found in the log is likely + // due to a previous snapshot triggered by the memory threshold exceeded, in that case we + // trim the log to the last applied index even if previous entries weren't replicated to all followers. + setReplicatedToAllIndex(tempMin); + } } + protected String getId(){ + return context.getId(); + } }