Measure follower activity in nanoseconds
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
index 9fc7b3393cbe1f86404e2eb9a08a9ca6f6e21bc1..15dbd74d01d9b2d3b3aed4b98fdb414b3943c5ae 100644 (file)
@@ -14,7 +14,6 @@ import akka.actor.Cancellable;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
 import com.google.common.io.ByteSource;
 import java.io.IOException;
 import java.io.ObjectOutputStream;
@@ -35,7 +34,6 @@ import org.opendaylight.controller.cluster.messaging.SliceOptions;
 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
-import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
 import org.opendaylight.controller.cluster.raft.PeerInfo;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
@@ -58,7 +56,7 @@ import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
 import scala.concurrent.duration.FiniteDuration;
 
 /**
- * The behavior of a RaftActor when it is in the Leader state
+ * The behavior of a RaftActor when it is in the Leader state.
  *
  * <p>
  * Leaders:
@@ -102,8 +100,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     private Optional<SnapshotHolder> snapshotHolder = Optional.absent();
     private int minReplicationCount;
 
-    protected AbstractLeader(RaftActorContext context, RaftState state,
-            @Nullable AbstractLeader initializeFromLeader) {
+    protected AbstractLeader(final RaftActorContext context, final RaftState state,
+            @Nullable final AbstractLeader initializeFromLeader) {
         super(context, state);
 
         appendEntriesMessageSlicer = MessageSlicer.builder().logContext(logName())
@@ -117,7 +115,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             trackers.addAll(initializeFromLeader.trackers);
         } else {
             for (PeerInfo peerInfo: context.getPeers()) {
-                FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context);
+                FollowerLogInformation followerLogInformation = new FollowerLogInformation(peerInfo, context);
                 followerToLog.put(peerInfo.getId(), followerLogInformation);
             }
         }
@@ -136,7 +134,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
     }
 
-    protected AbstractLeader(RaftActorContext context, RaftState state) {
+    protected AbstractLeader(final RaftActorContext context, final RaftState state) {
         this(context, state, null);
     }
 
@@ -149,9 +147,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return followerToLog.keySet();
     }
 
-    public void addFollower(String followerId) {
-        FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(
-                context.getPeerInfo(followerId), -1, context);
+    public void addFollower(final String followerId) {
+        FollowerLogInformation followerLogInformation = new FollowerLogInformation(context.getPeerInfo(followerId),
+            context);
         followerToLog.put(followerId, followerLogInformation);
 
         if (heartbeatSchedule == null) {
@@ -159,7 +157,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    public void removeFollower(String followerId) {
+    public void removeFollower(final String followerId) {
         followerToLog.remove(followerId);
     }
 
@@ -186,7 +184,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    void setSnapshot(@Nullable SnapshotHolder snapshotHolder) {
+    void setSnapshotHolder(@Nullable final SnapshotHolder snapshotHolder) {
         this.snapshotHolder = Optional.fromNullable(snapshotHolder);
     }
 
@@ -196,8 +194,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleAppendEntries(ActorRef sender,
-        AppendEntries appendEntries) {
+    protected RaftActorBehavior handleAppendEntries(final ActorRef sender,
+        final AppendEntries appendEntries) {
 
         log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
 
@@ -205,7 +203,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) {
+    protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
+            final AppendEntriesReply appendEntriesReply) {
         log.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
 
         // Update the FollowerLogInformation
@@ -217,11 +216,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             return this;
         }
 
-        if (followerLogInformation.timeSinceLastActivity()
-                > context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
+        final long lastActivityNanos = followerLogInformation.nanosSinceLastActivity();
+        if (lastActivityNanos > context.getConfigParams().getElectionTimeOutInterval().toNanos()) {
             log.warn("{} : handleAppendEntriesReply delayed beyond election timeout, "
                     + "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
-                    logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
+                    logName(), appendEntriesReply, TimeUnit.NANOSECONDS.toMillis(lastActivityNanos),
                     context.getLastApplied(), context.getCommitIndex());
         }
 
@@ -301,8 +300,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
                 log.info("{}: follower {} appears to be behind the leader from the last snapshot - "
                     + "updated: matchIndex: {}, nextIndex: {}", logName(), followerId,
-                    appendEntriesReply.getLogLastTerm(), followerLogInformation.getMatchIndex(),
-                    followerLogInformation.getNextIndex());
+                    followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
             } else {
                 // The follower's log conflicts with leader's log so decrement follower's next index by 1
                 // in an attempt to find where the logs match.
@@ -403,8 +401,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
-            AppendEntriesReply appendEntriesReply) {
+    private boolean updateFollowerLogInformation(final FollowerLogInformation followerLogInformation,
+            final AppendEntriesReply appendEntriesReply) {
         boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
         updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
 
@@ -431,7 +429,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
+    protected ClientRequestTracker removeClientRequestTracker(final long logIndex) {
         final Iterator<ClientRequestTracker> it = trackers.iterator();
         while (it.hasNext()) {
             final ClientRequestTracker t = it.next();
@@ -445,15 +443,15 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
-        RequestVoteReply requestVoteReply) {
+    protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender,
+        final RequestVoteReply requestVoteReply) {
         return this;
     }
 
     protected void beforeSendHeartbeat(){}
 
     @Override
-    public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
+    public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
         Preconditions.checkNotNull(sender, "sender should not be null");
 
         if (appendEntriesMessageSlicer.handleMessage(message)) {
@@ -494,7 +492,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
         } else if (message instanceof SendInstallSnapshot) {
             SendInstallSnapshot sendInstallSnapshot = (SendInstallSnapshot) message;
-            setSnapshot(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), sendInstallSnapshot.getSnapshotBytes()));
+            setSnapshotHolder(new SnapshotHolder(sendInstallSnapshot.getSnapshot(),
+                sendInstallSnapshot.getSnapshotBytes()));
             sendInstallSnapshot();
         } else if (message instanceof Replicate) {
             replicate((Replicate) message);
@@ -509,7 +508,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return this;
     }
 
-    private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
+    private void handleInstallSnapshotReply(final InstallSnapshotReply reply) {
         log.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
 
         String followerId = reply.getFollowerId();
@@ -548,7 +547,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     if (!anyFollowersInstallingSnapshot()) {
                         // once there are no pending followers receiving snapshots
                         // we can remove snapshot from the memory
-                        setSnapshot(null);
+                        setSnapshotHolder(null);
                     }
 
                     wasLastChunk = true;
@@ -603,7 +602,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return false;
     }
 
-    private void replicate(Replicate replicate) {
+    private void replicate(final Replicate replicate) {
         long logIndex = replicate.getReplicatedLogEntry().getIndex();
 
         log.debug("{}: Replicate message: identifier: {}, logIndex: {}, payload: {}, isSendImmediate: {}", logName(),
@@ -630,14 +629,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
+    protected void sendAppendEntries(final long timeSinceLastActivityIntervalNanos, final boolean isHeartbeat) {
         // Send an AppendEntries to all followers
         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
             final String followerId = e.getKey();
             final FollowerLogInformation followerLogInformation = e.getValue();
             // This checks helps not to send a repeat message to the follower
             if (!followerLogInformation.isFollowerActive()
-                    || followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
+                    || followerLogInformation.nanosSinceLastActivity() >= timeSinceLastActivityIntervalNanos) {
                 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
             }
         }
@@ -647,8 +646,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      * This method checks if any update needs to be sent to the given follower. This includes append log entries,
      * sending next snapshot chunk, and initiating a snapshot.
      */
-    private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
-                                       boolean sendHeartbeat, boolean isHeartbeat) {
+    private void sendUpdatesToFollower(final String followerId, final FollowerLogInformation followerLogInformation,
+                                       final boolean sendHeartbeat, final boolean isHeartbeat) {
 
         ActorSelection followerActor = context.getPeerActorSelection(followerId);
         if (followerActor != null) {
@@ -725,8 +724,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private List<ReplicatedLogEntry> getEntriesToSend(FollowerLogInformation followerLogInfo,
-            ActorSelection followerActor) {
+    private List<ReplicatedLogEntry> getEntriesToSend(final FollowerLogInformation followerLogInfo,
+            final ActorSelection followerActor) {
         // Try to get all the entries in the journal but not exceeding the max data size for a single AppendEntries
         // message.
         int maxEntries = (int) context.getReplicatedLog().size();
@@ -795,8 +794,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return Collections.emptyList();
     }
 
-    private void sendAppendEntriesToFollower(ActorSelection followerActor, List<ReplicatedLogEntry> entries,
-            FollowerLogInformation followerLogInformation) {
+    private void sendAppendEntriesToFollower(final ActorSelection followerActor, final List<ReplicatedLogEntry> entries,
+            final FollowerLogInformation followerLogInformation) {
         // In certain cases outlined below we don't want to send the actual commit index to prevent the follower from
         // possibly committing and applying conflicting entries (those with same index, different term) from a prior
         // term that weren't replicated to a majority, which would be a violation of raft.
@@ -843,7 +842,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      * @param followerId the id of the follower.
      * @return true if capture was initiated, false otherwise.
      */
-    public boolean initiateCaptureSnapshot(String followerId) {
+    public boolean initiateCaptureSnapshot(final String followerId) {
         FollowerLogInformation followerLogInfo = followerToLog.get(followerId);
         if (snapshotHolder.isPresent()) {
             // If a snapshot is present in the memory, most likely another install is in progress no need to capture
@@ -865,7 +864,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return captureInitiated;
     }
 
-    private boolean canInstallSnapshot(long nextIndex) {
+    private boolean canInstallSnapshot(final long nextIndex) {
         // If the follower's nextIndex is -1 then we might as well send it a snapshot
         // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
         // in the snapshot
@@ -897,7 +896,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      *  Sends a snapshot chunk to a given follower
      *  InstallSnapshot should qualify as a heartbeat too.
      */
-    private void sendSnapshotChunk(ActorSelection followerActor, FollowerLogInformation followerLogInfo) {
+    private void sendSnapshotChunk(final ActorSelection followerActor, final FollowerLogInformation followerLogInfo) {
         if (snapshotHolder.isPresent()) {
             LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
             if (installSnapshotState == null) {
@@ -938,18 +937,19 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     actor()
                 );
 
-                log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
-                        installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
             } catch (IOException e) {
-                throw Throwables.propagate(e);
+                throw new RuntimeException(e);
             }
+
+            log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
+                installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
         }
     }
 
     private void sendHeartBeat() {
         if (!followerToLog.isEmpty()) {
             log.trace("{}: Sending heartbeat", logName());
-            sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
+            sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toNanos(), true);
 
             appendEntriesMessageSlicer.checkExpiredSlicedMessageState();
         }
@@ -961,7 +961,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private void scheduleHeartBeat(FiniteDuration interval) {
+    private void scheduleHeartBeat(final FiniteDuration interval) {
         if (followerToLog.isEmpty()) {
             // Optimization - do not bother scheduling a heartbeat as there are
             // no followers
@@ -1028,7 +1028,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    public FollowerLogInformation getFollower(String followerId) {
+    public FollowerLogInformation getFollower(final String followerId) {
         return followerToLog.get(followerId);
     }
 
@@ -1042,7 +1042,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         private final long lastIncludedIndex;
         private final ByteSource snapshotBytes;
 
-        SnapshotHolder(Snapshot snapshot, ByteSource snapshotBytes) {
+        SnapshotHolder(final Snapshot snapshot, final ByteSource snapshotBytes) {
             this.lastIncludedTerm = snapshot.getLastAppliedTerm();
             this.lastIncludedIndex = snapshot.getLastAppliedIndex();
             this.snapshotBytes = snapshotBytes;