[ 
https://issues.apache.org/jira/browse/RATIS-698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16942448#comment-16942448
 ] 

Tsz-wo Sze commented on RATIS-698:
----------------------------------

[~swagle], thanks for working on this.

The 01 patch adds a ThreadLocal boolean to avoid the first minTimeoutMs.  It is 
not easy to understand how it works.  Also, if we change the threading model 
later, it will become not working.

I wonder if we could move the boolean to a local variable in FollowerState.
{code}
@@ -57,6 +58,7 @@ class FollowerState extends Daemon {
   private volatile Timestamp lastRpcTime = Timestamp.currentTime();
   private volatile boolean isRunning = true;
   private final AtomicInteger outstandingOp = new AtomicInteger();
+  private final AtomicBoolean includeMinTimeout = new AtomicBoolean();
 
   FollowerState(RaftServerImpl server) {
     this.name = server.getMemberId() + "-" + getClass().getSimpleName();
@@ -92,7 +94,7 @@ class FollowerState extends Daemon {
   public  void run() {
     long sleepDeviationThresholdMs = server.getSleepDeviationThresholdMs();
     while (isRunning && server.isFollower()) {
-      final long electionTimeout = server.getRandomTimeoutMs();
+      final long electionTimeout = 
server.getRandomTimeoutMs(includeMinTimeout.getAndSet(true));
       try {
         if (!JavaUtils.sleep(electionTimeout, sleepDeviationThresholdMs)) {
           continue;
{code}


> RaftServerImpl should not enforce minTimeout before first leaderElection cycle
> ------------------------------------------------------------------------------
>
>                 Key: RATIS-698
>                 URL: https://issues.apache.org/jira/browse/RATIS-698
>             Project: Ratis
>          Issue Type: Improvement
>          Components: server
>    Affects Versions: 0.4.0
>            Reporter: Siddharth Wagle
>            Assignee: Siddharth Wagle
>            Priority: Major
>             Fix For: 0.5.0
>
>         Attachments: RATIS-698.01.patch
>
>
> Followers always wait for (minTimeoutMillis + randomWait), before initiating 
> leader election. This penalizes the first time the ratis ring is created or 
> when server restarts by enforcing a wait for leader election term.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to