From 9d791bcd575dd7fc4ab3a89180a67fcd402663e4 Mon Sep 17 00:00:00 2001 From: yuz10 <845238369@qq.com> Date: Sat, 14 Dec 2024 13:23:31 +0800 Subject: [PATCH 1/2] [ISSUE #325] Fix unable to auto recover after follower isolation --- .../storage/dledger/DLedgerLeaderElector.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java index e23bff61..516e018d 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java @@ -192,7 +192,7 @@ public void changeRoleToFollower(long term, String leaderId) { handleRoleChange(term, MemberState.Role.FOLLOWER); } - public CompletableFuture handleVote(VoteRequest request, boolean self) { + public CompletableFuture handleVote(VoteRequest request, boolean self) { //hold the lock to get the latest term, leaderId, ledgerEndIndex synchronized (memberState) { if (!memberState.isPeerMember(request.getLeaderId())) { @@ -204,6 +204,14 @@ public CompletableFuture handleVote(VoteRequest request, boolean s return CompletableFuture.completedFuture(new VoteResponse(request).term(memberState.currTerm()).voteResult(VoteResponse.RESULT.REJECT_UNEXPECTED_LEADER)); } + if (request.getTerm() > memberState.currTerm()) { + //stepped down by larger term + changeRoleToCandidate(request.getTerm()); + needIncreaseTermImmediately = true; + //only can handleVote when the term is consistent + return CompletableFuture.completedFuture(new VoteResponse(request).term(memberState.currTerm()).voteResult(VoteResponse.RESULT.REJECT_TERM_NOT_READY)); + } + if (request.getLedgerEndTerm() < memberState.getLedgerEndTerm()) { return CompletableFuture.completedFuture(new VoteResponse(request).term(memberState.currTerm()).voteResult(VoteResponse.RESULT.REJECT_EXPIRED_LEDGER_TERM)); } else if (request.getLedgerEndTerm() == memberState.getLedgerEndTerm() && request.getLedgerEndIndex() < memberState.getLedgerEndIndex()) { @@ -224,12 +232,6 @@ public CompletableFuture handleVote(VoteRequest request, boolean s return CompletableFuture.completedFuture(new VoteResponse(request).term(memberState.currTerm()).voteResult(VoteResponse.RESULT.REJECT_ALREADY_VOTED)); } } - } else { - //stepped down by larger term - changeRoleToCandidate(request.getTerm()); - needIncreaseTermImmediately = true; - //only can handleVote when the term is consistent - return CompletableFuture.completedFuture(new VoteResponse(request).term(memberState.currTerm()).voteResult(VoteResponse.RESULT.REJECT_TERM_NOT_READY)); } if (request.getTerm() < memberState.getLedgerEndTerm()) { From 951489b78c65250a304eedd146efb4d3994eb46f Mon Sep 17 00:00:00 2001 From: yuz10 <845238369@qq.com> Date: Sat, 14 Dec 2024 13:26:08 +0800 Subject: [PATCH 2/2] fix --- .../io/openmessaging/storage/dledger/DLedgerLeaderElector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java index 516e018d..8360371a 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java @@ -192,7 +192,7 @@ public void changeRoleToFollower(long term, String leaderId) { handleRoleChange(term, MemberState.Role.FOLLOWER); } - public CompletableFuture handleVote(VoteRequest request, boolean self) { + public CompletableFuture handleVote(VoteRequest request, boolean self) { //hold the lock to get the latest term, leaderId, ledgerEndIndex synchronized (memberState) { if (!memberState.isPeerMember(request.getLeaderId())) {