You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sz...@apache.org on 2022/02/26 05:26:26 UTC

[ratis] branch master updated: RATIS-1535. Add a command to step down the leader (#610)

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

szetszwo 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 c37cc56  RATIS-1535. Add a command to step down the leader (#610)
c37cc56 is described below

commit c37cc56c6895762efcf643cd5b17d7a8c811f201
Author: Yaolong Liu <ly...@163.com>
AuthorDate: Sat Feb 26 13:26:20 2022 +0800

    RATIS-1535. Add a command to step down the leader (#610)
---
 .../cli/sh/command/AbstractParentCommand.java      |  6 +-
 .../shell/cli/sh/command/ElectionCommand.java      |  3 +-
 .../shell/cli/sh/election/StepDownCommand.java     | 85 ++++++++++++++++++++++
 3 files changed, 92 insertions(+), 2 deletions(-)

diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractParentCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractParentCommand.java
index 2129d18..01428df 100644
--- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractParentCommand.java
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractParentCommand.java
@@ -20,6 +20,7 @@ package org.apache.ratis.shell.cli.sh.command;
 import org.apache.ratis.shell.cli.Command;
 
 import java.util.Collections;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.function.Function;
@@ -32,7 +33,10 @@ public abstract class AbstractParentCommand extends AbstractRatisCommand{
     super(context);
     this.subs = Collections.unmodifiableMap(subCommandConstructors.stream()
         .map(constructor -> constructor.apply(context))
-        .collect(Collectors.toMap(Command::getCommandName, Function.identity())));
+        .collect(Collectors.toMap(Command::getCommandName, Function.identity(),
+        (a, b) -> {
+          throw new IllegalStateException("Found duplicated commands: " + a + " and " + b);
+          }, LinkedHashMap::new)));
   }
 
   @Override
diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
index 47ed3b0..43d6846 100644
--- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
@@ -19,6 +19,7 @@ package org.apache.ratis.shell.cli.sh.command;
 
 import org.apache.ratis.shell.cli.sh.election.PauseCommand;
 import org.apache.ratis.shell.cli.sh.election.ResumeCommand;
+import org.apache.ratis.shell.cli.sh.election.StepDownCommand;
 import org.apache.ratis.shell.cli.sh.election.TransferCommand;
 
 import java.util.Arrays;
@@ -29,7 +30,7 @@ import java.util.function.Function;
 public class ElectionCommand extends AbstractParentCommand {
   private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
       = Collections.unmodifiableList(Arrays.asList(
-      TransferCommand::new, PauseCommand::new, ResumeCommand::new));
+      TransferCommand::new, StepDownCommand::new, PauseCommand::new, ResumeCommand::new));
 
   /**
    * @param context command context
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
new file mode 100644
index 0000000..4576634
--- /dev/null
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.shell.cli.sh.election;
+
+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.shell.cli.RaftUtils;
+import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand;
+import org.apache.ratis.shell.cli.sh.command.Context;
+
+import java.io.IOException;
+
+/**
+ * Command for stepping down ratis leader server.
+ */
+public class StepDownCommand extends AbstractRatisCommand {
+
+  /**
+   * @param context command context
+   */
+  public StepDownCommand(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommandName() {
+    return "stepDown";
+  }
+
+  @Override
+  public int run(CommandLine cl) throws IOException {
+    super.run(cl);
+
+    try (RaftClient client = RaftUtils.createClient(getRaftGroup())) {
+      final RaftClientReply transferLeadershipReply = client.admin().transferLeadership(null, 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());
+      return -1;
+    }
+    println("Step down leader successfully");
+    return 0;
+  }
+
+  @Override
+  public String getUsage() {
+    return String.format("%s -%s <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT>"
+            + " [-%s <RAFT_GROUP_ID>]",
+        getCommandName(), PEER_OPTION_NAME, GROUPID_OPTION_NAME);
+  }
+
+  @Override
+  public String getDescription() {
+    return description();
+  }
+
+  @Override
+  public Options getOptions() {
+    return super.getOptions();
+  }
+
+  /**
+   * @return command's description
+   */
+  public static String description() {
+    return "Step down the leader server.";
+  }
+}