Bug 7747: Reply to the leader before applying previous state
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Follower.java
index 30c1264ee89acf122c5244faa2c8e7c1fd815e7d..b512089692eed80da136bc8f7dfc2f5a2e10274c 100644 (file)
@@ -284,6 +284,18 @@ public class Follower extends AbstractRaftActorBehavior {
             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.
+        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
@@ -298,17 +310,6 @@ public class Follower extends AbstractRaftActorBehavior {
             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());
         }