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 2021/12/03 16:12:58 UTC

[ratis] branch master updated: RATIS-1436.Add ratis-shell GroupInfo command (#540)

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 1f4ec88  RATIS-1436.Add ratis-shell GroupInfo command (#540)
1f4ec88 is described below

commit 1f4ec88041856c5e92a3f6f2e27a2b180c8403e3
Author: Yaolong Liu <ly...@163.com>
AuthorDate: Sat Dec 4 00:12:52 2021 +0800

    RATIS-1436.Add ratis-shell GroupInfo command (#540)
---
 .../java/org/apache/ratis/shell/cli/Command.java   |   3 +-
 .../shell/cli/sh/command/AbstractRatisCommand.java |  19 +++-
 .../ratis/shell/cli/sh/command/GroupCommand.java   | 115 ++++++---------------
 .../{GroupCommand.java => PeerCommand.java}        |  17 +--
 .../GroupInfoCommand.java}                         |   9 +-
 .../ratis/shell/cli/sh/group/GroupListCommand.java |  99 ++++++++++++++++++
 6 files changed, 156 insertions(+), 106 deletions(-)

diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/Command.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/Command.java
index 31b8688..606e55c 100644
--- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/Command.java
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/Command.java
@@ -27,6 +27,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
+import java.util.Optional;
 
 /**
  * An interface for all the commands that can be run from a shell.
@@ -54,7 +55,7 @@ public interface Command extends Closeable {
    * @return whether this command has sub-commands
    */
   default boolean hasSubCommand() {
-    return false;
+    return Optional.ofNullable(getSubCommands()).filter(subs -> !subs.isEmpty()).isPresent();
   }
 
   /**
diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java
index 6c8c8b6..c2b9e4d 100644
--- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java
@@ -46,6 +46,18 @@ public abstract class AbstractRatisCommand implements Command {
   public static final String GROUPID_OPTION_NAME = "groupid";
   public static final RaftGroupId DEFAULT_RAFT_GROUP_ID = RaftGroupId.randomId();
 
+  public static InetSocketAddress parseInetSocketAddress(String address) {
+    try {
+      final String[] hostPortPair = address.split(":");
+      if (hostPortPair.length < 2) {
+        throw new IllegalArgumentException("Unexpected address format <HOST:PORT>.");
+      }
+      return new InetSocketAddress(hostPortPair[0], Integer.parseInt(hostPortPair[1]));
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Failed to parse the server address parameter \"" + address + "\".", e);
+    }
+  }
+
   /**
    * Execute a given function with input parameter from the members of a list.
    *
@@ -83,11 +95,8 @@ public abstract class AbstractRatisCommand implements Command {
     List<InetSocketAddress> addresses = new ArrayList<>();
     String peersStr = cl.getOptionValue(PEER_OPTION_NAME);
     String[] peersArray = peersStr.split(",");
-    for (int i = 0; i < peersArray.length; i++) {
-      String[] hostPortPair = peersArray[i].split(":");
-      InetSocketAddress addr =
-          new InetSocketAddress(hostPortPair[0], Integer.parseInt(hostPortPair[1]));
-      addresses.add(addr);
+    for (String peer : peersArray) {
+      addresses.add(parseInetSocketAddress(peer));
     }
 
     final RaftGroupId raftGroupIdFromConfig = cl.hasOption(GROUPID_OPTION_NAME)?
diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/GroupCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/GroupCommand.java
index 7daff2b..dd87dc0 100644
--- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/GroupCommand.java
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/GroupCommand.java
@@ -17,38 +17,38 @@
  */
 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 org.apache.ratis.shell.cli.Command;
+import org.apache.ratis.shell.cli.sh.group.GroupInfoCommand;
+import org.apache.ratis.shell.cli.sh.group.GroupListCommand;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.function.BiConsumer;
+import java.util.function.Function;
 import java.util.stream.Collectors;
-import java.util.stream.Stream;
 
 /**
- * Command for remove and add ratis server.
+ * Command for the ratis group
  */
 public class GroupCommand extends AbstractRatisCommand {
-  public static final String REMOVE_OPTION_NAME = "remove";
-  public static final String ADD_OPTION_NAME = "add";
+
+  private static final List<Function<Context, AbstractRatisCommand>> SUB_COMMAND_CONSTRUCTORS
+          = Collections.unmodifiableList(Arrays.asList(
+          GroupInfoCommand::new, GroupListCommand::new));
+
+  private final Map<String, Command> subs;
 
   /**
    * @param context command context
    */
   public GroupCommand(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
@@ -57,91 +57,40 @@ public class GroupCommand extends AbstractRatisCommand {
   }
 
   @Override
-  public int run(CommandLine cl) throws IOException {
-    super.run(cl);
-    final Map<RaftPeerId, InetSocketAddress> peersInfo = new HashMap<>();
-    final List<RaftPeerId> toRemove = getIds(cl.getOptionValues(REMOVE_OPTION_NAME), (a, b) -> {});
-    final List<RaftPeerId> toAdd = getIds(cl.getOptionValues(ADD_OPTION_NAME), peersInfo::put);
-    if (toRemove.isEmpty() && toAdd.isEmpty()) {
-      throw new IllegalArgumentException(String.format("Both -%s and -%s options are empty",
-          REMOVE_OPTION_NAME, ADD_OPTION_NAME));
-    }
-
-    try (RaftClient client = RaftUtils.createClient(getRaftGroup())) {
-      final Stream<RaftPeer> remaining = getRaftGroup().getPeers().stream()
-          .filter(raftPeer -> !toRemove.contains(raftPeer.getId()))
-          .filter(raftPeer -> !toAdd.contains(raftPeer.getId()));
-      final Stream<RaftPeer> adding = toAdd.stream().map(raftPeerId -> RaftPeer.newBuilder()
-          .setId(raftPeerId)
-          .setAddress(peersInfo.get(raftPeerId))
-          .setPriority(0)
-          .build());
-      final List<RaftPeer> peers = Stream.concat(remaining, adding).collect(Collectors.toList());
-      System.out.println("New peer list: " + peers);
-      RaftClientReply reply = client.admin().setConfiguration(peers);
-      processReply(reply, () -> "failed to change raft peer");
-    }
-    return 0;
-  }
-
-  private static List<RaftPeerId> getIds(String[] optionValues, BiConsumer<RaftPeerId, InetSocketAddress> consumer) {
-    if (optionValues == null) {
-      return Collections.emptyList();
-    }
-    final List<RaftPeerId> ids = new ArrayList<>();
-    for (String address : optionValues) {
-      final String[] str = parse(address);
-      final InetSocketAddress serverAddress = InetSocketAddress.createUnresolved(str[0], Integer.parseInt(str[1]));
-      final RaftPeerId peerId = RaftUtils.getPeerId(serverAddress);
-      consumer.accept(peerId, serverAddress);
-      ids.add(peerId);
-    }
-    return ids;
-  }
+  public String getUsage() {
 
-  private static String[] parse(String address) {
-    String[] str = address.split(":");
-    if(str.length < 2) {
-      throw new IllegalArgumentException("Failed to parse the address parameter \"" + address + "\".");
+    StringBuilder usage = new StringBuilder(getCommandName());
+    for (String cmd : subs.keySet()) {
+      usage.append(" [").append(cmd).append("]");
     }
-    return str;
+    return usage.toString();
   }
 
   @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>]"
-                    + " -%s <PEER_HOST:PEER_PORT>"
-                    + " -%s <PEER_HOST:PEER_PORT>",
-            getCommandName(), PEER_OPTION_NAME, GROUPID_OPTION_NAME,
-            REMOVE_OPTION_NAME, ADD_OPTION_NAME);
+  public String getDescription() {
+    return description();
   }
 
   @Override
-  public String getDescription() {
-    return description();
+  public Map<String, Command> getSubCommands() {
+    return subs;
   }
 
   @Override
   public Options getOptions() {
-    return super.getOptions()
-            .addOption(Option.builder()
-                    .option(REMOVE_OPTION_NAME)
-                    .hasArg()
-                    .desc("peer address to be removed")
-                    .build())
-            .addOption(Option.builder()
-                    .option(ADD_OPTION_NAME)
-                    .hasArg()
-                    .desc("peer address to be added")
-                    .build());
+    return super.getOptions().addOption(
+        Option.builder()
+            .option(GROUPID_OPTION_NAME)
+            .hasArg()
+            .required()
+            .desc("the group id")
+            .build());
   }
 
   /**
    * @return command's description
    */
   public static String description() {
-    return "Remove or Add peers of a ratis group";
+    return "Manage ratis groups; see the sub-commands for the details.";
   }
 }
diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/GroupCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/PeerCommand.java
similarity index 90%
copy from ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/GroupCommand.java
copy to ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/PeerCommand.java
index 7daff2b..6bd59e9 100644
--- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/GroupCommand.java
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/PeerCommand.java
@@ -40,20 +40,20 @@ import java.util.stream.Stream;
 /**
  * Command for remove and add ratis server.
  */
-public class GroupCommand extends AbstractRatisCommand {
+public class PeerCommand extends AbstractRatisCommand {
   public static final String REMOVE_OPTION_NAME = "remove";
   public static final String ADD_OPTION_NAME = "add";
 
   /**
    * @param context command context
    */
-  public GroupCommand(Context context) {
+  public PeerCommand(Context context) {
     super(context);
   }
 
   @Override
   public String getCommandName() {
-    return "group";
+    return "peer";
   }
 
   @Override
@@ -90,8 +90,7 @@ public class GroupCommand extends AbstractRatisCommand {
     }
     final List<RaftPeerId> ids = new ArrayList<>();
     for (String address : optionValues) {
-      final String[] str = parse(address);
-      final InetSocketAddress serverAddress = InetSocketAddress.createUnresolved(str[0], Integer.parseInt(str[1]));
+      final InetSocketAddress serverAddress = parseInetSocketAddress(address);
       final RaftPeerId peerId = RaftUtils.getPeerId(serverAddress);
       consumer.accept(peerId, serverAddress);
       ids.add(peerId);
@@ -99,14 +98,6 @@ public class GroupCommand extends AbstractRatisCommand {
     return ids;
   }
 
-  private static String[] parse(String address) {
-    String[] str = address.split(":");
-    if(str.length < 2) {
-      throw new IllegalArgumentException("Failed to parse the address parameter \"" + address + "\".");
-    }
-    return str;
-  }
-
   @Override
   public String getUsage() {
     return String.format("%s"
diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/InfoCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java
similarity index 89%
rename from ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/InfoCommand.java
rename to ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java
index d9ab19c..d2c4e65 100644
--- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/InfoCommand.java
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ratis.shell.cli.sh.command;
+package org.apache.ratis.shell.cli.sh.group;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Options;
 import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.protocol.GroupInfoReply;
+import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand;
+import org.apache.ratis.shell.cli.sh.command.Context;
 
 import java.io.IOException;
 
 /**
  * Command for querying ratis group information.
  */
-public class InfoCommand extends AbstractRatisCommand {
-
+public class GroupInfoCommand extends AbstractRatisCommand {
   /**
    * @param context command context
    */
-  public InfoCommand(Context context) {
+  public GroupInfoCommand(Context context) {
     super(context);
   }
 
diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java
new file mode 100644
index 0000000..53db52c
--- /dev/null
+++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java
@@ -0,0 +1,99 @@
+/*
+ * 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.group;
+
+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.GroupListReply;
+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.net.InetSocketAddress;
+
+/**
+ * Command for querying the group information of a ratis server.
+ */
+public class GroupListCommand extends AbstractRatisCommand {
+  public static final String SERVER_ADDRESS_OPTION_NAME = "serverAddress";
+
+  /**
+   * @param context command context
+   */
+  public GroupListCommand(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommandName() {
+    return "list";
+  }
+
+  @Override
+  public int run(CommandLine cl) throws IOException {
+    super.run(cl);
+    String strAddr = cl.getOptionValue(SERVER_ADDRESS_OPTION_NAME);
+    final InetSocketAddress serverAddress = parseInetSocketAddress(strAddr);
+    final RaftPeerId peerId = RaftUtils.getPeerId(serverAddress);
+
+    try(final RaftClient raftClient = RaftUtils.createClient(getRaftGroup())) {
+      GroupListReply reply = raftClient.getGroupManagementApi(peerId).list();
+      processReply(reply, () -> String.format("Failed to get group information of server %s", strAddr));
+      printf(String.format("The server %s is in %d groups, and the groupIds is: %s",
+              strAddr, reply.getGroupIds().size(), reply.getGroupIds()));
+    }
+    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>]"
+                    + "-%s <PEER0_HOST:PEER0_PORT>",
+            getCommandName(), PEER_OPTION_NAME, GROUPID_OPTION_NAME, SERVER_ADDRESS_OPTION_NAME);
+  }
+
+  @Override
+  public String getDescription() {
+    return description();
+  }
+
+  @Override
+  public Options getOptions() {
+    return super.getOptions()
+            .addOption(Option.builder()
+                    .option(SERVER_ADDRESS_OPTION_NAME)
+                    .hasArg()
+                    .required()
+                    .desc("the server address")
+                    .build());
+  }
+
+  /**
+   * @return command's description
+   */
+  public static String description() {
+    return "Display the group information of a specific raft server";
+  }
+}