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

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

[~ckj], There is a race between leader election and committing new logs.  Yes, 
we can increase the term early to avoid changing back to follower by 
appendEntries.  However, the old leader and the other follower won't vote for 
it since they have committed new log.

The question is -- do we support transfer leadership when the group is busy?  
If yes, we need to step down the leader first.  We may provide an option to 
"force" transfer leadership.

> Fix TransferLeadership stopped by appendEntries from old leader
> ---------------------------------------------------------------
>
>                 Key: RATIS-1796
>                 URL: https://issues.apache.org/jira/browse/RATIS-1796
>             Project: Ratis
>          Issue Type: Sub-task
>            Reporter: Kaijie Chen
>            Assignee: Kaijie Chen
>            Priority: Major
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> Candidate state of transferee may be stopped by the appendEntries from old 
> leader, see the log below 
> {code:java}
> 2023-02-28 04:52:45,026 [s0-server-thread1] INFO  impl.TransferLeadership 
> (TransferLeadership.java:tryTransferLeadership(107)) - s0@group-43918D205BB2: 
> start transferring leadership to s1
> 2023-02-28 04:52:45,029 [s0-server-thread1] INFO  impl.TransferLeadership 
> (TransferLeadership.java:tryTransferLeadership(116)) - s0@group-43918D205BB2: 
> sent StartLeaderElection to transferee s1 immediately as it already has 
> up-to-date log
> 2023-02-28 04:52:45,031 [grpc-default-executor-6] INFO  impl.RoleInfo 
> (RoleInfo.java:shutdownFollowerState(111)) - s1: shutdown 
> s1@group-43918D205BB2-FollowerState
> 2023-02-28 04:52:45,032 [s1@group-43918D205BB2-FollowerState] INFO  
> impl.FollowerState (FollowerState.java:run(152)) - 
> s1@group-43918D205BB2-FollowerState was interrupted
> 2023-02-28 04:52:45,032 [grpc-default-executor-6] INFO  impl.RoleInfo 
> (RoleInfo.java:updateAndGet(140)) - s1: start 
> s1@group-43918D205BB2-LeaderElection4
> 2023-02-28 04:52:45,054 [s1-server-thread1] INFO  impl.RoleInfo 
> (RoleInfo.java:shutdownLeaderElection(131)) - s1: shutdown 
> s1@group-43918D205BB2-LeaderElection4
> 2023-02-28 04:52:45,054 [s1-server-thread1] INFO  impl.RoleInfo 
> (RoleInfo.java:startFollowerState(104)) - s1: startFollowerState 
> reason:appendEntries from s0 term 1,
>   trace: java.base/java.lang.Thread.getStackTrace(Thread.java:1602),
>     
> org.apache.ratis.server.impl.RoleInfo.startFollowerState(RoleInfo.java:104),
>     
> org.apache.ratis.server.impl.RaftServerImpl.changeToFollower(RaftServerImpl.java:547),
>     
> org.apache.ratis.server.impl.RaftServerImpl.changeToFollowerAndPersistMetadata(RaftServerImpl.java:556),
>     
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1498),
>     
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1396),
>     
> org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$26(RaftServerProxy.java:639),
>     org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:117),
>     
> org.apache.ratis.server.impl.RaftServerImpl.lambda$executeSubmitServerRequestAsync$11(RaftServerImpl.java:818),
>     
> java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700),
>     
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128),
>     
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628),
>     java.base/java.lang.Thread.run(Thread.java:829)
> 2023-02-28 04:52:45,055 [s1-server-thread1] INFO  impl.RoleInfo 
> (RoleInfo.java:updateAndGet(140)) - s1: start 
> s1@group-43918D205BB2-FollowerState
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to