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=618865cb88eb8877cdcfdcfb29208c80707c2c0f;hp=1cfdf9dba8b912a5bc23f78b85c447621298d908;hb=6eb0469b729fbe69dad58ce6223fc231669089c7;hpb=b5b204bafd8ee18692fc023cb2eae6e123369340 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 1cfdf9dba8..618865cb88 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 @@ -9,14 +9,15 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; -import akka.event.LoggingAdapter; -import com.google.protobuf.ByteString; +import com.google.common.annotations.VisibleForTesting; +import java.util.ArrayList; 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.Snapshot; import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; +import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot; @@ -24,8 +25,6 @@ import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply; import org.opendaylight.controller.cluster.raft.messages.RaftRPC; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; -import java.util.ArrayList; - /** * The behavior of a RaftActor in the Follower state *

@@ -37,25 +36,60 @@ import java.util.ArrayList; * */ public class Follower extends AbstractRaftActorBehavior { - private ByteString snapshotChunksCollected = ByteString.EMPTY; - private final LoggingAdapter LOG; - public Follower(RaftActorContext context) { - super(context); - LOG = context.getLogger(); + private SnapshotTracker snapshotTracker = null; + + private final InitialSyncStatusTracker initialSyncStatusTracker; + + public Follower(RaftActorContext context) { + super(context, RaftState.Follower); scheduleElection(electionDuration()); + + initialSyncStatusTracker = new InitialSyncStatusTracker(context.getActor()); } - @Override protected RaftState handleAppendEntries(ActorRef sender, - AppendEntries appendEntries) { + private boolean isLogEntryPresent(long index){ + if(index == context.getReplicatedLog().getSnapshotIndex()){ + return true; + } - if(appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) { - if(LOG.isDebugEnabled()) { - LOG.debug(appendEntries.toString()); - } + ReplicatedLogEntry previousEntry = context.getReplicatedLog() + .get(index); + + return previousEntry != null; + + } + + private long getLogEntryTerm(long index){ + if(index == context.getReplicatedLog().getSnapshotIndex()){ + return context.getReplicatedLog().getSnapshotTerm(); + } + + ReplicatedLogEntry previousEntry = context.getReplicatedLog() + .get(index); + + if(previousEntry != null){ + return previousEntry.getTerm(); + } + + return -1; + } + + private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){ + initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex()); + } + + @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender, + AppendEntries appendEntries) { + + int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0; + if(LOG.isTraceEnabled()) { + LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries); + } else if(LOG.isDebugEnabled() && numLogEntries > 0) { + LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries); } // TODO : Refactor this method into a bunch of smaller methods @@ -71,51 +105,39 @@ public class Follower extends AbstractRaftActorBehavior { // 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()); + long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex()); + boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex()); + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); boolean outOfSync = true; // First check if the logs are in sync or not - if (lastIndex() == -1 - && appendEntries.getPrevLogIndex() != -1) { + long lastIndex = lastIndex(); + if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) { // The follower's log is out of sync because the leader does have // an entry at prevLogIndex and this follower has no entries in // it's log. - if(LOG.isDebugEnabled()) { - LOG.debug("The followers log is empty and the senders prevLogIndex is {}", - appendEntries.getPrevLogIndex()); - } - - } else if (lastIndex() > -1 - && appendEntries.getPrevLogIndex() != -1 - && previousEntry == null) { + LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}", + logName(), appendEntries.getPrevLogIndex()); + } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) { // The follower's log is out of sync because the Leader's // prevLogIndex entry was not found in it's log - if(LOG.isDebugEnabled()) { - LOG.debug("The log is not empty but the prevLogIndex {} was not found in it", - appendEntries.getPrevLogIndex()); - } - - } else if (lastIndex() > -1 - && previousEntry != null - && previousEntry.getTerm()!= appendEntries.getPrevLogTerm()) { + LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it", + logName(), appendEntries.getPrevLogIndex()); + } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) { // The follower's log is out of sync because the Leader's // prevLogIndex entry does exist in the follower's log but it has // a different term in it - if(LOG.isDebugEnabled()) { - LOG.debug( - "Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}" - , previousEntry.getTerm() - , appendEntries.getPrevLogTerm()); - } + LOG.debug( + "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}", + logName(), prevLogTerm, appendEntries.getPrevLogTerm()); } else { outOfSync = false; } @@ -123,27 +145,19 @@ public class Follower extends AbstractRaftActorBehavior { if (outOfSync) { // We found that the log was out of sync so just send a negative // reply and return - if(LOG.isDebugEnabled()) { - LOG.debug("Follower is out-of-sync, " + - "so sending negative reply, lastIndex():{}, lastTerm():{}", - lastIndex(), lastTerm() - ); - } - sender.tell( - new AppendEntriesReply(context.getId(), currentTerm(), false, - lastIndex(), lastTerm()), actor() - ); - return state(); + + LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}", + logName(), lastIndex, lastTerm()); + + sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, + lastTerm()), actor()); + return this; } - if (appendEntries.getEntries() != null - && appendEntries.getEntries().size() > 0) { - if(LOG.isDebugEnabled()) { - LOG.debug( - "Number of entries to be appended = " + appendEntries - .getEntries().size() - ); - } + if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) { + + LOG.debug("{}: Number of entries to be appended = {}", logName(), + appendEntries.getEntries().size()); // 3. If an existing entry conflicts with a new one (same index // but different terms), delete the existing entry and all that @@ -161,102 +175,91 @@ public class Follower extends AbstractRaftActorBehavior { break; } - if (newEntry.getTerm() == matchEntry - .getTerm()) { + if (newEntry.getTerm() == matchEntry.getTerm()) { continue; } - if(LOG.isDebugEnabled()) { - LOG.debug( - "Removing entries from log starting at " - + matchEntry.getIndex() - ); - } + LOG.debug("{}: Removing entries from log starting at {}", logName(), + matchEntry.getIndex()); // Entries do not match so remove all subsequent entries - context.getReplicatedLog() - .removeFromAndPersist(matchEntry.getIndex()); + context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex()); break; } } - if(LOG.isDebugEnabled()) { - context.getLogger().debug( - "After cleanup entries to be added from = " + (addEntriesFrom - + lastIndex()) - ); - } + lastIndex = lastIndex(); + LOG.debug("{}: After cleanup entries to be added from = {}", logName(), + (addEntriesFrom + lastIndex)); // 4. Append any new entries not already in the log - for (int i = addEntriesFrom; - i < appendEntries.getEntries().size(); i++) { - - context.getLogger().info( - "Append entry to log " + appendEntries.getEntries().get( - i).getData() - .toString() - ); - context.getReplicatedLog() - .appendAndPersist(appendEntries.getEntries().get(i)); - } + for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) { + ReplicatedLogEntry entry = appendEntries.getEntries().get(i); - if(LOG.isDebugEnabled()) { - LOG.debug("Log size is now " + context.getReplicatedLog().size()); + LOG.debug("{}: Append entry to log {}", logName(), entry.getData()); + + context.getReplicatedLog().appendAndPersist(entry); } - } + LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); + } // 5. If leaderCommit > commitIndex, set commitIndex = // min(leaderCommit, index of last new entry) + lastIndex = lastIndex(); long prevCommitIndex = context.getCommitIndex(); - context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), - context.getReplicatedLog().lastIndex())); + context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); if (prevCommitIndex != context.getCommitIndex()) { - if(LOG.isDebugEnabled()) { - LOG.debug("Commit index set to " + context.getCommitIndex()); - } + LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); } // If commitIndex > lastApplied: increment lastApplied, apply // log[lastApplied] to state machine (§5.3) // check if there are any entries to be applied. last-applied can be equal to last-index if (appendEntries.getLeaderCommit() > context.getLastApplied() && - context.getLastApplied() < lastIndex()) { + context.getLastApplied() < lastIndex) { if(LOG.isDebugEnabled()) { - LOG.debug("applyLogToStateMachine, " + - "appendEntries.getLeaderCommit():{}," + - "context.getLastApplied():{}, lastIndex():{}", - appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex() - ); + LOG.debug("{}: applyLogToStateMachine, " + + "appendEntries.getLeaderCommit(): {}," + + "context.getLastApplied(): {}, lastIndex(): {}", logName(), + appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex); } applyLogToStateMachine(appendEntries.getLeaderCommit()); } - sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), true, - lastIndex(), lastTerm()), actor()); + AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, + lastIndex, lastTerm()); + + if(LOG.isTraceEnabled()) { + LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply); + } else if(LOG.isDebugEnabled() && numLogEntries > 0) { + LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply); + } + + sender.tell(reply, actor()); + + if (!context.isSnapshotCaptureInitiated()) { + super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex()); + } - return state(); + return this; } - @Override protected RaftState handleAppendEntriesReply(ActorRef sender, + @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { - return state(); + return this; } - @Override protected RaftState handleRequestVoteReply(ActorRef sender, + @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, RequestVoteReply requestVoteReply) { - return state(); + return this; } - @Override public RaftState state() { - return RaftState.Follower; - } - - @Override public RaftState handleMessage(ActorRef sender, Object originalMessage) { + @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { Object message = fromSerializableMessage(originalMessage); @@ -266,12 +269,16 @@ public class Follower extends AbstractRaftActorBehavior { // set currentTerm = T, convert to follower (§5.1) // This applies to all RPC messages and responses if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) { + LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term", + logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); } } if (message instanceof ElectionTimeout) { - return RaftState.Candidate; + LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); + return switchBehavior(new Candidate(context)); } else if (message instanceof InstallSnapshot) { InstallSnapshot installSnapshot = (InstallSnapshot) message; @@ -285,53 +292,95 @@ public class Follower extends AbstractRaftActorBehavior { private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) { - if(LOG.isDebugEnabled()) { - LOG.debug("InstallSnapshot received by follower " + - "datasize:{} , Chunk:{}/{}", installSnapshot.getData().size(), - installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks() - ); - } + LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}", + logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(), + installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks()); - try { - if (installSnapshot.getChunkIndex() == installSnapshot.getTotalChunks()) { - // this is the last chunk, create a snapshot object and apply + if(snapshotTracker == null){ + snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks()); + } - snapshotChunksCollected = snapshotChunksCollected.concat(installSnapshot.getData()); - context.getLogger().debug("Last chunk received: snapshotChunksCollected.size:{}", - snapshotChunksCollected.size()); + updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId()); - Snapshot snapshot = Snapshot.create(snapshotChunksCollected.toByteArray(), - new ArrayList(), - installSnapshot.getLastIncludedIndex(), - installSnapshot.getLastIncludedTerm(), - installSnapshot.getLastIncludedIndex(), - installSnapshot.getLastIncludedTerm()); + try { + if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(), + installSnapshot.getLastChunkHashCode())){ + Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(), + new ArrayList(), + installSnapshot.getLastIncludedIndex(), + installSnapshot.getLastIncludedTerm(), + installSnapshot.getLastIncludedIndex(), + installSnapshot.getLastIncludedTerm()); actor().tell(new ApplySnapshot(snapshot), actor()); - } else { - // we have more to go - snapshotChunksCollected = snapshotChunksCollected.concat(installSnapshot.getData()); + snapshotTracker = null; - if(LOG.isDebugEnabled()) { - LOG.debug("Chunk={},snapshotChunksCollected.size:{}", - installSnapshot.getChunkIndex(), snapshotChunksCollected.size()); - } } - sender.tell(new InstallSnapshotReply( - currentTerm(), context.getId(), installSnapshot.getChunkIndex(), - true), actor()); + InstallSnapshotReply reply = new InstallSnapshotReply( + currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true); + + LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); + + sender.tell(reply, actor()); + + } catch (SnapshotTracker.InvalidChunkException e) { + LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e); + + sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), + -1, false), actor()); + snapshotTracker = null; + + } catch (Exception e){ + LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e); - } catch (Exception e) { - context.getLogger().error("Exception in InstallSnapshot of follower", e); //send reply with success as false. The chunk will be sent again on failure sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), - installSnapshot.getChunkIndex(), false), actor()); + installSnapshot.getChunkIndex(), false), actor()); + } } - @Override public void close() throws Exception { + @Override + public void close() throws Exception { stopElection(); } + + @VisibleForTesting + SnapshotTracker getSnapshotTracker(){ + return snapshotTracker; + } + + private static class InitialSyncStatusTracker { + + private static final long INVALID_LOG_INDEX = -2L; + private long initialLeaderCommit = INVALID_LOG_INDEX; + private boolean initialSyncUpDone = false; + private String syncedLeaderId = null; + private final ActorRef actor; + + public InitialSyncStatusTracker(ActorRef actor) { + this.actor = actor; + } + + public void update(String leaderId, long leaderCommit, long commitIndex){ + + if(!leaderId.equals(syncedLeaderId)){ + initialSyncUpDone = false; + initialLeaderCommit = INVALID_LOG_INDEX; + syncedLeaderId = leaderId; + } + + if(!initialSyncUpDone){ + if(initialLeaderCommit == INVALID_LOG_INDEX){ + actor.tell(new FollowerInitialSyncUpStatus(false), ActorRef.noSender()); + initialLeaderCommit = leaderCommit; + } else if(commitIndex >= initialLeaderCommit){ + actor.tell(new FollowerInitialSyncUpStatus(true), ActorRef.noSender()); + initialSyncUpDone = true; + } + } + } + } }