This is an automated email from the ASF dual-hosted git repository.

tanxinyu pushed a commit to branch release-3.1.1
in repository https://gitbox.apache.org/repos/asf/ratis.git

commit 1e10b7186fa7f4bea695cab01c82579af8fee66b
Author: 133tosakarin <[email protected]>
AuthorDate: Sat Sep 7 10:56:42 2024 +0800

    RATIS-2149.Do not perform leader election if the current RaftServer has not 
started yet (#1143)
---
 .../src/main/java/org/apache/ratis/server/impl/LeaderElection.java    | 4 ++++
 .../src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java    | 4 ++++
 2 files changed, 8 insertions(+)

diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
index da9c51348..af25ae912 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
@@ -241,6 +241,10 @@ class LeaderElection implements Runnable {
     }
 
     try (AutoCloseable ignored = 
Timekeeper.start(server.getLeaderElectionMetrics().getLeaderElectionTimer())) {
+      if (!server.isRunning()) {
+        LOG.info("{}: skip since the server is not running", this);
+        return;
+      }
       for (int round = 0; shouldRun(); round++) {
         if (skipPreVote || askForVotes(Phase.PRE_VOTE, round)) {
           if (askForVotes(Phase.ELECTION, round)) {
diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index c682de8bf..a66c8ec5f 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -1859,4 +1859,8 @@ class RaftServerImpl implements RaftServer.Division,
   void onGroupLeaderElected() {
     transferLeadership.complete(TransferLeadership.Result.SUCCESS);
   }
+
+  boolean isRunning() {
+    return startComplete.get() && lifeCycle.getCurrentState() == State.RUNNING;
+  }
 }

Reply via email to