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 ba998d3295d9d13b1e234890aa149ed4fb842032..15dbd74d01d9b2d3b3aed4b98fdb414b3943c5ae 100644 (file)
@@ -216,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());
         }
 
@@ -629,14 +629,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    protected void sendAppendEntries(final long timeSinceLastActivityInterval, final 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);
             }
         }
@@ -949,7 +949,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     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();
         }