Fix warnings and clean up javadocs in sal-akka-raft
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehavior.java
index 375b3779b891dd94e1a3b702f9396853478de265..5c5c520761ecc4fba107cf5274a95292dc207273 100644 (file)
@@ -28,41 +28,43 @@ import org.slf4j.Logger;
 import scala.concurrent.duration.FiniteDuration;
 
 /**
- * Abstract class that represents the behavior of a RaftActor
- * <p/>
- * All Servers:
- * <ul>
- * <li> If commitIndex > lastApplied: increment lastApplied, apply
- * log[lastApplied] to state machine (§5.3)
- * <li> If RPC request or response contains term T > currentTerm:
- * set currentTerm = T, convert to follower (§5.1)
+ * Abstract class that provides common code for a RaftActor behavior.
  */
 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     /**
-     * Information about the RaftActor whose behavior this class represents
+     * Information about the RaftActor whose behavior this class represents.
      */
     protected final RaftActorContext context;
 
     /**
-     *
+     * Used for message logging.
      */
-    protected final Logger LOG;
+    protected final Logger log;
 
     /**
-     *
+     * Prepended to log messages to provide appropriate context.
      */
-    private Cancellable electionCancel = null;
-
-    private long replicatedToAllIndex = -1;
-
     private final String logName;
 
+    /**
+     * The RaftState corresponding to his behavior.
+     */
     private final RaftState state;
 
+    /**
+     * Used to cancel a scheduled election.
+     */
+    private Cancellable electionCancel = null;
+
+    /**
+     * The index of the last log entry that has been replicated to all raft peers.
+     */
+    private long replicatedToAllIndex = -1;
+
     AbstractRaftActorBehavior(final RaftActorContext context, final RaftState state) {
         this.context = Preconditions.checkNotNull(context);
         this.state = Preconditions.checkNotNull(state);
-        this.LOG = context.getLogger();
+        this.log = context.getLogger();
 
         logName = String.format("%s (%s)", context.getId(), state);
     }
@@ -118,29 +120,22 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     protected abstract RaftActorBehavior handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries);
 
-
     /**
-     * appendEntries first processes the AppendEntries message and then
-     * delegates handling to a specific behavior
+     * Handles the common logic for the AppendEntries message and delegates handling to the derived class.
      *
-     * @param sender
-     * @param appendEntries
+     * @param sender the ActorRef that sent the message
+     * @param appendEntries the message
      * @return a new behavior if it was changed or the current behavior
      */
-    protected RaftActorBehavior appendEntries(ActorRef sender,
-        AppendEntries appendEntries) {
+    protected RaftActorBehavior appendEntries(ActorRef sender, AppendEntries appendEntries) {
 
         // 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 {}",
-                        logName(), appendEntries.getTerm(), currentTerm());
-            }
+            log.debug("{}: Cannot append entries because sender term {} is less than {}", logName(),
+                    appendEntries.getTerm(), currentTerm());
 
-            sender.tell(
-                new AppendEntriesReply(context.getId(), currentTerm(), false,
-                    lastIndex(), lastTerm(), context.getPayloadVersion()), actor()
-            );
+            sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex(), lastTerm(),
+                    context.getPayloadVersion()), actor());
             return this;
         }
 
@@ -164,33 +159,32 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         AppendEntriesReply appendEntriesReply);
 
     /**
-     * requestVote handles the RequestVote message. This logic is common
-     * for all behaviors
+     * Handles the logic for the RequestVote message that is common for all behaviors.
      *
-     * @param sender
-     * @param requestVote
+     * @param sender the ActorRef that sent the message
+     * @param requestVote the message
      * @return a new behavior if it was changed or the current behavior
      */
     protected RaftActorBehavior requestVote(ActorRef sender, RequestVote requestVote) {
 
-        LOG.debug("{}: In requestVote:  {}", logName(), requestVote);
+        log.debug("{}: In requestVote:  {}", logName(), requestVote);
 
         boolean grantVote = canGrantVote(requestVote);
 
-        if(grantVote) {
+        if (grantVote) {
             context.getTermInformation().updateAndPersist(requestVote.getTerm(), requestVote.getCandidateId());
         }
 
         RequestVoteReply reply = new RequestVoteReply(currentTerm(), grantVote);
 
-        LOG.debug("{}: requestVote returning: {}", logName(), reply);
+        log.debug("{}: requestVote returning: {}", logName(), reply);
 
         sender.tell(reply, actor());
 
         return this;
     }
 
-    protected boolean canGrantVote(RequestVote requestVote){
+    protected boolean canGrantVote(RequestVote requestVote) {
         boolean grantVote = false;
 
         //  Reply false if term < currentTerm (§5.1)
@@ -213,7 +207,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
             // more up-to-date.
             if (requestVote.getLastLogTerm() > lastTerm()) {
                 candidateLatest = true;
-            } else if ((requestVote.getLastLogTerm() == lastTerm())
+            } else if (requestVote.getLastLogTerm() == lastTerm()
                     && requestVote.getLastLogIndex() >= lastIndex()) {
                 candidateLatest = true;
             }
@@ -241,6 +235,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         RequestVoteReply requestVoteReply);
 
     /**
+     * Returns a duration for election with an additional variance for randomness.
      *
      * @return a random election duration
      */
@@ -251,7 +246,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     /**
-     * stop the scheduled election
+     * Stops the currently scheduled election.
      */
     protected void stopElection() {
         if (electionCancel != null && !electionCancel.isCancelled()) {
@@ -264,7 +259,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     /**
-     * schedule a new election
+     * Schedule a new election.
      *
      * @param interval the duration after which we should trigger a new election
      */
@@ -277,6 +272,8 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     /**
+     * Returns the current election term.
+     *
      * @return the current term
      */
     protected long currentTerm() {
@@ -284,6 +281,8 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     /**
+     * Returns the id of the candidate that this server voted for in current term.
+     *
      * @return the candidate for whom we voted in the current term
      */
     protected String votedFor() {
@@ -291,46 +290,53 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     /**
-     * @return the actor associated with this behavior
+     * Returns the actor associated with this behavior.
+     *
+     * @return the actor
      */
     protected ActorRef actor() {
         return context.getActor();
     }
 
     /**
+     * Returns the term of the last entry in the log.
      *
-     * @return the term from the last entry in the log
+     * @return the term
      */
     protected long lastTerm() {
         return context.getReplicatedLog().lastTerm();
     }
 
     /**
-     * @return the index from the last entry in the log
+     * Returns the index of the last entry in the log.
+     *
+     * @return the index
      */
     protected long lastIndex() {
         return context.getReplicatedLog().lastIndex();
     }
 
     /**
-     * @param logIndex
-     * @return the client request tracker for the specified logIndex
+     * Removes and returns the ClientRequestTracker for the specified log index.
+     * @param logIndex the log index
+     * @return the ClientRequestTracker or null if none available
      */
     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
         return null;
     }
 
     /**
+     * Returns the actual index of the entry in replicated log for the given index or -1 if not found.
      *
-     * @return the log entry index for the given index or -1 if not found
+     * @return the log entry index or -1 if not found
      */
-    protected long getLogEntryIndex(long index){
-        if(index == context.getReplicatedLog().getSnapshotIndex()){
+    protected long getLogEntryIndex(long index) {
+        if (index == context.getReplicatedLog().getSnapshotIndex()) {
             return context.getReplicatedLog().getSnapshotIndex();
         }
 
         ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
-        if(entry != null){
+        if (entry != null) {
             return entry.getIndex();
         }
 
@@ -338,15 +344,17 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     /**
-     * @return the log entry term for the given index or -1 if not found
+     * Returns the actual term of the entry in replicated log for the given index or -1 if not found.
+     *
+     * @return the log entry term or -1 if not found
      */
-    protected long getLogEntryTerm(long index){
-        if(index == context.getReplicatedLog().getSnapshotIndex()){
+    protected long getLogEntryTerm(long index) {
+        if (index == context.getReplicatedLog().getSnapshotIndex()) {
             return context.getReplicatedLog().getSnapshotTerm();
         }
 
         ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
-        if(entry != null){
+        if (entry != null) {
             return entry.getTerm();
         }
 
@@ -354,9 +362,9 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     /**
-     * Apply the provided index to the state machine
+     * Applies the log entries up to the specified index that is known to be committed to the state machine.
      *
-     * @param index a log index that is known to be committed
+     * @param index the log index
      */
     protected void applyLogToStateMachine(final long index) {
         long newLastApplied = context.getLastApplied();
@@ -381,14 +389,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.warn("{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}",
+                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 {}", logName(), newLastApplied);
-        }
+
+        log.debug("{}: Setting last applied to {}", logName(), newLastApplied);
+
         context.setLastApplied(newLastApplied);
 
         // send a message to persist a ApplyLogEntries marker message into akka's persistent journal
@@ -422,16 +430,17 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         return internalSwitchBehavior(createBehavior(context, newState));
     }
 
+    @SuppressWarnings("checkstyle:IllegalCatch")
     protected RaftActorBehavior internalSwitchBehavior(RaftActorBehavior newBehavior) {
-        if(!context.getRaftPolicy().automaticElectionsEnabled()) {
+        if (!context.getRaftPolicy().automaticElectionsEnabled()) {
             return this;
         }
 
-        LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), newBehavior.state());
+        log.info("{} :- Switching from behavior {} to {}", logName(), this.state(), newBehavior.state());
         try {
             close();
-        } catch (Exception e) {
-            LOG.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
+        } catch (RuntimeException e) {
+            log.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
         }
         return newBehavior;
     }
@@ -462,20 +471,20 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
 
 
     /**
-     * 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.
+     * Performs a snapshot with no capture on the replicated log. It clears the log from the supplied index or
+     * lastApplied-1 which ever is minimum.
      *
-     * @param snapshotCapturedIndex
+     * @param snapshotCapturedIndex the index from which to clear
      */
     protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
         long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex);
 
-        if(actualIndex != -1){
+        if (actualIndex != -1) {
             setReplicatedToAllIndex(actualIndex);
         }
     }
 
-    protected String getId(){
+    protected String getId() {
         return context.getId();
     }
 }