@Override
protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) {
-
int numLogEntries = appendEntries.getEntries().size();
if (log.isTraceEnabled()) {
log.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
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());
leaderPayloadVersion = appendEntries.getPayloadVersion();
// First check if the logs are in sync or not
- long lastIndex = lastIndex();
-
- if (isOutOfSync(appendEntries)) {
- // We found that the log was out of sync so just send a negative
- // reply and return
-
- final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
- lastTerm(), context.getPayloadVersion());
-
- log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply);
+ if (isOutOfSync(appendEntries, sender)) {
updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
- sender.tell(reply, actor());
return this;
}
- if (numLogEntries > 0) {
- log.debug("{}: Number of entries to be appended = {}", logName(), numLogEntries);
-
- // 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;
- if (context.getReplicatedLog().size() > 0) {
-
- // Find the entry up until the one that is not in the follower's log
- for (int i = 0;i < numLogEntries; i++, addEntriesFrom++) {
- ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
-
- if (!isLogEntryPresent(matchEntry.getIndex())) {
- // newEntry not found in the log
- break;
- }
-
- long existingEntryTerm = getLogEntryTerm(matchEntry.getIndex());
-
- 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()) {
- continue;
- }
-
- if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
-
- 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())) {
- // 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.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;
- }
-
- break;
- } else {
- updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
- sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
- lastTerm(), context.getPayloadVersion(), true), actor());
- return this;
- }
- }
- }
-
- lastIndex = lastIndex();
- 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<ReplicatedLogEntry> appendAndPersistCallback = logEntry -> {
- final List<ReplicatedLogEntry> entries = appendEntries.getEntries();
- final ReplicatedLogEntry lastEntryToAppend = entries.get(entries.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 < numLogEntries; i++) {
- ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
-
- log.debug("{}: Append entry to log {}", logName(), entry.getData());
-
- context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback, false);
-
- shouldCaptureSnapshot.compareAndSet(false,
- context.getReplicatedLog().shouldCaptureSnapshot(entry.getIndex()));
-
- if (entry.getData() instanceof ServerConfigurationPayload) {
- context.updatePeerIds((ServerConfigurationPayload)entry.getData());
- }
- }
-
- log.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
+ if (!processNewEntries(appendEntries, sender)) {
+ updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
+ return this;
}
- // 5. If leaderCommit > commitIndex, set commitIndex =
- // min(leaderCommit, index of last new entry)
-
- lastIndex = lastIndex();
+ long lastIndex = lastIndex();
long prevCommitIndex = context.getCommitIndex();
+ // If leaderCommit > commitIndex, set commitIndex = min(leaderCommit, index of last new entry)
if (appendEntries.getLeaderCommit() > prevCommitIndex) {
context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
}
}
// 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) {
+ updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
+
+ // If leaderCommit > lastApplied, increment lastApplied and apply log[lastApplied] to state machine (§5.3).
+ // lastApplied can be equal to lastIndex.
+ if (appendEntries.getLeaderCommit() > context.getLastApplied() && context.getLastApplied() < lastIndex) {
if (log.isDebugEnabled()) {
log.debug("{}: applyLogToStateMachine, appendEntries.getLeaderCommit(): {}, "
+ "context.getLastApplied(): {}, lastIndex(): {}", logName(),
return this;
}
- private boolean isOutOfSync(final AppendEntries appendEntries) {
+ private boolean processNewEntries(final AppendEntries appendEntries, final ActorRef sender) {
+ int numLogEntries = appendEntries.getEntries().size();
+ if (numLogEntries == 0) {
+ return true;
+ }
+
+ log.debug("{}: Number of entries to be appended = {}", logName(), numLogEntries);
+
+ long lastIndex = lastIndex();
+ int addEntriesFrom = 0;
+
+ // First check for conflicting entries. If an existing entry conflicts with a new one (same index but different
+ // term), delete the existing entry and all that follow it (§5.3)
+ if (context.getReplicatedLog().size() > 0) {
+ // Find the entry up until the one that is not in the follower's log
+ for (int i = 0;i < numLogEntries; i++, addEntriesFrom++) {
+ ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
+
+ if (!isLogEntryPresent(matchEntry.getIndex())) {
+ // newEntry not found in the log
+ break;
+ }
+
+ long existingEntryTerm = getLogEntryTerm(matchEntry.getIndex());
+
+ 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()) {
+ continue;
+ }
+
+ if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
+ 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())) {
+ // 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.info("{}: Could not remove entries - sending reply to force snapshot", logName());
+ sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
+ lastTerm(), context.getPayloadVersion(), true), actor());
+ return false;
+ }
+
+ break;
+ } else {
+ sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
+ lastTerm(), context.getPayloadVersion(), true), actor());
+ return false;
+ }
+ }
+ }
+
+ lastIndex = lastIndex();
+ 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<ReplicatedLogEntry> appendAndPersistCallback = logEntry -> {
+ final List<ReplicatedLogEntry> entries = appendEntries.getEntries();
+ final ReplicatedLogEntry lastEntryToAppend = entries.get(entries.size() - 1);
+ if (shouldCaptureSnapshot.get() && logEntry == lastEntryToAppend) {
+ context.getSnapshotManager().capture(context.getReplicatedLog().last(), getReplicatedToAllIndex());
+ }
+ };
+
+ // Append any new entries not already in the log
+ for (int i = addEntriesFrom; i < numLogEntries; i++) {
+ ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
+
+ log.debug("{}: Append entry to log {}", logName(), entry.getData());
+
+ context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback, false);
+
+ shouldCaptureSnapshot.compareAndSet(false,
+ context.getReplicatedLog().shouldCaptureSnapshot(entry.getIndex()));
+
+ if (entry.getData() instanceof ServerConfigurationPayload) {
+ context.updatePeerIds((ServerConfigurationPayload)entry.getData());
+ }
+ }
+
+ log.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
+
+ return true;
+ }
+
+ private boolean isOutOfSync(final AppendEntries appendEntries, final ActorRef sender) {
final long lastIndex = lastIndex();
if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
log.info("{}: The followers log is empty and the senders prevLogIndex is {}", logName(),
appendEntries.getPrevLogIndex());
+
+ sendOutOfSyncAppendEntriesReply(sender, false);
return true;
}
+ "entries prevLogTerm {} - lastIndex: {}, snapshotIndex: {}", logName(),
appendEntries.getPrevLogIndex(), prevLogTerm, appendEntries.getPrevLogTerm(), lastIndex,
context.getReplicatedLog().getSnapshotIndex());
+
+ sendOutOfSyncAppendEntriesReply(sender, false);
return true;
}
} else if (appendEntries.getPrevLogIndex() != -1) {
log.info("{}: The log is not empty but the prevLogIndex {} was not found in it - lastIndex: {}, "
+ "snapshotIndex: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex,
context.getReplicatedLog().getSnapshotIndex());
+
+ sendOutOfSyncAppendEntriesReply(sender, false);
return true;
}
}
log.info("{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the "
+ "in-memory journal", logName(), appendEntries.getReplicatedToAllIndex());
+
+ sendOutOfSyncAppendEntriesReply(sender, false);
return true;
}
if (entries.size() > 0 && !isLogEntryPresent(entries.get(0).getIndex() - 1)) {
log.info("{}: Cannot append entries because the calculated previousIndex {} was not found in the "
+ "in-memory journal", logName(), entries.get(0).getIndex() - 1);
+
+ sendOutOfSyncAppendEntriesReply(sender, false);
return true;
}
}
return false;
}
+ private void sendOutOfSyncAppendEntriesReply(final ActorRef sender, boolean forceInstallSnapshot) {
+ // We found that the log was out of sync so just send a negative reply.
+ final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex(),
+ lastTerm(), context.getPayloadVersion(), forceInstallSnapshot);
+
+ log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply);
+ sender.tell(reply, actor());
+ }
+
@Override
protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
final AppendEntriesReply appendEntriesReply) {