Xinhao GU created RATIS-2274:
--------------------------------
Summary: Newly added peer may retain outdated configuration after
membership change, causing election failure
Key: RATIS-2274
URL: https://issues.apache.org/jira/browse/RATIS-2274
Project: Ratis
Issue Type: Improvement
Components: conf, election, gRPC, raft-group
Reporter: Xinhao GU
Assignee: Xinhao GU
Attachments: image-2025-04-04-16-53-49-441.png
h1. *Problem Description*
When dynamically adding a new peer (G) through membership change, the peer
might retain an obsolete configuration (e.g., ancient {{{}[A,B,C]{}}}) instead
of adopting the latest configuration {{{}[D,E,F]{}}}. If the original leader
(D) and another peer (E) fail simultaneously, the surviving peer (F) will
trigger an election. The newly added peer G (retaining old config) will reject
F's vote request because F doesn't exist in its configuration
({{{}[A,B,C]{}}}), leading to election deadlock and cluster unavailability.
h2. *Steps to Reproduce*
# Initial Cluster: 3 peers {{[D(Leader), E, F]}} with all confirmed at config
{{[D,E,F]}}
# Client sends {{setConfiguration(D,E,F,G)}}
# Leader D creates {{GrpcLogAppender}} thread for peer G
# New peer G joins and starts log catch-up
# Leader marks G as "caught up" based on current criteria, triggers joint
consensus mechanism to commit new config {{[D,E,F,G]}}
# Client receives success response, cluster considers config {{[D,E,F,G]}}
active
# Simulate failures: Shut down Leader D and peer E
# Peer F initiates election, sends {{RequestVote}} RPC to G
h2. *Expected Behavior*
After new config {{[D,E,F,G]}} is committed, all peers including G should use
the latest config for elections and log replication.
h2. *Actual Behavior*
Peer G retains outdated config {{[A,B,C]}} (not containing F), rejecting F's
vote request and preventing leader election.
h2. *Log Evidence*
{code:java}
{code}
*2025-01-18 00:29:35,164 [grpc-default-executor-1] INFO
o.a.r.s.i.RaftServerImpl:1410 - G@group-00020000000F: receive
requestVote(PRE_VOTE, F, group-00020000000F, 149, (t:149, i:606))*
*2025-01-18 00:29:35,164 [grpc-default-executor-1] INFO
o.a.r.s.impl.VoteContext:49 - G@group-00020000000F-FOLLOWER: reject PRE_VOTE
from F: F is not in current conf [A|192.168.130.3:10750, B|192.168.130.5:10750,
C|192.168.130.4:10750]*
*2025-01-18 00:29:35,164 [grpc-default-executor-1] INFO
o.a.r.s.i.RaftServerImpl:1444 - G@group-00020000000F replies to PRE_VOTE vote
request: F<-G#0:FAIL-t149. Peer's state: G@group-00020000000F:t149, leader=D,
voted=null,
raftlog=Memoized:G@group-00020000000F-SegmentedRaftLog:OPENED:c333:last(t:78,
i:357), conf=conf: \{index: 447, cur=peers:[A|192.168.130.3:10750,
B|192.168.130.5:10750, C|192.168.130.4:10750]|listeners:[], old=null}*
h1. *Root Cause Analysis*
The current catch-up validation in {{checkProgress()}} relies on:
* {{stagingCatchupGap}} (default 1000 entries)
* {{LastRpcResponseTime}}
* {{hasAttemptedToInstallSnapshot}}
!image-2025-04-04-16-53-49-441.png|width=657,height=251!{{{}{}}}
{{{*}Critical Flaw{*}: When the leader has no snapshot (forcing log-based
catch-up), the new peer might *lack timely configuration change logs* due to
}}{{{}stagingCatchupGap{}}}{{{}. This leaves the new peer with outdated
configurations (e.g., {}}}{{{}[A,B,C]{}}}{{{}), violating Raft's membership
change safety guarantees.{}}}
{{{}To solve this problem, the Leader is essentially required to ensure that
the configuration of the Follower is the latest when the Follower completes the
catch-up.{}}}{{{}{}}}
h1. *Proposed Solutions*
{{Observe the checkProgress() function to determine the CAUGHTUP logic. We can
modify the logic in the following ways.}}
h2. First Idea
*More Strict Catch-Up* *Validation*
Add one more condition to the CAUGHTUP judgment logic. That is, CAUGHTUP is
complete only after a follower receives the latest conf logs from the leader.
{code:java}
follower.getMatchIndex() >= server.getRaftConf().getLogEntryIndex() {code}
h2. Second Idea
*Snapshot Enforcement*
{{If the follower is in the Bootstrapping state, the snapshot is forcibly
transmitted first. If the leader does not have the snapshot, a new snapshot is
generated and then sent. (Because the snapshot carries the latest conf)}}
{code:java}
if (follower.isBootstrapping() && !leader.hasSnapshot()) {
leader.triggerSnapshotCreation();
}{code}
h2. Third Idea
*Disable* *Staging* ** *Gap* *for Config Changes*
{{Set stagingCatchupGap to 0. CAUGHTUP is complete only when the matIndex of
followers is completely equal to the commitIndex of the current leader.}}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)