You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by dr...@apache.org on 2023/01/13 09:36:09 UTC

[ratis] branch master updated: RATIS-1768. Fix stepDown command don't work issue (#806)

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

dragonyliu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new 71a9fdffa RATIS-1768. Fix stepDown command don't work issue (#806)
71a9fdffa is described below

commit 71a9fdffa9ab49d6f4f39402ff3873423021c107
Author: Yaolong Liu <ly...@163.com>
AuthorDate: Fri Jan 13 17:36:03 2023 +0800

    RATIS-1768. Fix stepDown command don't work issue (#806)
    
    * Fix stepDown command don't work issue
    
    * fix checkstyle issue
    
    * address review comment
    
    Co-authored-by: dragonyliu <dr...@tencent.com>
---
 .../java/org/apache/ratis/client/api/AdminApi.java     |  6 +++++-
 .../java/org/apache/ratis/client/impl/AdminImpl.java   |  6 ++++--
 .../org/apache/ratis/server/impl/RaftServerImpl.java   |  3 ++-
 .../ratis/shell/cli/sh/election/StepDownCommand.java   |  4 +++-
 .../shell/cli/sh/ElectionCommandIntegrationTest.java   | 18 ++++++++++++++++++
 5 files changed, 32 insertions(+), 5 deletions(-)

diff --git a/ratis-client/src/main/java/org/apache/ratis/client/api/AdminApi.java b/ratis-client/src/main/java/org/apache/ratis/client/api/AdminApi.java
index 86e25ef9a..e27c8ae86 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/api/AdminApi.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/api/AdminApi.java
@@ -66,5 +66,9 @@ public interface AdminApi {
   }
 
   /** Transfer leadership to the given server.*/
-  RaftClientReply transferLeadership(RaftPeerId newLeader, long timeoutMs) throws IOException;
+  default RaftClientReply transferLeadership(RaftPeerId newLeader, long timeoutMs) throws IOException {
+    return transferLeadership(newLeader, null, timeoutMs);
+  };
+
+  RaftClientReply transferLeadership(RaftPeerId newLeader, RaftPeerId leaderId, long timeoutMs) throws IOException;
 }
\ No newline at end of file
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/AdminImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/AdminImpl.java
index 445515430..b9e9968a6 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/impl/AdminImpl.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/AdminImpl.java
@@ -48,9 +48,11 @@ class AdminImpl implements AdminApi {
   }
 
   @Override
-  public RaftClientReply transferLeadership(RaftPeerId newLeader, long timeoutMs) throws IOException {
+  public RaftClientReply transferLeadership(
+      RaftPeerId newLeader, RaftPeerId leaderId, long timeoutMs) throws IOException {
     final long callId = CallId.getAndIncrement();
     return client.io().sendRequestWithRetry(() -> new TransferLeadershipRequest(
-        client.getId(), client.getLeaderId(), client.getGroupId(), callId, newLeader, timeoutMs));
+        client.getId(), leaderId == null ? client.getLeaderId() : leaderId,
+        client.getGroupId(), callId, newLeader, timeoutMs));
   }
 }
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 d04f3126d..210d0f18c 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
@@ -1177,7 +1177,8 @@ class RaftServerImpl implements RaftServer.Division,
     assertGroup(request.getRequestorId(), request.getRaftGroupId());
 
     return role.getLeaderState().map(leader -> leader.submitStepDownRequestAsync(request))
-        .orElseGet(() -> CompletableFuture.completedFuture(newSuccessReply(request)));
+        .orElseGet(() -> CompletableFuture.completedFuture(
+            newExceptionReply(request, generateNotLeaderException())));
   }
 
   public RaftClientReply setConfiguration(SetConfigurationRequest request) throws IOException {
diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java
index 457663456..50abfe37c 100644
--- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java
@@ -21,6 +21,7 @@ import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Options;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.shell.cli.RaftUtils;
 import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand;
 import org.apache.ratis.shell.cli.sh.command.Context;
@@ -49,7 +50,8 @@ public class StepDownCommand extends AbstractRatisCommand {
     super.run(cl);
 
     try (RaftClient client = RaftUtils.createClient(getRaftGroup())) {
-      final RaftClientReply transferLeadershipReply = client.admin().transferLeadership(null, 60_000);
+      RaftPeerId leaderId = RaftPeerId.valueOf(getLeader(getGroupInfoReply().getRoleInfoProto()).getId());
+      final RaftClientReply transferLeadershipReply = client.admin().transferLeadership(null, leaderId, 60_000);
       processReply(transferLeadershipReply, () -> "Failed to step down leader");
     } catch (Throwable t) {
       printf("caught an error when executing step down leader: %s%n", t.getMessage());
diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java
index f3ea7c3ab..cea1b17e2 100644
--- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java
+++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java
@@ -105,4 +105,22 @@ public abstract class ElectionCommandIntegrationTest <CLUSTER extends MiniRaftCl
       Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId());
     }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionPauseResumeCommand", LOG);
   }
+
+  @Test
+  public void testElectionStepDownCommand() throws Exception {
+    runWithNewCluster(NUM_SERVERS, this::runTestElectionStepDownCommand);
+  }
+
+  void runTestElectionStepDownCommand(MiniRaftCluster cluster) throws Exception {
+    final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster);
+    String sb = getClusterAddress(cluster);
+    RaftServer.Division newLeader = cluster.getFollowers().get(0);
+    final StringPrintStream out = new StringPrintStream();
+    RatisShell shell = new RatisShell(out.getPrintStream());
+    Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId());
+    Assert.assertEquals(2, cluster.getFollowers().size());
+    int ret = shell.run("election", "stepDown", "-peers", sb.toString());
+    Assert.assertEquals(0, ret);
+    Assert.assertEquals(3, cluster.getFollowers().size());
+  }
 }