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=f8524b5174b9c3916e43f73aa9f35e6bbf91bd43;hp=8650d96f60b5b9b64f3bb69e4f8f649cdccab617;hb=758b6995d878eec225fdd959913b5495576649c2;hpb=9d5ec5cdd146a56bc03e35b6718e9492a5c8410a 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 8650d96f60..f8524b5174 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,16 +9,25 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; +import akka.actor.ActorSelection; +import akka.actor.Address; +import akka.cluster.Cluster; +import akka.cluster.ClusterEvent.CurrentClusterState; +import akka.cluster.Member; +import akka.cluster.MemberStatus; import akka.japi.Procedure; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; +import java.io.IOException; import java.util.ArrayList; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; 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.TimeoutNow; @@ -30,10 +39,10 @@ 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.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload; +import org.opendaylight.controller.cluster.raft.persisted.Snapshot; /** - * The behavior of a RaftActor in the Follower state - *

+ * The behavior of a RaftActor in the Follower raft state. *

*/ public class Follower extends AbstractRaftActorBehavior { - private static final int SYNC_THRESHOLD = 10; + private static final long MAX_ELECTION_TIMEOUT_FACTOR = 18; private final SyncStatusTracker initialSyncStatusTracker; - private final Procedure appendAndPersistCallback = new Procedure() { - @Override - public void apply(ReplicatedLogEntry logEntry) { - context.getReplicatedLog().captureSnapshotIfReady(logEntry); - } - }; - - private final Stopwatch lastLeaderMessageTimer = Stopwatch.createUnstarted(); + private final Stopwatch lastLeaderMessageTimer = Stopwatch.createStarted(); private SnapshotTracker snapshotTracker = null; private String leaderId; private short leaderPayloadVersion; - public Follower(RaftActorContext context) { + public Follower(final RaftActorContext context) { this(context, null, (short)-1); } - public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) { + public Follower(final RaftActorContext context, final String initialLeaderId, + final short initialLeaderPayloadVersion) { super(context, RaftState.Follower); this.leaderId = initialLeaderId; this.leaderPayloadVersion = initialLeaderPayloadVersion; - initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD); + initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), context.getConfigParams() + .getSyncIndexThreshold()); if (context.getPeerIds().isEmpty() && getLeaderId() == null) { actor().tell(TimeoutNow.INSTANCE, actor()); @@ -92,7 +96,7 @@ public class Follower extends AbstractRaftActorBehavior { } @VisibleForTesting - protected final void setLeaderPayloadVersion(short leaderPayloadVersion) { + protected final void setLeaderPayloadVersion(final short leaderPayloadVersion) { this.leaderPayloadVersion = leaderPayloadVersion; } @@ -104,8 +108,8 @@ public class Follower extends AbstractRaftActorBehavior { lastLeaderMessageTimer.start(); } - private boolean isLogEntryPresent(long index){ - if(context.getReplicatedLog().isInSnapshot(index)) { + private boolean isLogEntryPresent(final long index) { + if (context.getReplicatedLog().isInSnapshot(index)) { return true; } @@ -114,32 +118,36 @@ public class Follower extends AbstractRaftActorBehavior { } - private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){ - initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex()); + private void updateInitialSyncStatus(final long currentLeaderCommit, final String newLeaderId) { + initialSyncStatusTracker.update(newLeaderId, currentLeaderCommit, context.getCommitIndex()); } @Override - protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { + protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final 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); + 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 // to make it easier to read. Before refactoring ensure tests // cover the code properly + if (snapshotTracker != null && !snapshotTracker.getLeaderId().equals(appendEntries.getLeaderId())) { + log.debug("{}: snapshot install is in progress but the prior snapshot leaderId {} does not match the " + + "AppendEntries leaderId {}", logName(), snapshotTracker.getLeaderId(), appendEntries.getLeaderId()); + closeSnapshotTracker(); + } + if (snapshotTracker != null || context.getSnapshotManager().isApplying()) { // if snapshot install is in progress, follower should just acknowledge append entries with a reply. AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, lastIndex(), lastTerm(), context.getPayloadVersion()); - if(LOG.isDebugEnabled()) { - LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply); - } + log.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply); sender.tell(reply, actor()); return this; @@ -149,7 +157,6 @@ public class Follower extends AbstractRaftActorBehavior { leaderId = appendEntries.getLeaderId(); leaderPayloadVersion = appendEntries.getPayloadVersion(); - updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); // First check if the logs are in sync or not long lastIndex = lastIndex(); @@ -157,17 +164,18 @@ public class Follower extends AbstractRaftActorBehavior { // We found that the log was out of sync so just send a negative // reply and return - LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}", - logName(), lastIndex, lastTerm()); + final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, + lastTerm(), context.getPayloadVersion()); - sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, - lastTerm(), context.getPayloadVersion()), actor()); + log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply); + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + sender.tell(reply, actor()); return this; } if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) { - LOG.debug("{}: Number of entries to be appended = {}", logName(), + log.debug("{}: Number of entries to be appended = {}", logName(), appendEntries.getEntries().size()); // 3. If an existing entry conflicts with a new one (same index @@ -180,35 +188,35 @@ public class Follower extends AbstractRaftActorBehavior { for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) { ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i); - if(!isLogEntryPresent(matchEntry.getIndex())) { + if (!isLogEntryPresent(matchEntry.getIndex())) { // newEntry not found in the log break; } long existingEntryTerm = getLogEntryTerm(matchEntry.getIndex()); - LOG.debug("{}: matchEntry {} is present: existingEntryTerm: {}", logName(), matchEntry, + log.debug("{}: matchEntry {} is present: existingEntryTerm: {}", logName(), matchEntry, existingEntryTerm); // existingEntryTerm == -1 means it's in the snapshot and not in the log. We don't know // what the term was so we'll assume it matches. - if(existingEntryTerm == -1 || existingEntryTerm == matchEntry.getTerm()) { + if (existingEntryTerm == -1 || existingEntryTerm == matchEntry.getTerm()) { continue; } - if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) { + if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) { - LOG.debug("{}: Removing entries from log starting at {}", logName(), - matchEntry.getIndex()); + log.info("{}: Removing entries from log starting at {}", logName(), matchEntry.getIndex()); // Entries do not match so remove all subsequent entries - if(!context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex())) { + if (!context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex())) { // Could not remove the entries - this means the matchEntry index must be in the // snapshot and not the log. In this case the prior entries are part of the state // so we must send back a reply to force a snapshot to completely re-sync the // follower's log and state. - LOG.debug("{}: Could not remove entries - sending reply to force snapshot", logName()); + log.info("{}: Could not remove entries - sending reply to force snapshot", logName()); + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, lastTerm(), context.getPayloadVersion(), true), actor()); return this; @@ -216,6 +224,7 @@ public class Follower extends AbstractRaftActorBehavior { break; } else { + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, lastTerm(), context.getPayloadVersion(), true), actor()); return this; @@ -224,23 +233,42 @@ public class Follower extends AbstractRaftActorBehavior { } lastIndex = lastIndex(); - LOG.debug("{}: After cleanup, lastIndex: {}, entries to be added from: {}", logName(), + log.debug("{}: After cleanup, lastIndex: {}, entries to be added from: {}", logName(), lastIndex, addEntriesFrom); + // When persistence successfully completes for each new log entry appended, we need to determine if we + // should capture a snapshot to compact the persisted log. shouldCaptureSnapshot tracks whether or not + // one of the log entries has exceeded the log size threshold whereby a snapshot should be taken. However + // we don't initiate the snapshot at that log entry but rather after the last log entry has been persisted. + // This is done because subsequent log entries after the one that tripped the threshold may have been + // applied to the state already, as the persistence callback occurs async, and we want those entries + // purged from the persisted log as well. + final AtomicBoolean shouldCaptureSnapshot = new AtomicBoolean(false); + final Procedure appendAndPersistCallback = logEntry -> { + final ReplicatedLogEntry lastEntryToAppend = appendEntries.getEntries().get( + appendEntries.getEntries().size() - 1); + if (shouldCaptureSnapshot.get() && logEntry == lastEntryToAppend) { + context.getSnapshotManager().capture(context.getReplicatedLog().last(), getReplicatedToAllIndex()); + } + }; + // 4. Append any new entries not already in the log for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) { ReplicatedLogEntry entry = appendEntries.getEntries().get(i); - LOG.debug("{}: Append entry to log {}", logName(), entry.getData()); + log.debug("{}: Append entry to log {}", logName(), entry.getData()); + + context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback, false); - context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback); + shouldCaptureSnapshot.compareAndSet(false, + context.getReplicatedLog().shouldCaptureSnapshot(entry.getIndex())); - if(entry.getData() instanceof ServerConfigurationPayload) { + if (entry.getData() instanceof ServerConfigurationPayload) { context.updatePeerIds((ServerConfigurationPayload)entry.getData()); } } - LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); + log.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); } // 5. If leaderCommit > commitIndex, set commitIndex = @@ -249,38 +277,41 @@ public class Follower extends AbstractRaftActorBehavior { lastIndex = lastIndex(); long prevCommitIndex = context.getCommitIndex(); - context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); + if (appendEntries.getLeaderCommit() > prevCommitIndex) { + context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); + } if (prevCommitIndex != context.getCommitIndex()) { - LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); + log.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); + } + + AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, + lastIndex, lastTerm(), context.getPayloadVersion()); + + if (log.isTraceEnabled()) { + log.trace("{}: handleAppendEntries returning : {}", logName(), reply); + } else if (log.isDebugEnabled() && numLogEntries > 0) { + log.debug("{}: handleAppendEntries returning : {}", logName(), reply); } + // Reply to the leader before applying any previous state so as not to hold up leader consensus. + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + sender.tell(reply, actor()); + // 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) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: applyLogToStateMachine, " + - "appendEntries.getLeaderCommit(): {}," + - "context.getLastApplied(): {}, lastIndex(): {}", logName(), + if (appendEntries.getLeaderCommit() > context.getLastApplied() + && context.getLastApplied() < lastIndex) { + if (log.isDebugEnabled()) { + log.debug("{}: applyLogToStateMachine, appendEntries.getLeaderCommit(): {}," + + "context.getLastApplied(): {}, lastIndex(): {}", logName(), appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex); } applyLogToStateMachine(appendEntries.getLeaderCommit()); } - AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, - lastIndex, lastTerm(), context.getPayloadVersion()); - - 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.getSnapshotManager().isCapturing()) { super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex()); } @@ -288,7 +319,7 @@ public class Follower extends AbstractRaftActorBehavior { return this; } - private boolean isOutOfSync(AppendEntries appendEntries) { + private boolean isOutOfSync(final AppendEntries appendEntries) { long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex()); boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex()); @@ -302,39 +333,39 @@ public class Follower extends AbstractRaftActorBehavior { // an entry at prevLogIndex and this follower has no entries in // it's log. - LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}", + log.info("{}: 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 - LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it - lastIndex: {}, snapshotIndex: {}", - logName(), appendEntries.getPrevLogIndex(), lastIndex, context.getReplicatedLog().getSnapshotIndex()); + log.info("{}: The log is not empty but the prevLogIndex {} was not found in it - " + + "lastIndex: {}, snapshotIndex: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex, + context.getReplicatedLog().getSnapshotIndex()); } 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 - LOG.debug( - "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}", - logName(), prevLogTerm, appendEntries.getPrevLogTerm()); - } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 + log.info("{}: The prevLogIndex {} was found in the log but the term {} is not equal to the append entries" + + "prevLogTerm {} - lastIndex: {}, snapshotIndex: {}", logName(), appendEntries.getPrevLogIndex(), + prevLogTerm, appendEntries.getPrevLogTerm(), lastIndex, + context.getReplicatedLog().getSnapshotIndex()); + } else if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 && appendEntries.getReplicatedToAllIndex() != -1 && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) { // This append entry comes from a leader who has it's log aggressively trimmed and so does not have // the previous entry in it's in-memory journal - LOG.debug( - "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal", - logName(), appendEntries.getReplicatedToAllIndex()); - } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 + log.info("{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the" + + " in-memory journal", logName(), appendEntries.getReplicatedToAllIndex()); + } else if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0 && !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)) { - LOG.debug( - "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal", - logName(), appendEntries.getEntries().get(0).getIndex() - 1); + log.info("{}: Cannot append entries because the calculated previousIndex {} was not found in the " + + " in-memory journal", logName(), appendEntries.getEntries().get(0).getIndex() - 1); } else { outOfSync = false; } @@ -342,24 +373,23 @@ public class Follower extends AbstractRaftActorBehavior { } @Override - protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, - AppendEntriesReply appendEntriesReply) { + protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender, + final AppendEntriesReply appendEntriesReply) { return this; } @Override - protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply) { + protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, + final RequestVoteReply requestVoteReply) { return this; } @Override - public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { - if (originalMessage instanceof ElectionTimeout || originalMessage instanceof TimeoutNow) { - return handleElectionTimeout(originalMessage); + public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) { + if (message instanceof ElectionTimeout || message instanceof TimeoutNow) { + return handleElectionTimeout(message); } - final Object message = fromSerializableMessage(originalMessage); if (!(message instanceof RaftRPC)) { // The rest of the processing requires the message to be a RaftRPC return null; @@ -370,7 +400,7 @@ 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", + log.info("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term", logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); context.getTermInformation().updateAndPersist(rpc.getTerm(), null); @@ -391,27 +421,41 @@ public class Follower extends AbstractRaftActorBehavior { return super.handleMessage(sender, rpc); } - private RaftActorBehavior handleElectionTimeout(Object message) { + private RaftActorBehavior handleElectionTimeout(final Object message) { // If the message is ElectionTimeout, verify we haven't actually seen a message from the leader // during the election timeout interval. It may that the election timer expired b/c this actor // was busy and messages got delayed, in which case leader messages would be backed up in the // queue but would be processed before the ElectionTimeout message and thus would restart the // lastLeaderMessageTimer. long lastLeaderMessageInterval = lastLeaderMessageTimer.elapsed(TimeUnit.MILLISECONDS); - boolean noLeaderMessageReceived = !lastLeaderMessageTimer.isRunning() || lastLeaderMessageInterval >= - context.getConfigParams().getElectionTimeOutInterval().toMillis(); + long electionTimeoutInMillis = context.getConfigParams().getElectionTimeOutInterval().toMillis(); + boolean noLeaderMessageReceived = !lastLeaderMessageTimer.isRunning() + || lastLeaderMessageInterval >= electionTimeoutInMillis; - if(canStartElection()) { - if(message instanceof TimeoutNow || noLeaderMessageReceived) { - LOG.debug("{}: Received {} - switching to Candidate", logName(), message.getClass().getSimpleName()); + if (canStartElection()) { + if (message instanceof TimeoutNow) { + log.debug("{}: Received TimeoutNow - switching to Candidate", logName()); return internalSwitchBehavior(RaftState.Candidate); + } else if (noLeaderMessageReceived) { + // Check the cluster state to see if the leader is known to be up before we go to Candidate. + // However if we haven't heard from the leader in a long time even though the cluster state + // indicates it's up then something is wrong - leader might be stuck indefinitely - so switch + // to Candidate, + long maxElectionTimeout = electionTimeoutInMillis * MAX_ELECTION_TIMEOUT_FACTOR; + if (isLeaderAvailabilityKnown() && lastLeaderMessageInterval < maxElectionTimeout) { + log.debug("{}: Received ElectionTimeout but leader appears to be available", logName()); + scheduleElection(electionDuration()); + } else { + log.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); + return internalSwitchBehavior(RaftState.Candidate); + } } else { - LOG.debug("{}: Received ElectionTimeout but lastLeaderMessageInterval {} < election timeout", - logName(), lastLeaderMessageInterval); + log.debug("{}: Received ElectionTimeout but lastLeaderMessageInterval {} < election timeout {}", + logName(), lastLeaderMessageInterval, context.getConfigParams().getElectionTimeOutInterval()); scheduleElection(electionDuration()); } - } else if(message instanceof ElectionTimeout) { - if(noLeaderMessageReceived) { + } else if (message instanceof ElectionTimeout) { + if (noLeaderMessageReceived) { setLeaderId(null); } @@ -421,14 +465,64 @@ public class Follower extends AbstractRaftActorBehavior { return this; } - private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) { + private boolean isLeaderAvailabilityKnown() { + if (leaderId == null) { + return false; + } + + Optional cluster = context.getCluster(); + if (!cluster.isPresent()) { + return false; + } + + ActorSelection leaderActor = context.getPeerActorSelection(leaderId); + if (leaderActor == null) { + return false; + } + + Address leaderAddress = leaderActor.anchorPath().address(); + + CurrentClusterState state = cluster.get().state(); + Set unreachable = state.getUnreachable(); + + log.debug("{}: Checking for leader {} in the cluster unreachable set {}", logName(), leaderAddress, + unreachable); + + for (Member m: unreachable) { + if (leaderAddress.equals(m.address())) { + log.info("{}: Leader {} is unreachable", logName(), leaderAddress); + return false; + } + } + + for (Member m: state.getMembers()) { + if (leaderAddress.equals(m.address())) { + if (m.status() == MemberStatus.up() || m.status() == MemberStatus.weaklyUp()) { + log.debug("{}: Leader {} cluster status is {} - leader is available", logName(), + leaderAddress, m.status()); + return true; + } else { + log.debug("{}: Leader {} cluster status is {} - leader is unavailable", logName(), + leaderAddress, m.status()); + return false; + } + } + } + + log.debug("{}: Leader {} not found in the cluster member set", logName(), leaderAddress); + + return false; + } + + private void handleInstallSnapshot(final ActorRef sender, final InstallSnapshot installSnapshot) { - LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot); + log.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot); leaderId = installSnapshot.getLeaderId(); - if(snapshotTracker == null){ - snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks()); + if (snapshotTracker == null) { + snapshotTracker = new SnapshotTracker(log, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId(), + context); } updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId()); @@ -437,10 +531,14 @@ public class Follower extends AbstractRaftActorBehavior { final InstallSnapshotReply reply = new InstallSnapshotReply( currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true); - if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(), - installSnapshot.getLastChunkHashCode())){ - Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(), - new ArrayList(), + if (snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(), + installSnapshot.getLastChunkHashCode())) { + + log.info("{}: Snapshot installed from leader: {}", logName(), installSnapshot.getLeaderId()); + + Snapshot snapshot = Snapshot.create( + context.getSnapshotManager().convertSnapshot(snapshotTracker.getSnapshotBytes()), + new ArrayList<>(), installSnapshot.getLastIncludedIndex(), installSnapshot.getLastIncludedTerm(), installSnapshot.getLastIncludedIndex(), @@ -452,7 +550,7 @@ public class Follower extends AbstractRaftActorBehavior { ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() { @Override public void onSuccess() { - LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); + log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); sender.tell(reply, actor()); } @@ -465,36 +563,37 @@ public class Follower extends AbstractRaftActorBehavior { actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor()); - snapshotTracker = null; + closeSnapshotTracker(); } else { - LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); + log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); sender.tell(reply, actor()); } - } catch (SnapshotTracker.InvalidChunkException e) { - LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e); + } catch (IOException 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); - //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()); + closeSnapshotTracker(); + } + } + private void closeSnapshotTracker() { + if (snapshotTracker != null) { + snapshotTracker.close(); + snapshotTracker = null; } } @Override public void close() { + closeSnapshotTracker(); stopElection(); } @VisibleForTesting - SnapshotTracker getSnapshotTracker(){ + SnapshotTracker getSnapshotTracker() { return snapshotTracker; } }