You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/02/17 13:24:58 UTC

[GitHub] [ratis] codings-dan opened a new pull request #603: Add ratis-shell pause&resume leader election command

codings-dan opened a new pull request #603:
URL: https://github.com/apache/ratis/pull/603


   ## What changes were proposed in this pull request?
   Add a sub command to pause and resume leader election on the specific peer.
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/RATIS-1526
   
   ## How was this patch tested?
   UT
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] szetszwo commented on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1046947031


   ```
   +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/RatisShell.java
   @@ -72,9 +72,12 @@ public class RatisShell extends AbstractShell {
          // Add commands from <pkgName>.command.*
          if (cls.getPackage().getName().equals(pkgName + ".command")
              && !Modifier.isAbstract(cls.getModifiers())) {
   +        System.out.println("Found " + cls);
            // Only instantiate a concrete class
            final Command cmd = ReflectionUtils.newInstance(cls, classArgs, objectArgs);
            commandsMap.put(cmd.getCommandName(), cmd);
   +      } else {
   +        System.out.println("Ignored " + cls);
          }
        }
   ```
   @codings-dan , if we add the code above, the new subcommnads will be ignored.
   ```
   Ignored class org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand
   Found class org.apache.ratis.shell.cli.sh.command.PeerCommand
   Found class org.apache.ratis.shell.cli.sh.command.ElectionCommand
   Ignored class org.apache.ratis.shell.cli.sh.group.GroupListCommand
   Found class org.apache.ratis.shell.cli.sh.command.SetPriorityCommand
   Ignored class org.apache.ratis.shell.cli.sh.group.GroupInfoCommand
   Ignored class org.apache.ratis.shell.cli.sh.snapshot.TakeSnapshotCommand
   Ignored class org.apache.ratis.shell.cli.sh.election.PauseCommand
   Found class org.apache.ratis.shell.cli.sh.command.SnapshotCommand
   Found class org.apache.ratis.shell.cli.sh.command.GroupCommand
   Ignored class org.apache.ratis.shell.cli.sh.election.TransferCommand
   Ignored class org.apache.ratis.shell.cli.sh.election.ResumeCommand
   Usage: ratis sh [generic options]
   	 [election [resume] [transfer] [pause]]                    
   	 [group [list] [info]]                                     
   	 [peer -peers <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT> [-groupid <RAFT_GROUP_ID>] -remove <PEER_HOST:PEER_PORT> -add <PEER_HOST:PEER_PORT>]
   	 [setPriority -peers <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT> [-groupid <RAFT_GROUP_ID>] -addressPriority <PEER_HOST:PEER_PORT|PRIORITY>]
   	 [snapshot [create]]                                       
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] szetszwo commented on a change in pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #603:
URL: https://github.com/apache/ratis/pull/603#discussion_r810505814



##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/LeaderElectionCommand.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.shell.cli.RaftUtils;
+
+import java.io.IOException;
+import java.util.Optional;
+
+public class LeaderElectionCommand extends AbstractRatisCommand{
+  public static final String ADDRESS_OPTION_NAME = "address";
+  public static final String PAUSE_OPTION_NAME = "pause";
+  public static final String RESUME_OPTION_NAME = "resume";
+
+  /**
+   * @param context command context
+   */
+  public LeaderElectionCommand(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommandName() {
+    return "leaderElection";

Review comment:
       This command and the ElectCommand are closely related.  How about we have a base `election` command and then have the following subcommands?
   - transfer
   - pause
   - resume




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] codings-dan edited a comment on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
codings-dan edited a comment on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1047669741


   @szetszwo  We don't need load classes outside the `org.apache.ratis.shell.cli.sh.command` package, they are all sub-commands, and their baseCommand are in the command package. BaseCommand maintains a map of the mapping relationship between sub-command name and sub-command. See `org.apache.ratis.shell.cli.AbstractShell.java#run`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] codings-dan commented on a change in pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
codings-dan commented on a change in pull request #603:
URL: https://github.com/apache/ratis/pull/603#discussion_r810636771



##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/LeaderElectionCommand.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.shell.cli.RaftUtils;
+
+import java.io.IOException;
+import java.util.Optional;
+
+public class LeaderElectionCommand extends AbstractRatisCommand{
+  public static final String ADDRESS_OPTION_NAME = "address";
+  public static final String PAUSE_OPTION_NAME = "pause";
+  public static final String RESUME_OPTION_NAME = "resume";
+
+  /**
+   * @param context command context
+   */
+  public LeaderElectionCommand(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommandName() {
+    return "leaderElection";

Review comment:
       will do




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] szetszwo commented on a change in pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #603:
URL: https://github.com/apache/ratis/pull/603#discussion_r811890566



##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.shell.cli.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.TransferCommand;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ElectionCommand extends AbstractRatisCommand{
+
+  private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
+      = Collections.unmodifiableList(Arrays.asList(
+      TransferCommand::new, PauseCommand::new, ResumeCommand::new));
+
+  private final Map<String, Command> subs;

Review comment:
       Let's move the common code to a parent class:
   ```
   public abstract class AbstractParentCommand extends AbstractRatisCommand{
     private final Map<String, Command> subs;
   
     public AbstractParentCommand(Context context, List<Function<Context, AbstractRatisCommand>> subCommandConstructors) {
       super(context);
       this.subs = Collections.unmodifiableMap(subCommandConstructors.stream()
           .map(constructor -> constructor.apply(context))
           .collect(Collectors.toMap(Command::getCommandName, Function.identity())));
     }
   
     @Override
     public final Map<String, Command> getSubCommands() {
       return subs;
     }
   
     @Override
     public final String getUsage() {
       final StringBuilder usage = new StringBuilder(getCommandName());
       for (String cmd : getSubCommands().keySet()) {
         usage.append(" [").append(cmd).append("]");
       }
       return usage.toString();
     }
   }
   ```

##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+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;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * Command for resuming leader election on specific server
+ */
+public class ResumeCommand extends AbstractRatisCommand {
+
+  public static final String ADDRESS_OPTION_NAME = "address";
+  /**
+   * @param context command context
+   */
+  public ResumeCommand(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommandName() {
+    return "resume";
+  }
+
+  @Override
+  public int run(CommandLine cl) throws IOException {
+    super.run(cl);
+
+    String strAddr = cl.getOptionValue(ADDRESS_OPTION_NAME);
+    final RaftPeerId peerId;
+    Optional<RaftPeer> peer =
+        getRaftGroup().getPeers().stream().filter(p -> p.getAddress().equals(strAddr)).findAny();
+    if (peer.isPresent()) {
+      peerId = peer.get().getId();
+    } else {

Review comment:
       We may simplify the code as in the PauseCommnad.

##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.shell.cli.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.TransferCommand;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ElectionCommand extends AbstractRatisCommand{
+
+  private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
+      = Collections.unmodifiableList(Arrays.asList(
+      TransferCommand::new, PauseCommand::new, ResumeCommand::new));
+
+  private final Map<String, Command> subs;
+  public static final String ADDRESS_OPTION_NAME = "address";
+
+  /**
+   * @param context command context
+   */
+  public ElectionCommand(Context context) {
+    super(context);
+    this.subs = Collections.unmodifiableMap(SUB_COMMAND_CONSTRUCTORS.stream()
+        .map(constructor -> constructor.apply(context))
+        .collect(Collectors.toMap(Command::getCommandName, Function.identity())));
+  }
+
+  @Override
+  public String getCommandName() {
+    return "election";
+  }
+
+  @Override
+  public String getUsage() {
+
+    StringBuilder usage = new StringBuilder(getCommandName());
+    for (String cmd : subs.keySet()) {
+      usage.append(" [").append(cmd).append("]");
+    }
+    return usage.toString();
+  }
+
+  @Override
+  public String getDescription() {
+    return description();
+  }
+
+  @Override
+  public Map<String, Command> getSubCommands() {
+    return subs;
+  }
+
+  @Override
+  public Options getOptions() {

Review comment:
       Let's remove it.  This is not really used.

##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+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;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * Command for pause leader election on specific server
+ */
+public class PauseCommand extends AbstractRatisCommand {
+
+  public static final String ADDRESS_OPTION_NAME = "address";
+  /**
+   * @param context command context
+   */
+  public PauseCommand(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommandName() {
+    return "pause";
+  }
+
+  @Override
+  public int run(CommandLine cl) throws IOException {
+    super.run(cl);
+
+    String strAddr = cl.getOptionValue(ADDRESS_OPTION_NAME);
+    final RaftPeerId peerId;
+    Optional<RaftPeer> peer =
+        getRaftGroup().getPeers().stream().filter(p -> p.getAddress().equals(strAddr)).findAny();
+    if (peer.isPresent()) {
+      peerId = peer.get().getId();
+    } else {
+      printf("Can't find a sever with the address:%s", strAddr);
+      return -1;
+    }

Review comment:
       The code can be simplified as below.
   ```
       final RaftPeerId peerId = getRaftGroup().getPeers().stream()
           .filter(p -> p.getAddress().equals(strAddr)).findAny()
           .map(RaftPeer::getId)
           .orElse(null);
       if (peerId == null) {
         printf("Peer not found: %s", strAddr);
         return -1;
       }
   ```

##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.shell.cli.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.TransferCommand;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ElectionCommand extends AbstractRatisCommand{
+
+  private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
+      = Collections.unmodifiableList(Arrays.asList(
+      TransferCommand::new, PauseCommand::new, ResumeCommand::new));
+
+  private final Map<String, Command> subs;
+  public static final String ADDRESS_OPTION_NAME = "address";

Review comment:
       Let's remove it.  This is not really used.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] codings-dan commented on a change in pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
codings-dan commented on a change in pull request #603:
URL: https://github.com/apache/ratis/pull/603#discussion_r812859588



##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+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;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * Command for resuming leader election on specific server
+ */
+public class ResumeCommand extends AbstractRatisCommand {
+
+  public static final String ADDRESS_OPTION_NAME = "address";
+  /**
+   * @param context command context
+   */
+  public ResumeCommand(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommandName() {
+    return "resume";
+  }
+
+  @Override
+  public int run(CommandLine cl) throws IOException {
+    super.run(cl);
+
+    String strAddr = cl.getOptionValue(ADDRESS_OPTION_NAME);
+    final RaftPeerId peerId;
+    Optional<RaftPeer> peer =
+        getRaftGroup().getPeers().stream().filter(p -> p.getAddress().equals(strAddr)).findAny();
+    if (peer.isPresent()) {
+      peerId = peer.get().getId();
+    } else {

Review comment:
       done

##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+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;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * Command for pause leader election on specific server
+ */
+public class PauseCommand extends AbstractRatisCommand {
+
+  public static final String ADDRESS_OPTION_NAME = "address";
+  /**
+   * @param context command context
+   */
+  public PauseCommand(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommandName() {
+    return "pause";
+  }
+
+  @Override
+  public int run(CommandLine cl) throws IOException {
+    super.run(cl);
+
+    String strAddr = cl.getOptionValue(ADDRESS_OPTION_NAME);
+    final RaftPeerId peerId;
+    Optional<RaftPeer> peer =
+        getRaftGroup().getPeers().stream().filter(p -> p.getAddress().equals(strAddr)).findAny();
+    if (peer.isPresent()) {
+      peerId = peer.get().getId();
+    } else {
+      printf("Can't find a sever with the address:%s", strAddr);
+      return -1;
+    }

Review comment:
       done

##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.shell.cli.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.TransferCommand;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ElectionCommand extends AbstractRatisCommand{
+
+  private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
+      = Collections.unmodifiableList(Arrays.asList(
+      TransferCommand::new, PauseCommand::new, ResumeCommand::new));
+
+  private final Map<String, Command> subs;
+  public static final String ADDRESS_OPTION_NAME = "address";
+
+  /**
+   * @param context command context
+   */
+  public ElectionCommand(Context context) {
+    super(context);
+    this.subs = Collections.unmodifiableMap(SUB_COMMAND_CONSTRUCTORS.stream()
+        .map(constructor -> constructor.apply(context))
+        .collect(Collectors.toMap(Command::getCommandName, Function.identity())));
+  }
+
+  @Override
+  public String getCommandName() {
+    return "election";
+  }
+
+  @Override
+  public String getUsage() {
+
+    StringBuilder usage = new StringBuilder(getCommandName());
+    for (String cmd : subs.keySet()) {
+      usage.append(" [").append(cmd).append("]");
+    }
+    return usage.toString();
+  }
+
+  @Override
+  public String getDescription() {
+    return description();
+  }
+
+  @Override
+  public Map<String, Command> getSubCommands() {
+    return subs;
+  }
+
+  @Override
+  public Options getOptions() {

Review comment:
       done

##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.shell.cli.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.TransferCommand;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ElectionCommand extends AbstractRatisCommand{
+
+  private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
+      = Collections.unmodifiableList(Arrays.asList(
+      TransferCommand::new, PauseCommand::new, ResumeCommand::new));
+
+  private final Map<String, Command> subs;
+  public static final String ADDRESS_OPTION_NAME = "address";

Review comment:
       done

##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.shell.cli.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.TransferCommand;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ElectionCommand extends AbstractRatisCommand{
+
+  private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
+      = Collections.unmodifiableList(Arrays.asList(
+      TransferCommand::new, PauseCommand::new, ResumeCommand::new));
+
+  private final Map<String, Command> subs;

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] szetszwo commented on a change in pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #603:
URL: https://github.com/apache/ratis/pull/603#discussion_r811181167



##########
File path: ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/ElectionCommand.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.command;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.ratis.shell.cli.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.TransferCommand;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ElectionCommand extends AbstractRatisCommand{
+
+  private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
+      = Collections.unmodifiableList(Arrays.asList(
+      TransferCommand::new, PauseCommand::new, ResumeCommand::new));
+
+  private final Map<String, Command> subs;
+  public static final String ADDRESS_OPTION_NAME = "address";
+
+  /**
+   * @param context command context
+   */
+  public ElectionCommand(Context context) {
+    super(context);
+    this.subs = Collections.unmodifiableMap(SUB_COMMAND_CONSTRUCTORS.stream()
+        .map(constructor -> constructor.apply(context))
+        .collect(Collectors.toMap(Command::getCommandName, Function.identity())));
+  }
+
+  @Override
+  public String getCommandName() {
+    return "election";
+  }
+
+  @Override
+  public String getUsage() {
+
+    StringBuilder usage = new StringBuilder(getCommandName());
+    for (String cmd : subs.keySet()) {
+      usage.append(" [").append(cmd).append("]");
+    }
+    return usage.toString();
+  }

Review comment:
       If we add the following code,
   ```
   +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/RatisShell.java
   @@ -39,6 +39,9 @@ public class RatisShell extends AbstractShell {
       */
      public static void main(String[] args) {
        RatisShell extensionShell = new RatisShell();
   +    if (args.length == 0) {
   +      extensionShell.printUsage();
   +    }
        System.exit(extensionShell.run(args));
      }
   ```
   it will print out `[election [resume] [transfer] [pause]] ` under `[generic options]` as shown below.
   ```
   Usage: ratis sh [generic options]
   	 [election [resume] [transfer] [pause]]                    
   	 [group [list] [info]]                                     
   	 [peer -peers <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT> [-groupid <RAFT_GROUP_ID>] -remove <PEER_HOST:PEER_PORT> -add <PEER_HOST:PEER_PORT>]
   	 [setPriority -peers <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT> [-groupid <RAFT_GROUP_ID>] -addressPriority <PEER_HOST:PEER_PORT|PRIORITY>]
   	 [snapshot [create]]                                       
   Usage: ratis sh [generic options]
   	 [election [resume] [transfer] [pause]]                    
   	 [group [list] [info]]                                     
   	 [peer -peers <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT> [-groupid <RAFT_GROUP_ID>] -remove <PEER_HOST:PEER_PORT> -add <PEER_HOST:PEER_PORT>]
   	 [setPriority -peers <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT> [-groupid <RAFT_GROUP_ID>] -addressPriority <PEER_HOST:PEER_PORT|PRIORITY>]
   	 [snapshot [create]]                                       
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] codings-dan commented on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
codings-dan commented on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1047669741


   @szetszwo  We don't need load classes outside the `org.apache.ratis.shell.cli.sh.command` package, they are all sub-commands, and their baseCommand are in the command package. BaseCommand maintains a map of the mapping relationship between sub-command name and sub-command.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] codings-dan commented on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
codings-dan commented on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1046811836


   @szetszwo I did code splitting related to election command, PTAL, thx!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] szetszwo commented on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1047597602


   > Do you mean we add these lines of code and then display this information during compilation?
   
   @codings-dan, The code lines are for printing debug messages.  Currently, the debug messages show that many of the command classes are ignored since they are not under the `org.apache.ratis.shell.cli.sh.command` package.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] szetszwo commented on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1047735266


   @codings-dan , I see.  The print out is a little bit confusing.  Thanks for explaining it.
   
   I suggest that we move the setPriority command as a subcommand of the peer command and change the peer command with the following subcommands.
   - add
   - remove
   - setPriority
   
   See if you are interested working on it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] codings-dan commented on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
codings-dan commented on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1048752317


   > @codings-dan , I see. The print out is a little bit confusing. Thanks for explaining it.
   > 
   > I suggest that we move the setPriority command as a subcommand of the peer command and change the peer command with the following subcommands.
   > 
   > * add
   > * remove
   > * setPriority
   > 
   > See if you are interested working on it.
   
   @szetszwo I have refactored the code, PTAL, thx!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] codings-dan commented on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
codings-dan commented on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1047354253


   > ```
   > +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/RatisShell.java
   > @@ -72,9 +72,12 @@ public class RatisShell extends AbstractShell {
   >        // Add commands from <pkgName>.command.*
   >        if (cls.getPackage().getName().equals(pkgName + ".command")
   >            && !Modifier.isAbstract(cls.getModifiers())) {
   > +        System.out.println("Found " + cls);
   >          // Only instantiate a concrete class
   >          final Command cmd = ReflectionUtils.newInstance(cls, classArgs, objectArgs);
   >          commandsMap.put(cmd.getCommandName(), cmd);
   > +      } else {
   > +        System.out.println("Ignored " + cls);
   >        }
   >      }
   > ```
   > 
   > @codings-dan , if we add the code above, the new subcommnads will be ignored.
   > 
   > ```
   > Ignored class org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand
   > Found class org.apache.ratis.shell.cli.sh.command.PeerCommand
   > Found class org.apache.ratis.shell.cli.sh.command.ElectionCommand
   > Ignored class org.apache.ratis.shell.cli.sh.group.GroupListCommand
   > Found class org.apache.ratis.shell.cli.sh.command.SetPriorityCommand
   > Ignored class org.apache.ratis.shell.cli.sh.group.GroupInfoCommand
   > Ignored class org.apache.ratis.shell.cli.sh.snapshot.TakeSnapshotCommand
   > Ignored class org.apache.ratis.shell.cli.sh.election.PauseCommand
   > Found class org.apache.ratis.shell.cli.sh.command.SnapshotCommand
   > Found class org.apache.ratis.shell.cli.sh.command.GroupCommand
   > Ignored class org.apache.ratis.shell.cli.sh.election.TransferCommand
   > Ignored class org.apache.ratis.shell.cli.sh.election.ResumeCommand
   > Usage: ratis sh [generic options]
   > 	 [election [resume] [transfer] [pause]]                    
   > 	 [group [list] [info]]                                     
   > 	 [peer -peers <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT> [-groupid <RAFT_GROUP_ID>] -remove <PEER_HOST:PEER_PORT> -add <PEER_HOST:PEER_PORT>]
   > 	 [setPriority -peers <PEER0_HOST:PEER0_PORT,PEER1_HOST:PEER1_PORT,PEER2_HOST:PEER2_PORT> [-groupid <RAFT_GROUP_ID>] -addressPriority <PEER_HOST:PEER_PORT|PRIORITY>]
   > 	 [snapshot [create]]                                       
   > ```
   
   Do you mean we add these lines of code and then display this information during compilation?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] codings-dan commented on pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
codings-dan commented on pull request #603:
URL: https://github.com/apache/ratis/pull/603#issuecomment-1047740004


   > @codings-dan , I see. The print out is a little bit confusing. Thanks for explaining it.
   > 
   > I suggest that we move the setPriority command as a subcommand of the peer command and change the peer command with the following subcommands.
   > 
   > * add
   > * remove
   > * setPriority
   > 
   > See if you are interested working on it.
   
   Good idea, since this change has nothing to do with this jira, I will complete the code change in the future.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ratis] szetszwo merged pull request #603: RATIS-1526. Add ratis-shell pause&resume leader election command

Posted by GitBox <gi...@apache.org>.
szetszwo merged pull request #603:
URL: https://github.com/apache/ratis/pull/603


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org