BUG 2773 : Transition Shard to Leader state when it has no peers
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehavior.java
index 04462be0420eaa3f0504be0523b7c9371128273d..45671ea31e4c804f9993df96e3d534ceaf6e4247 100644 (file)
@@ -10,20 +10,21 @@ package org.opendaylight.controller.cluster.raft.behaviors;
 
 import akka.actor.ActorRef;
 import akka.actor.Cancellable;
-import akka.event.LoggingAdapter;
 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.SerializationUtils;
-import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
+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.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
+import org.slf4j.Logger;
 import scala.concurrent.duration.FiniteDuration;
 
 /**
@@ -38,6 +39,8 @@ import scala.concurrent.duration.FiniteDuration;
  */
 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
 
+    protected static final ElectionTimeout ELECTION_TIMEOUT = new ElectionTimeout();
+
     /**
      * Information about the RaftActor whose behavior this class represents
      */
@@ -46,7 +49,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     /**
      *
      */
-    protected final LoggingAdapter LOG;
+    protected final Logger LOG;
 
     /**
      *
@@ -58,10 +61,37 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      */
     protected String leaderId = 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;
     }
 
     /**
@@ -94,8 +124,8 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         // 1. Reply false if term < currentTerm (ยง5.1)
         if (appendEntries.getTerm() < currentTerm()) {
             if(LOG.isDebugEnabled()) {
-                LOG.debug("Cannot append entries because sender term {} is less than {}",
-                        appendEntries.getTerm(), currentTerm());
+                LOG.debug("{}: Cannot append entries because sender term {} is less than {}",
+                        logName(), appendEntries.getTerm(), currentTerm());
             }
 
             sender.tell(
@@ -132,12 +162,9 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      * @param requestVote
      * @return
      */
-    protected RaftActorBehavior requestVote(ActorRef sender,
-        RequestVote requestVote) {
+    protected RaftActorBehavior requestVote(ActorRef sender, RequestVote requestVote) {
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug(requestVote.toString());
-        }
+        LOG.debug("{}: In requestVote:  {}", logName(), requestVote);
 
         boolean grantVote = false;
 
@@ -173,7 +200,11 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
             }
         }
 
-        sender.tell(new RequestVoteReply(currentTerm(), grantVote), actor());
+        RequestVoteReply reply = new RequestVoteReply(currentTerm(), grantVote);
+
+        LOG.debug("{}: requestVote returning: {}", logName(), reply);
+
+        sender.tell(reply, actor());
 
         return this;
     }
@@ -225,7 +256,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         // message is sent to itself
         electionCancel =
             context.getActorSystem().scheduler().scheduleOnce(interval,
-                context.getActor(), new ElectionTimeout(),
+                context.getActor(), ELECTION_TIMEOUT,
                 context.getActorSystem().dispatcher(), context.getActor());
     }
 
@@ -349,13 +380,14 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
             } else {
                 //if one index is not present in the log, no point in looping
                 // around as the rest wont be present either
-                LOG.warning(
-                        "Missing index {} from log. Cannot apply state. Ignoring {} to {}", i, i, index);
+                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 {}", newLastApplied);
+            LOG.debug("{}: Setting last applied to {}", logName(), newLastApplied);
         }
         context.setLastApplied(newLastApplied);
 
@@ -363,7 +395,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         // 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 ApplyLogEntries((int) context.getLastApplied()), actor());
+        actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
     }
 
     protected Object fromSerializableMessage(Object serializable){
@@ -389,11 +421,11 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     protected RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
-        LOG.info("{} :- Switching from behavior {} to {}", context.getId(), this.state(), behavior.state());
+        LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), behavior.state());
         try {
             close();
         } catch (Exception e) {
-            LOG.error(e, "Failed to close behavior : {}", this.state());
+            LOG.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
         }
 
         return behavior;
@@ -421,4 +453,45 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         return numMajority;
 
     }
+
+
+    /**
+     * 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();
+    }
+
 }