You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2018/06/05 21:40:03 UTC

[GitHub] sijie closed pull request #1478: Update the new bookkeeper tools to use the new framework

sijie closed pull request #1478: Update the new bookkeeper tools to use the new framework
URL: https://github.com/apache/bookkeeper/pull/1478
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/bin/bookkeeper-cli b/bin/bkctl
similarity index 83%
rename from bin/bookkeeper-cli
rename to bin/bkctl
index 2e53be057..1cb735716 100755
--- a/bin/bookkeeper-cli
+++ b/bin/bkctl
@@ -26,7 +26,7 @@ BK_HOME=`cd ${BINDIR}/..;pwd`
 source ${BK_HOME}/bin/common.sh
 source ${BK_HOME}/conf/bk_cli_env.sh
 
-CLI_MODULE_PATH=bookkeeper-tools
+CLI_MODULE_PATH=tools/all
 CLI_MODULE_NAME="(org.apache.bookkeeper-)?bookkeeper-tools"
 CLI_MODULE_HOME=${BK_HOME}/${CLI_MODULE_PATH}
 
@@ -46,8 +46,8 @@ if [ -z "${CLI_LOG_CONF}" ]; then
   CLI_LOG_CONF=${DEFAULT_LOG_CONF}
 fi
 CLI_LOG_DIR=${CLI_LOG_DIR:-"$BK_HOME/logs"}
-CLI_LOG_FILE=${CLI_LOG_FILE:-"bookkeeper-cli.log"}
-CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,CONSOLE"}
+CLI_LOG_FILE=${CLI_LOG_FILE:-"bkctl.log"}
+CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,ROLLINGFILE"}
 
 # Configure the classpath
 CLI_CLASSPATH="$CLI_JAR:$CLI_CLASSPATH:$CLI_EXTRA_CLASSPATH"
@@ -55,12 +55,12 @@ CLI_CLASSPATH="`dirname $CLI_LOG_CONF`:$CLI_CLASSPATH"
 
 # Build the OPTs
 BOOKIE_OPTS=$(build_bookie_opts)
-GC_OPTS=$(build_cli_jvm_opts ${CLI_LOG_DIR} "bookkeeper-cli-gc.log")
+GC_OPTS=$(build_cli_jvm_opts ${CLI_LOG_DIR} "bkctl-gc.log")
 NETTY_OPTS=$(build_netty_opts)
-LOGGING_OPTS=$(build_logging_opts ${CLI_LOG_CONF} ${CLI_LOG_DIR} ${CLI_LOG_FILE} ${CLI_ROOT_LOGGER})
+LOGGING_OPTS=$(build_cli_logging_opts ${CLI_LOG_CONF} ${CLI_LOG_DIR} ${CLI_LOG_FILE} ${CLI_ROOT_LOGGER})
 
 OPTS="${OPTS} -cp ${CLI_CLASSPATH} ${BOOKIE_OPTS} ${GC_OPTS} ${NETTY_OPTS} ${LOGGING_OPTS} ${CLI_EXTRA_OPTS}"
 
 #Change to BK_HOME to support relative paths
 cd "$BK_HOME"
-exec ${JAVA} ${OPTS} org.apache.bookkeeper.tools.cli.BookKeeperCLI --conf ${CLI_CONF} $@
+exec ${JAVA} ${OPTS} org.apache.bookkeeper.tools.cli.BKCtl --conf ${CLI_CONF} $@
diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml
index d90f6bbae..31a8bd8fb 100644
--- a/bookkeeper-server/pom.xml
+++ b/bookkeeper-server/pom.xml
@@ -42,6 +42,11 @@
       <artifactId>bookkeeper-proto</artifactId>
       <version>${project.parent.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-tools-framework</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
     <dependency>
       <groupId>org.rocksdb</groupId>
       <artifactId>rocksdbjni</artifactId>
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
index 6dc2883ff..d500e710f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
@@ -107,8 +107,9 @@
 import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
+import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand;
 import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand;
-import org.apache.bookkeeper.tools.cli.commands.cluster.ListBookiesCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
 import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.util.EntryFormatter;
@@ -1128,13 +1129,15 @@ public int runCmd(CommandLine cmdLine) throws Exception {
             int ackQuorum = getOptionIntValue(cmdLine, "ackQuorum", 2);
             int numEntries = getOptionIntValue(cmdLine, "numEntries", 1000);
 
-            SimpleTestCommand command = new SimpleTestCommand()
+            SimpleTestCommand.Flags flags = new SimpleTestCommand.Flags()
                 .ensembleSize(ensemble)
                 .writeQuorumSize(writeQuorum)
                 .ackQuorumSize(ackQuorum)
                 .numEntries(numEntries);
 
-            command.run(bkConf);
+            SimpleTestCommand command = new SimpleTestCommand(flags);
+
+            command.apply(bkConf, flags);
             return 0;
         }
 
@@ -1482,7 +1485,7 @@ Options getOptions() {
         @Override
         public int runCmd(CommandLine c) throws Exception {
             LastMarkCommand command = new LastMarkCommand();
-            command.run(bkConf);
+            command.apply(bkConf, new CliFlags());
             return 0;
         }
 
@@ -1527,10 +1530,13 @@ public int runCmd(CommandLine cmdLine) throws Exception {
                 return 1;
             }
 
-            ListBookiesCommand command = new ListBookiesCommand()
+            ListBookiesCommand.Flags flags = new ListBookiesCommand.Flags()
                 .readwrite(readwrite)
                 .readonly(readonly);
-            command.run(bkConf);
+
+            ListBookiesCommand command = new ListBookiesCommand(flags);
+
+            command.apply(bkConf, flags);
             return 0;
         }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java
index f31885c8f..61c1f9265 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java
@@ -18,8 +18,6 @@
  */
 package org.apache.bookkeeper.tools.cli.commands.bookie;
 
-import com.beust.jcommander.Parameters;
-
 import java.io.File;
 
 import org.apache.bookkeeper.bookie.Journal;
@@ -27,21 +25,28 @@
 import org.apache.bookkeeper.bookie.LogMark;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
 import org.apache.bookkeeper.util.DiskChecker;
 
 /**
  * A bookie command to print the last log marker.
  */
-@Parameters(commandDescription = "Print last log marker")
-public class LastMarkCommand extends BookieCommand {
+public class LastMarkCommand extends BookieCommand<CliFlags> {
 
-    @Override
-    public String name() {
-        return "lastmark";
+    private static final String NAME = "lastmark";
+    private static final String DESC = "Print last log marker";
+
+    public LastMarkCommand() {
+        super(CliSpec.newBuilder()
+            .withName(NAME)
+            .withFlags(new CliFlags())
+            .withDescription(DESC)
+            .build());
     }
 
     @Override
-    public void run(ServerConfiguration conf) throws Exception {
+    public boolean apply(ServerConfiguration conf, CliFlags flags) {
         LedgerDirsManager dirsManager = new LedgerDirsManager(
             conf, conf.getJournalDirs(),
             new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
@@ -54,5 +59,6 @@ public void run(ServerConfiguration conf) throws Exception {
                 + Long.toHexString(lastLogMark.getLogFileId()) + ".txn), Pos - "
                 + lastLogMark.getLogFileOffset());
         }
+        return true;
     }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java
similarity index 63%
rename from bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommand.java
rename to bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java
index 8ec872e4b..4563530e3 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java
@@ -16,44 +16,67 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.bookkeeper.tools.cli.commands.cluster;
+package org.apache.bookkeeper.tools.cli.commands.bookies;
 
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
 import static org.apache.bookkeeper.tools.cli.helpers.CommandHelpers.getBookieSocketAddrStringRepresentation;
 
 import com.beust.jcommander.Parameter;
-import com.beust.jcommander.Parameters;
 import java.util.Collection;
 import java.util.Set;
 import lombok.Setter;
 import lombok.experimental.Accessors;
 import org.apache.bookkeeper.discover.RegistrationClient;
 import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand.Flags;
 import org.apache.bookkeeper.tools.cli.helpers.DiscoveryCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
 
 /**
  * Command to list available bookies.
  */
-@Accessors(fluent = true)
-@Setter
-@Parameters(commandDescription = "List the bookies, which are running as either readwrite or readonly mode.")
-public class ListBookiesCommand extends DiscoveryCommand {
+public class ListBookiesCommand extends DiscoveryCommand<Flags> {
 
-    @Parameter(names = { "-rw", "--readwrite" }, description = "Print readwrite bookies")
-    private boolean readwrite = false;
-    @Parameter(names = { "-ro", "--readonly" }, description = "Print readonly bookies")
-    private boolean readonly = false;
+    private static final String NAME = "list";
+    private static final String DESC = "List the bookies, which are running as either readwrite or readonly mode.";
+
+    public ListBookiesCommand() {
+        this(new Flags());
+    }
+
+    public ListBookiesCommand(Flags flags) {
+        super(CliSpec.<Flags>newBuilder()
+            .withName(NAME)
+            .withDescription(DESC)
+            .withFlags(flags)
+            .build());
+    }
+
+    /**
+     * Flags for list bookies command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class Flags extends CliFlags {
+
+        @Parameter(names = { "-rw", "--readwrite" }, description = "Print readwrite bookies")
+        private boolean readwrite = false;
+        @Parameter(names = { "-ro", "--readonly" }, description = "Print readonly bookies")
+        private boolean readonly = false;
+
+    }
 
     @Override
-    protected void run(RegistrationClient regClient) throws Exception {
-        if (!readwrite && !readonly) {
+    protected void run(RegistrationClient regClient, Flags flags) throws Exception {
+        if (!flags.readwrite && !flags.readonly) {
             // case: no args is provided. list all the bookies by default.
-            readwrite = true;
-            readonly = true;
+            flags.readwrite = true;
+            flags.readonly = true;
         }
 
         boolean hasBookies = false;
-        if (readwrite) {
+        if (flags.readwrite) {
             Set<BookieSocketAddress> bookies = result(
                 regClient.getWritableBookies()
             ).getValue();
@@ -63,7 +86,7 @@ protected void run(RegistrationClient regClient) throws Exception {
                 hasBookies = true;
             }
         }
-        if (readonly) {
+        if (flags.readonly) {
             Set<BookieSocketAddress> bookies = result(
                 regClient.getReadOnlyBookies()
             ).getValue();
@@ -84,8 +107,4 @@ private static void printBookies(Collection<BookieSocketAddress> bookies) {
         }
     }
 
-    @Override
-    public String name() {
-        return "listbookies";
-    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cluster/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/package-info.java
similarity index 88%
rename from bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cluster/package-info.java
rename to bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/package-info.java
index cc3229150..acbb0a9e6 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cluster/package-info.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/package-info.java
@@ -18,6 +18,6 @@
  */
 
 /**
- * Commands on operating a cluster.
+ * Commands on operating a cluster of bookies.
  */
-package org.apache.bookkeeper.tools.cli.commands.cluster;
\ No newline at end of file
+package org.apache.bookkeeper.tools.cli.commands.bookies;
\ No newline at end of file
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommand.java
index d54486f0c..ea20a6627 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommand.java
@@ -21,52 +21,69 @@
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
 
 import com.beust.jcommander.Parameter;
-import com.beust.jcommander.Parameters;
 import java.util.concurrent.TimeUnit;
 import lombok.Setter;
 import lombok.experimental.Accessors;
 import org.apache.bookkeeper.client.api.BookKeeper;
 import org.apache.bookkeeper.client.api.DigestType;
 import org.apache.bookkeeper.client.api.WriteHandle;
+import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand.Flags;
 import org.apache.bookkeeper.tools.cli.helpers.ClientCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
 
 /**
  * A client command that simply tests if a cluster is healthy.
  */
-@Accessors(fluent = true)
-@Setter
-@Parameters(commandDescription = "Simple test to create a ledger and write entries to it.")
-public class SimpleTestCommand extends ClientCommand {
+public class SimpleTestCommand extends ClientCommand<Flags> {
 
-    @Parameter(names = { "-e", "--ensemble-size" }, description = "Ensemble size (default 3)")
-    private int ensembleSize = 3;
-    @Parameter(names = { "-w", "--write-quorum-size" }, description = "Write quorum size (default 2)")
-    private int writeQuorumSize = 2;
-    @Parameter(names = { "-a", "--ack-quorum-size" }, description = "Ack quorum size (default 2)")
-    private int ackQuorumSize = 2;
-    @Parameter(names = { "-n", "--num-entries" }, description = "Entries to write (default 100)")
-    private int numEntries = 100;
+    private static final String NAME = "simpletest";
+    private static final String DESC = "Simple test to create a ledger and write entries to it.";
 
-    @Override
-    public String name() {
-        return "simpletest";
+    /**
+     * Flags for simple test command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class Flags extends CliFlags {
+
+        @Parameter(names = { "-e", "--ensemble-size" }, description = "Ensemble size (default 3)")
+        private int ensembleSize = 3;
+        @Parameter(names = { "-w", "--write-quorum-size" }, description = "Write quorum size (default 2)")
+        private int writeQuorumSize = 2;
+        @Parameter(names = { "-a", "--ack-quorum-size" }, description = "Ack quorum size (default 2)")
+        private int ackQuorumSize = 2;
+        @Parameter(names = { "-n", "--num-entries" }, description = "Entries to write (default 100)")
+        private int numEntries = 100;
+
+    }
+    public SimpleTestCommand() {
+        this(new Flags());
+    }
+
+    public SimpleTestCommand(Flags flags) {
+        super(CliSpec.<Flags>newBuilder()
+            .withName(NAME)
+            .withDescription(DESC)
+            .withFlags(flags)
+            .build());
     }
 
     @Override
-    protected void run(BookKeeper bk) throws Exception {
+    protected void run(BookKeeper bk, Flags flags) throws Exception {
         byte[] data = new byte[100]; // test data
 
         try (WriteHandle wh = result(bk.newCreateLedgerOp()
-            .withEnsembleSize(ensembleSize)
-            .withWriteQuorumSize(writeQuorumSize)
-            .withAckQuorumSize(ackQuorumSize)
+            .withEnsembleSize(flags.ensembleSize)
+            .withWriteQuorumSize(flags.writeQuorumSize)
+            .withAckQuorumSize(flags.ackQuorumSize)
             .withDigestType(DigestType.CRC32C)
             .withPassword(new byte[0])
             .execute())) {
 
             System.out.println("Ledger ID: " + wh.getId());
             long lastReport = System.nanoTime();
-            for (int i = 0; i < numEntries; i++) {
+            for (int i = 0; i < flags.numEntries; i++) {
                 wh.append(data);
                 if (TimeUnit.SECONDS.convert(System.nanoTime() - lastReport,
                         TimeUnit.NANOSECONDS) > 1) {
@@ -74,7 +91,7 @@ protected void run(BookKeeper bk) throws Exception {
                     lastReport = System.nanoTime();
                 }
             }
-            System.out.println(numEntries + " entries written to ledger " + wh.getId());
+            System.out.println(flags.numEntries + " entries written to ledger " + wh.getId());
         }
     }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommand.java
index 91967c0d1..64c630879 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommand.java
@@ -18,8 +18,35 @@
  */
 package org.apache.bookkeeper.tools.cli.helpers;
 
+import org.apache.bookkeeper.common.net.ServiceURI;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.common.BKCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.apache.commons.configuration.CompositeConfiguration;
+
 /**
  * This is a mixin for bookie related commands to extends.
  */
-public abstract class BookieCommand implements Command {
+public abstract class BookieCommand<BookieFlagsT extends CliFlags> extends BKCommand<BookieFlagsT> {
+
+    protected BookieCommand(CliSpec<BookieFlagsT> spec) {
+        super(spec);
+    }
+
+    @Override
+    protected boolean apply(ServiceURI serviceURI,
+                            CompositeConfiguration conf,
+                            BookieFlagsT cmdFlags) {
+        ServerConfiguration serverConf = new ServerConfiguration();
+        serverConf.loadConf(conf);
+
+        if (null != serviceURI) {
+            serverConf.setMetadataServiceUri(serviceURI.getUri().toString());
+        }
+
+        return apply(serverConf, cmdFlags);
+    }
+
+    public abstract boolean apply(ServerConfiguration conf, BookieFlagsT cmdFlags);
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java
index c7a65d83f..7738911ca 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java
@@ -20,29 +20,56 @@
 
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.client.api.BookKeeper;
+import org.apache.bookkeeper.common.net.ServiceURI;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.common.BKCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.apache.commons.configuration.CompositeConfiguration;
 
 /**
  * This is a mixin class for commands that needs a bookkeeper client.
  */
 @Slf4j
-public abstract class ClientCommand implements Command {
+public abstract class ClientCommand<ClientFlagsT extends CliFlags> extends BKCommand<ClientFlagsT> {
+
+    protected ClientCommand(CliSpec<ClientFlagsT> spec) {
+        super(spec);
+    }
 
     @Override
-    public void run(ServerConfiguration conf) throws Exception {
-        // cast the server configuration to a client configuration object.
+    protected boolean apply(ServiceURI serviceURI,
+                            CompositeConfiguration conf,
+                            ClientFlagsT cmdFlags) {
+        ClientConfiguration clientConf = new ClientConfiguration();
+        clientConf.loadConf(conf);
+
+        if (null != serviceURI) {
+            clientConf.setMetadataServiceUri(serviceURI.getUri().toString());
+        }
+
+        return apply(clientConf, cmdFlags);
+    }
+
+    public boolean apply(ServerConfiguration conf,
+                         ClientFlagsT cmdFlags) {
         ClientConfiguration clientConf = new ClientConfiguration(conf);
-        run(clientConf);
+        return apply(clientConf, cmdFlags);
     }
 
-    protected void run(ClientConfiguration conf) throws Exception {
+    protected boolean apply(ClientConfiguration conf,
+                            ClientFlagsT cmdFlags) {
         try (BookKeeper bk = BookKeeper.newBuilder(conf).build()) {
-            run(bk);
+            run(bk, cmdFlags);
+            return true;
+        } catch (Exception e) {
+            log.error("Faild to process command '{}'", name(), e);
+            return false;
         }
     }
 
-    protected abstract void run(BookKeeper bk) throws Exception;
-
+    protected abstract void run(BookKeeper bk, ClientFlagsT cmdFlags)
+        throws Exception;
 
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/Command.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/Command.java
deleted file mode 100644
index 1ce60933c..000000000
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/Command.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.bookkeeper.tools.cli.helpers;
-
-import org.apache.bookkeeper.conf.ServerConfiguration;
-
-/**
- * A basic command interface provides a run method to execute it.
- */
-public interface Command {
-
-    /**
-     * Command name.
-     *
-     * @return command name.
-     */
-    String name();
-
-    /**
-     * Validate the args before running the command.
-     *
-     * @return true if the args are valid, otherwise false.
-     */
-    default boolean validateArgs() {
-        return true;
-    }
-
-    /**
-     * Run the command with provided configuration.
-     *
-     * @param conf server configuration.
-     * @throws Exception
-     */
-    void run(ServerConfiguration conf) throws Exception;
-
-}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommand.java
index b8f93a517..5af6af392 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommand.java
@@ -24,34 +24,51 @@
 import java.util.concurrent.ScheduledExecutorService;
 import lombok.Cleanup;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.api.BookKeeper;
 import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationClient;
 import org.apache.bookkeeper.meta.MetadataClientDriver;
 import org.apache.bookkeeper.meta.MetadataDrivers;
 import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
 
 /**
  * This is a mixin for commands that talks to discovery service.
  */
 @Slf4j
-public abstract class DiscoveryCommand implements Command {
+public abstract class DiscoveryCommand<DiscoveryFlagsT extends CliFlags> extends ClientCommand<DiscoveryFlagsT> {
+
+    protected DiscoveryCommand(CliSpec<DiscoveryFlagsT> spec) {
+        super(spec);
+    }
 
     @Override
-    public void run(ServerConfiguration conf) throws Exception {
-        URI metadataServiceUri = URI.create(conf.getMetadataServiceUri());
-        @Cleanup("shutdown") ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
-        try (MetadataClientDriver driver = MetadataDrivers.getClientDriver(metadataServiceUri)) {
-            ClientConfiguration clientConf = new ClientConfiguration(conf);
-            driver.initialize(
-                clientConf,
-                executor,
-                NullStatsLogger.INSTANCE,
-                Optional.empty());
-            run(driver.getRegistrationClient());
+    protected boolean apply(ClientConfiguration clientConf, DiscoveryFlagsT cmdFlags) {
+        try {
+            URI metadataServiceUri = URI.create(clientConf.getMetadataServiceUri());
+            @Cleanup("shutdown") ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+            try (MetadataClientDriver driver = MetadataDrivers.getClientDriver(metadataServiceUri)) {
+                driver.initialize(
+                    clientConf,
+                    executor,
+                    NullStatsLogger.INSTANCE,
+                    Optional.empty());
+                run(driver.getRegistrationClient(), cmdFlags);
+                return true;
+            }
+        } catch (Exception e) {
+            log.error("Fail to process command '{}'", name(), e);
+            return false;
         }
     }
 
-    protected abstract void run(RegistrationClient regClient) throws Exception;
+    @Override
+    protected void run(BookKeeper bk, DiscoveryFlagsT cmdFlags) throws Exception {
+        throw new IllegalStateException("It should never be called.");
+    }
+
+    protected abstract void run(RegistrationClient regClient, DiscoveryFlagsT cmdFlags)
+        throws Exception;
 
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
index 57fd10131..9f12e15e4 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
@@ -26,7 +26,7 @@
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.ArgumentMatchers.same;
-import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
@@ -50,8 +50,9 @@
 import org.apache.bookkeeper.meta.MetadataBookieDriver;
 import org.apache.bookkeeper.meta.MetadataDrivers;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
+import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand;
 import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand;
-import org.apache.bookkeeper.tools.cli.commands.cluster.ListBookiesCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
 import org.apache.bookkeeper.util.EntryFormatter;
 import org.apache.bookkeeper.util.LedgerIdFormatter;
 import org.apache.bookkeeper.versioning.LongVersion;
@@ -85,7 +86,9 @@
 
     // commands
     private LastMarkCommand mockLastMarkCommand;
+    private SimpleTestCommand.Flags mockSimpleTestFlags;
     private SimpleTestCommand mockSimpleTestCommand;
+    private ListBookiesCommand.Flags mockListBookiesFlags;
     private ListBookiesCommand mockListBookiesCommand;
 
     @Before
@@ -95,15 +98,33 @@ public void setup() throws Exception {
         whenNew(LastMarkCommand.class)
             .withNoArguments()
             .thenReturn(mockLastMarkCommand);
+
+        // setup the mocks for simple test command
+        this.mockSimpleTestFlags = spy(new SimpleTestCommand.Flags());
+        whenNew(SimpleTestCommand.Flags.class)
+            .withNoArguments()
+            .thenReturn(mockSimpleTestFlags);
+
         this.mockSimpleTestCommand = spy(new SimpleTestCommand());
-        doNothing().when(mockSimpleTestCommand).run(any(ServerConfiguration.class));
+        doReturn(true).when(mockSimpleTestCommand)
+            .apply(any(ServerConfiguration.class), any(SimpleTestCommand.Flags.class));
         whenNew(SimpleTestCommand.class)
-            .withNoArguments()
+            .withParameterTypes(SimpleTestCommand.Flags.class)
+            .withArguments(mockSimpleTestFlags)
             .thenReturn(mockSimpleTestCommand);
+
+        // setup the mocks for list bookies command
+        this.mockListBookiesFlags = spy(new ListBookiesCommand.Flags());
+        whenNew(ListBookiesCommand.Flags.class)
+            .withNoArguments()
+            .thenReturn(mockListBookiesFlags);
+
         this.mockListBookiesCommand = spy(new ListBookiesCommand());
-        doNothing().when(mockListBookiesCommand).run(any(ServerConfiguration.class));
+        doReturn(true).when(mockListBookiesCommand)
+            .apply(any(ServerConfiguration.class), any(ListBookiesCommand.Flags.class));
         whenNew(ListBookiesCommand.class)
-            .withNoArguments()
+            .withParameterTypes(ListBookiesCommand.Flags.class)
+            .withArguments(mockListBookiesFlags)
             .thenReturn(mockListBookiesCommand);
 
         // construct the bookie shell.
@@ -323,7 +344,8 @@ void testRecoverCmdRecover(boolean dryrun,
     public void testLastMarkCmd() throws Exception {
         shell.run(new String[] { "lastmark"});
         verifyNew(LastMarkCommand.class, times(1)).withNoArguments();
-        verify(mockLastMarkCommand, times(1)).run(same(shell.bkConf));
+        verify(mockLastMarkCommand, times(1))
+            .apply(same(shell.bkConf), any(CliFlags.class));
     }
 
     @Test
@@ -335,12 +357,14 @@ public void testSimpleTestCmd() throws Exception {
             "-a", "3",
             "-n", "200"
         });
-        verifyNew(SimpleTestCommand.class, times(1)).withNoArguments();
-        verify(mockSimpleTestCommand, times(1)).run(same(shell.bkConf));
-        verify(mockSimpleTestCommand, times(1)).ensembleSize(eq(10));
-        verify(mockSimpleTestCommand, times(1)).writeQuorumSize(eq(5));
-        verify(mockSimpleTestCommand, times(1)).ackQuorumSize(eq(3));
-        verify(mockSimpleTestCommand, times(1)).numEntries(eq(200));
+        verifyNew(SimpleTestCommand.class, times(1))
+            .withArguments(same(mockSimpleTestFlags));
+        verify(mockSimpleTestCommand, times(1))
+            .apply(same(shell.bkConf), same(mockSimpleTestFlags));
+        verify(mockSimpleTestFlags, times(1)).ensembleSize(eq(10));
+        verify(mockSimpleTestFlags, times(1)).writeQuorumSize(eq(5));
+        verify(mockSimpleTestFlags, times(1)).ackQuorumSize(eq(3));
+        verify(mockSimpleTestFlags, times(1)).numEntries(eq(200));
     }
 
     @Test
@@ -364,10 +388,12 @@ public void testListBookiesCmdReadOnly() throws Exception {
         assertEquals(0, shell.run(new String[] {
             "listbookies", "-ro"
         }));
-        verifyNew(ListBookiesCommand.class, times(1)).withNoArguments();
-        verify(mockListBookiesCommand, times(1)).run(same(shell.bkConf));
-        verify(mockListBookiesCommand, times(1)).readonly(true);
-        verify(mockListBookiesCommand, times(1)).readwrite(false);
+        verifyNew(ListBookiesCommand.class, times(1))
+            .withArguments(same(mockListBookiesFlags));
+        verify(mockListBookiesCommand, times(1))
+            .apply(same(shell.bkConf), same(mockListBookiesFlags));
+        verify(mockListBookiesFlags, times(1)).readonly(true);
+        verify(mockListBookiesFlags, times(1)).readwrite(false);
     }
 
     @Test
@@ -375,9 +401,11 @@ public void testListBookiesCmdReadWrite() throws Exception {
         assertEquals(0, shell.run(new String[] {
             "listbookies", "-rw"
         }));
-        verifyNew(ListBookiesCommand.class, times(1)).withNoArguments();
-        verify(mockListBookiesCommand, times(1)).run(same(shell.bkConf));
-        verify(mockListBookiesCommand, times(1)).readonly(false);
-        verify(mockListBookiesCommand, times(1)).readwrite(true);
+        verifyNew(ListBookiesCommand.class, times(1))
+            .withArguments(same(mockListBookiesFlags));
+        verify(mockListBookiesCommand, times(1))
+            .apply(same(shell.bkConf), same(mockListBookiesFlags));
+        verify(mockListBookiesFlags, times(1)).readonly(false);
+        verify(mockListBookiesFlags, times(1)).readwrite(true);
     }
 }
diff --git a/conf/log4j.cli.properties b/conf/log4j.cli.properties
index 3c4d0ded7..51c95f58c 100644
--- a/conf/log4j.cli.properties
+++ b/conf/log4j.cli.properties
@@ -25,7 +25,7 @@
 
 # DEFAULT: console appender only
 # Define some default values that can be overridden by system properties
-bookkeeper.cli.root.logger=ERROR,CONSOLE
+bookkeeper.cli.root.logger=INFO,CONSOLE
 bookkeeper.cli.log.dir=logs
 bookkeeper.cli.log.file=bookkeeper-cli.log
 
diff --git a/shaded/bookkeeper-server-shaded/pom.xml b/shaded/bookkeeper-server-shaded/pom.xml
index e4b2597e8..2f0e2b564 100644
--- a/shaded/bookkeeper-server-shaded/pom.xml
+++ b/shaded/bookkeeper-server-shaded/pom.xml
@@ -66,6 +66,7 @@
                   <include>com.google.guava:guava</include>
                   <include>com.google.protobuf:protobuf-java</include>
                   <include>org.apache.bookkeeper:bookkeeper-common</include>
+                  <include>org.apache.bookkeeper:bookkeeper-tools-framework</include>
                   <include>org.apache.bookkeeper:bookkeeper-proto</include>
                   <include>org.apache.bookkeeper:bookkeeper-server</include>
                   <include>org.apache.bookkeeper:circe-checksum</include>
diff --git a/shaded/bookkeeper-server-tests-shaded/pom.xml b/shaded/bookkeeper-server-tests-shaded/pom.xml
index da86686c6..a8ebfaa70 100644
--- a/shaded/bookkeeper-server-tests-shaded/pom.xml
+++ b/shaded/bookkeeper-server-tests-shaded/pom.xml
@@ -47,6 +47,10 @@
           <groupId>org.apache.bookkeeper</groupId>
           <artifactId>bookkeeper-common</artifactId>
         </exclusion>
+        <exclusion>
+          <groupId>org.apache.bookkeeper</groupId>
+          <artifactId>bookkeeper-tools-framework</artifactId>
+        </exclusion>
         <exclusion>
           <groupId>org.apache.bookkeeper</groupId>
           <artifactId>bookkeeper-proto</artifactId>
diff --git a/shaded/distributedlog-core-shaded/pom.xml b/shaded/distributedlog-core-shaded/pom.xml
index e8929dcf5..565665711 100644
--- a/shaded/distributedlog-core-shaded/pom.xml
+++ b/shaded/distributedlog-core-shaded/pom.xml
@@ -87,6 +87,7 @@
                   <include>net.java.dev.jna:jna</include>
                   <include>net.jpountz.lz4:lz4</include>
                   <include>org.apache.bookkeeper:bookkeeper-common</include>
+                  <include>org.apache.bookkeeper:bookkeeper-tools-framework</include>
                   <include>org.apache.bookkeeper:bookkeeper-proto</include>
                   <include>org.apache.bookkeeper:bookkeeper-server</include>
                   <include>org.apache.bookkeeper:circe-checksum</include>
diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java
index 27c376cd8..f31beef8a 100644
--- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java
+++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java
@@ -33,7 +33,7 @@
 import org.junit.runners.MethodSorters;
 
 /**
- * Test `bin/bookkeeper-cli`.
+ * Test `bin/bkctl`.
  */
 @Slf4j
 @RunWith(Arquillian.class)
@@ -44,11 +44,11 @@
     private DockerClient docker;
 
     private String currentVersion = System.getProperty("currentVersion");
-    private String bkCLI;
+    private String bkctl;
 
     @Before
     public void setup() {
-        bkCLI = "/opt/bookkeeper/" + currentVersion + "/bin/bookkeeper-cli";
+        bkctl = "/opt/bookkeeper/" + currentVersion + "/bin/bkctl";
     }
 
     @Test
@@ -70,8 +70,8 @@ public void test999_Teardown() throws Exception {
     public void test001_SimpleTest() throws Exception {
         String bookie = BookKeeperClusterUtils.getAnyBookie();
         assertTrue(DockerUtils.runCommand(docker, bookie,
-            bkCLI,
-            "client",
+            bkctl,
+            "ledger",
             "simpletest",
             "--ensemble-size", "3",
             "--write-quorum-size", "3",
@@ -84,9 +84,9 @@ public void test001_SimpleTest() throws Exception {
     public void test002_ListROBookies() throws Exception {
         String bookie = BookKeeperClusterUtils.getAnyBookie();
         assertTrue(DockerUtils.runCommand(docker, bookie,
-            bkCLI,
-            "cluster",
-            "listbookies",
+            bkctl,
+            "bookies",
+            "list",
             "-ro"
         ).contains("No bookie exists!"));
     }
@@ -95,9 +95,9 @@ public void test002_ListROBookies() throws Exception {
     public void test003_ListRWBookies() throws Exception {
         String bookie = BookKeeperClusterUtils.getAnyBookie();
         assertTrue(DockerUtils.runCommand(docker, bookie,
-            bkCLI,
-            "cluster",
-            "listbookies",
+            bkctl,
+            "bookies",
+            "list",
             "-rw"
         ).contains("ReadWrite Bookies :"));
     }
diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/BKCtl.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/BKCtl.java
new file mode 100644
index 000000000..970f4634d
--- /dev/null
+++ b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/BKCtl.java
@@ -0,0 +1,62 @@
+/*
+ * 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.bookkeeper.tools.cli;
+
+import java.util.Iterator;
+import java.util.ServiceLoader;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.Cli;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.apache.bookkeeper.tools.framework.CommandGroup;
+
+/**
+ * <b>bkctl</b> interacts and operates the <i>Apache BookKeeper</i> cluster.
+ */
+@Slf4j
+public class BKCtl {
+
+    public static final String NAME = "bkctl";
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) {
+        // load command groups
+        ServiceLoader<CommandGroup> loader = ServiceLoader.load(
+            CommandGroup.class, BKCtl.class.getClassLoader());
+
+        CliSpec.Builder<BKFlags> specBuilder = CliSpec.<BKFlags>newBuilder()
+            .withName(NAME)
+            .withUsage(NAME + " [flags] [command group] [commands]")
+            .withDescription(NAME + " interacts and operates Apache BookKeeper clusters")
+            .withFlags(new BKFlags())
+            .withConsole(System.out);
+
+        Iterator<CommandGroup> cgIter = loader.iterator();
+        while (cgIter.hasNext()) {
+            CommandGroup<BKFlags> cg = cgIter.next();
+            specBuilder.addCommand(cg);
+        }
+
+        CliSpec<BKFlags> spec = specBuilder.build();
+
+        int retCode = Cli.runCli(spec, args);
+        Runtime.getRuntime().exit(retCode);
+    }
+
+}
diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/BookKeeperCLI.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/BookKeeperCLI.java
deleted file mode 100644
index 3bc0888ca..000000000
--- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/BookKeeperCLI.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * 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.bookkeeper.tools.cli;
-
-import com.beust.jcommander.JCommander;
-import com.beust.jcommander.Parameter;
-import com.google.common.annotations.VisibleForTesting;
-import java.net.MalformedURLException;
-import java.nio.file.Paths;
-import java.util.Arrays;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-import lombok.AccessLevel;
-import lombok.Getter;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.tools.cli.commands.CmdBase;
-import org.apache.bookkeeper.tools.cli.commands.CmdBookie;
-import org.apache.bookkeeper.tools.cli.commands.CmdClient;
-import org.apache.bookkeeper.tools.cli.commands.CmdCluster;
-import org.apache.bookkeeper.tools.cli.commands.CmdMetadata;
-import org.apache.commons.configuration.ConfigurationException;
-
-/**
- * BookKeeper CLI.
- */
-@Slf4j
-public class BookKeeperCLI {
-
-    /**
-     * Make this command map static. This provides a way to plugin different sub commands.
-     */
-    private static final Map<String, Class> commandMap;
-
-    static {
-        commandMap = new TreeMap<>();
-
-        // build the default command map
-        commandMap.put("bookie", CmdBookie.class);
-        commandMap.put("client", CmdClient.class);
-        commandMap.put("cluster", CmdCluster.class);
-        commandMap.put("metadata", CmdMetadata.class);
-    }
-
-    static JCommander newJCommander() {
-        return new JCommander();
-    }
-
-    @SuppressWarnings("unchecked")
-    @VisibleForTesting
-    public static Object newCommandInstance(Class cls, ServerConfiguration config) throws Exception {
-        return cls.getConstructor(ServerConfiguration.class).newInstance(config);
-    }
-
-    @SuppressWarnings("unchecked")
-    @VisibleForTesting
-    public static Object newCommandInstance(Class cls,
-                                            ServerConfiguration config,
-                                            JCommander commander) throws Exception {
-        return cls.getConstructor(ServerConfiguration.class, JCommander.class)
-            .newInstance(config, commander);
-    }
-
-    public static void registerSubcommand(String commandName, Class commandClass) {
-        synchronized (commandMap) {
-            commandMap.put(commandName, commandClass);
-        }
-    }
-
-    public static void unregisterSubcommand(String commandName) {
-        synchronized (commandMap) {
-            commandMap.remove(commandName);
-        }
-    }
-
-    @Getter(AccessLevel.PACKAGE)
-    static class ShellArguments {
-
-        @Parameter(names = { "-c", "--conf" }, description = "Bookie Configuration File")
-        private String configFile = null;
-
-        @Parameter(names = { "-h", "--help" }, description = "Show this help message")
-        private boolean help = false;
-
-    }
-
-    @Getter(value = AccessLevel.PACKAGE)
-    private final ShellArguments shellArgs;
-    @Getter(value = AccessLevel.PACKAGE)
-    private final JCommander commander;
-    private final ServerConfiguration config;
-
-    BookKeeperCLI() throws Exception {
-        this.shellArgs = new ShellArguments();
-        this.commander = newJCommander();
-        this.commander.setProgramName("bookkeeper-cli");
-        this.commander.addObject(shellArgs);
-
-        this.config = new ServerConfiguration();
-    }
-
-    boolean setupShell() {
-        for (Entry<String, Class> entry : commandMap.entrySet()) {
-            try {
-                Object obj = newCommandInstance(entry.getValue(), config);
-                log.info("Setup command {}", entry.getValue());
-                this.commander.addCommand(
-                    entry.getKey(),
-                    obj);
-            } catch (Exception e) {
-                System.err.println("Fail to load sub command '" + entry.getKey() + "' : " + e.getMessage());
-                e.printStackTrace();
-                return false;
-            }
-        }
-        return true;
-    }
-
-    @VisibleForTesting
-    boolean runArgs(String... args) {
-        return run(args);
-    }
-
-    boolean run(String[] args) {
-        if (!setupShell()) {
-            // fail to setup the shell, fail this command.
-            return false;
-        }
-
-        if (args.length == 0) {
-            commander.usage();
-            return false;
-        }
-
-        int cmdPos;
-        for (cmdPos = 0; cmdPos < args.length; cmdPos++) {
-            if (commandMap.containsKey(args[cmdPos])) {
-                break;
-            }
-        }
-
-        try {
-            commander.parse(Arrays.copyOfRange(args, 0, Math.min(cmdPos, args.length)));
-        } catch (Exception e) {
-            System.err.println(e.getMessage());
-            System.err.println();
-            commander.usage();
-            return false;
-        }
-
-        if (shellArgs.help) {
-            commander.usage();
-            return false;
-        }
-
-        if (null != shellArgs.configFile) {
-            try {
-                config.loadConf(Paths.get(shellArgs.configFile).toUri().toURL());
-            } catch (ConfigurationException | MalformedURLException e) {
-                System.err.println("Failed to load configuration file '" + shellArgs.configFile + "' : "
-                    + e.getMessage());
-                System.err.println();
-                commander.usage();
-                return false;
-            }
-        }
-
-        log.info("cmd pos = {}", cmdPos);
-
-        if (cmdPos == args.length) {
-            commander.usage();
-            return false;
-        } else {
-            String cmd = args[cmdPos];
-            JCommander subCmd = commander.getCommands().get(cmd);
-            CmdBase subCmdObj = (CmdBase) subCmd.getObjects().get(0);
-            String[] subCmdArgs = Arrays.copyOfRange(args, cmdPos + 1, args.length);
-
-            log.info("Run sub command : {}", subCmdArgs);
-
-            return subCmdObj.run(subCmdArgs);
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        BookKeeperCLI shell = new BookKeeperCLI();
-
-        if (shell.run(args)) {
-            System.exit(0);
-        } else {
-            System.exit(1);
-        }
-    }
-
-}
diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBookie.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
similarity index 58%
rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBookie.java
rename to tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
index 0c01ccb24..ec4fbc5f6 100644
--- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBookie.java
+++ b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
@@ -18,17 +18,28 @@
  */
 package org.apache.bookkeeper.tools.cli.commands;
 
-import com.beust.jcommander.Parameters;
-import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.cli.BKCtl;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliCommandGroup;
+import org.apache.bookkeeper.tools.framework.CliSpec;
 
 /**
  * Commands that operates a single bookie.
  */
-@Parameters(commandDescription = "Commands on operating a single bookie")
-public class CmdBookie extends CmdBase {
-    public CmdBookie(ServerConfiguration conf) {
-        super("bookie", conf);
-        addSubCommand(new LastMarkCommand());
+public class BookieCommandGroup extends CliCommandGroup<BKFlags> {
+
+    private static final String NAME = "bookie";
+    private static final String DESC = "Commands on operating a single bookie";
+
+    private static final CliSpec<BKFlags> spec = CliSpec.<BKFlags>newBuilder()
+        .withName(NAME)
+        .withDescription(DESC)
+        .withParent(BKCtl.NAME)
+        .addCommand(new LastMarkCommand())
+        .build();
+
+    public BookieCommandGroup() {
+        super(spec);
     }
 }
diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdCluster.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java
similarity index 51%
rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdCluster.java
rename to tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java
index 4ba1cb4c8..1fff60f8f 100644
--- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdCluster.java
+++ b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java
@@ -18,17 +18,28 @@
  */
 package org.apache.bookkeeper.tools.cli.commands;
 
-import com.beust.jcommander.Parameters;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.tools.cli.commands.cluster.ListBookiesCommand;
+import org.apache.bookkeeper.tools.cli.BKCtl;
+import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliCommandGroup;
+import org.apache.bookkeeper.tools.framework.CliSpec;
 
 /**
- * Commands that interact with a cluster, such as format.
+ * Commands that interact with a cluster of bookies.
  */
-@Parameters(commandDescription = "Commands that operate a cluster")
-public class CmdCluster extends CmdBase {
-    public CmdCluster(ServerConfiguration conf) {
-        super("cluster", conf);
-        addSubCommand(new ListBookiesCommand());
+public class BookiesCommandGroup extends CliCommandGroup<BKFlags> {
+
+    private static final String NAME = "bookies";
+    private static final String DESC = "Commands on operating a cluster of bookies";
+
+    private static final CliSpec<BKFlags> spec = CliSpec.<BKFlags>newBuilder()
+        .withName(NAME)
+        .withDescription(DESC)
+        .withParent(BKCtl.NAME)
+        .addCommand(new ListBookiesCommand())
+        .build();
+
+    public BookiesCommandGroup() {
+        super(spec);
     }
 }
diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBase.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBase.java
deleted file mode 100644
index 1b3da3e0b..000000000
--- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBase.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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.bookkeeper.tools.cli.commands;
-
-import com.beust.jcommander.JCommander;
-import com.beust.jcommander.Parameter;
-import lombok.AccessLevel;
-import lombok.Getter;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.tools.cli.helpers.Command;
-
-/**
- * The command base for other sub commands to extend.
- */
-@Slf4j
-public abstract class CmdBase {
-
-    // Parameters defined for this command
-
-    @Parameter(names = { "-h", "--help" }, help = true, hidden = true)
-    private boolean help;
-
-    // Parameters defined for this command (end)
-
-    protected final JCommander commander;
-    @Getter(AccessLevel.PUBLIC)
-    protected final ServerConfiguration conf;
-
-    protected CmdBase(String cmdName, ServerConfiguration conf) {
-        this(cmdName, conf, new JCommander());
-    }
-
-    protected CmdBase(String cmdName, ServerConfiguration conf, JCommander commander) {
-        this.conf = conf;
-        this.commander = commander;
-        this.commander.setProgramName("bookkeeper-cli " + cmdName);
-    }
-
-    protected void addSubCommand(Command command) {
-        this.commander.addCommand(command.name(), command);
-    }
-
-    public boolean run(String[] args) {
-        try {
-            commander.parse(args);
-        } catch (Exception e) {
-            System.err.println(e.getMessage());
-            System.err.println();
-            commander.usage();
-            return false;
-        }
-
-        String cmd = commander.getParsedCommand();
-        if (null == cmd) {
-            commander.usage();
-            return false;
-        }
-
-        JCommander cmdObj = commander.getCommands().get(cmd);
-        Command subCmd = (Command) cmdObj.getObjects().get(0);
-
-        try {
-            if (subCmd.validateArgs()) {
-                subCmd.run(conf);
-                return true;
-            } else {
-                return false;
-            }
-        } catch (Exception e) {
-            System.err.println("Failed to execute command '" + cmd + "' : " + e.getMessage());
-            e.printStackTrace();
-            System.err.println();
-            commander.usage();
-            return false;
-        }
-
-    }
-}
diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdMetadata.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdMetadata.java
deleted file mode 100644
index 7fa97947d..000000000
--- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdMetadata.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.bookkeeper.tools.cli.commands;
-
-import com.beust.jcommander.Parameters;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-
-/**
- *
- */
-@Parameters(commandDescription = "Commands that interact with metadata storage")
-public class CmdMetadata extends CmdBase {
-    public CmdMetadata(ServerConfiguration conf) {
-        super("metadata", conf);
-    }
-}
diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdClient.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java
similarity index 59%
rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdClient.java
rename to tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java
index d57e1afbe..4236c4918 100644
--- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdClient.java
+++ b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java
@@ -18,18 +18,29 @@
  */
 package org.apache.bookkeeper.tools.cli.commands;
 
-import com.beust.jcommander.Parameters;
-import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.cli.BKCtl;
 import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliCommandGroup;
+import org.apache.bookkeeper.tools.framework.CliSpec;
 
 /**
  * Client operations that interacts with a cluster, such as simpletest.
  */
-@Parameters(commandDescription = "Commands that interact with a cluster")
-public class CmdClient extends CmdBase {
+public class LedgerCommandGroup extends CliCommandGroup<BKFlags> {
 
-    public CmdClient(ServerConfiguration conf) {
-        super("client", conf);
-        addSubCommand(new SimpleTestCommand());
+    private static final String NAME = "ledger";
+    private static final String DESC = "Commands on interacting with ledgers";
+
+    private static CliSpec<BKFlags> spec = CliSpec.<BKFlags>newBuilder()
+        .withName(NAME)
+        .withDescription(DESC)
+        .withParent(BKCtl.NAME)
+        .addCommand(new SimpleTestCommand())
+        .build();
+
+    public LedgerCommandGroup() {
+        super(spec);
     }
+
 }
diff --git a/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup b/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup
new file mode 100644
index 000000000..47df897dd
--- /dev/null
+++ b/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+org.apache.bookkeeper.tools.cli.commands.BookieCommandGroup
+org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup
+org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/BookKeeperCLITest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/BookKeeperCLITest.java
deleted file mode 100644
index 22dad5952..000000000
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/BookKeeperCLITest.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/*
- * 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.bookkeeper.tools.cli;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.powermock.api.mockito.PowerMockito.spy;
-import static org.powermock.api.mockito.PowerMockito.when;
-
-import com.beust.jcommander.JCommander;
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.Parameters;
-import java.io.File;
-import java.net.URL;
-import java.nio.file.Paths;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.tools.cli.commands.CmdBase;
-import org.apache.bookkeeper.tools.cli.commands.CmdBookie;
-import org.apache.bookkeeper.tools.cli.commands.CmdClient;
-import org.apache.bookkeeper.tools.cli.commands.CmdCluster;
-import org.apache.bookkeeper.tools.cli.commands.CmdMetadata;
-import org.apache.bookkeeper.tools.cli.helpers.Command;
-import org.apache.commons.configuration.ConfigurationException;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.powermock.api.mockito.PowerMockito;
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-/**
- * Unit test of {@link BookKeeperCLI}.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({ BookKeeperCLI.class, CmdBase.class })
-@PowerMockIgnore("org.apache.log4j.*")
-@Slf4j
-public class BookKeeperCLITest {
-
-    @Parameters(commandDescription = "sub test command")
-    static class SubTestCommand implements Command {
-
-        @Parameter(names = "--value")
-        private int value = -1;
-
-        @Override
-        public String name() {
-            return "subtest";
-        }
-
-        @Override
-        public void run(ServerConfiguration conf) throws Exception {
-        }
-    }
-
-    @Parameters(commandDescription = "test command")
-    static class CmdTest extends CmdBase {
-
-        public CmdTest(ServerConfiguration conf) {
-            this(conf, new JCommander());
-        }
-
-        public CmdTest(ServerConfiguration conf, JCommander commander) {
-            super("test", conf, commander);
-            addSubCommand(new SubTestCommand());
-        }
-
-        @Override
-        public boolean run(String[] args) {
-            return super.run(args);
-        }
-    }
-
-    @Rule
-    public final TemporaryFolder testDir = new TemporaryFolder();
-
-    private ServerConfiguration conf;
-    private JCommander commander;
-    private JCommander subCommander;
-    private BookKeeperCLI shell;
-    private CmdTest cmdTest;
-    private SubTestCommand subCmdTest;
-
-    @Before
-    public void setup() throws Exception {
-        this.conf = spy(new ServerConfiguration());
-        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments()
-            .thenReturn(conf);
-
-        this.commander = spy(new JCommander());
-        PowerMockito.mockStatic(BookKeeperCLI.class);
-        when(BookKeeperCLI.newJCommander()).thenReturn(commander);
-
-        this.subCommander = spy(new JCommander());
-        this.cmdTest = spy(new CmdTest(conf, subCommander));
-        when(BookKeeperCLI.newCommandInstance(eq(CmdTest.class), eq(conf)))
-            .thenReturn(cmdTest);
-        when(BookKeeperCLI.newCommandInstance(eq(CmdClient.class), eq(conf)))
-            .thenReturn(new CmdClient(conf));
-        when(BookKeeperCLI.newCommandInstance(eq(CmdCluster.class), eq(conf)))
-            .thenReturn(new CmdCluster(conf));
-        when(BookKeeperCLI.newCommandInstance(eq(CmdMetadata.class), eq(conf)))
-            .thenReturn(new CmdMetadata(conf));
-        when(BookKeeperCLI.newCommandInstance(eq(CmdBookie.class), eq(conf)))
-            .thenReturn(new CmdBookie(conf));
-
-        PowerMockito.doCallRealMethod().when(
-            BookKeeperCLI.class, "registerSubcommand", eq("test"), eq(CmdTest.class));
-        BookKeeperCLI.registerSubcommand("test", CmdTest.class);
-
-
-        this.subCmdTest = spy(new SubTestCommand());
-        PowerMockito.whenNew(SubTestCommand.class).withNoArguments()
-            .thenReturn(subCmdTest);
-
-        this.shell = spy(new BookKeeperCLI());
-    }
-
-    @Test
-    public void testNoArgs() throws ConfigurationException {
-        BookKeeperCLI.unregisterSubcommand("test");
-
-        assertFalse(shell.runArgs());
-        verify(shell, times(1)).setupShell();
-        assertNull(shell.getShellArgs().getConfigFile());
-        assertFalse(shell.getShellArgs().isHelp());
-        verify(conf, times(0)).loadConf(any(URL.class));
-        verify(commander, times(1)).usage();
-    }
-
-    @Test
-    public void testHelpShort() throws ConfigurationException {
-        assertFalse(shell.runArgs("-h"));
-        verify(shell, times(1)).setupShell();
-        assertNull(shell.getShellArgs().getConfigFile());
-        assertTrue(shell.getShellArgs().isHelp());
-        verify(conf, times(0)).loadConf(any(URL.class));
-        verify(commander, times(1)).usage();
-    }
-
-    @Test
-    public void testHelpLong() throws Exception {
-        assertFalse(shell.runArgs("--help"));
-        verify(shell, times(1)).setupShell();
-        assertNull(shell.getShellArgs().getConfigFile());
-        assertTrue(shell.getShellArgs().isHelp());
-        verify(conf, times(0)).loadConf(any(URL.class));
-        verify(commander, times(1)).usage();
-    }
-
-    @Test
-    public void testUnknownCommand() throws Exception {
-        assertFalse(shell.runArgs("unknown"));
-        verify(shell, times(1)).setupShell();
-        assertNull(shell.getShellArgs().getConfigFile());
-        assertFalse(shell.getShellArgs().isHelp());
-        verify(conf, times(0)).loadConf(any(URL.class));
-        verify(commander, times(1)).usage();
-    }
-
-    @Test
-    public void testConfShort() throws Exception {
-        File confFile = testDir.newFile();
-        confFile.createNewFile();
-        assertFalse(shell.runArgs("-c", confFile.getAbsolutePath()));
-        assertEquals(confFile.getAbsolutePath(), shell.getShellArgs().getConfigFile());
-        assertFalse(shell.getShellArgs().isHelp());
-        verify(conf, times(1)).loadConf(
-            eq(Paths.get(confFile.getAbsolutePath()).toUri().toURL()));
-        verify(commander, times(1)).usage();
-    }
-
-    @Test
-    public void testConfLong() throws Exception {
-        File confFile = testDir.newFile();
-        confFile.createNewFile();
-        assertFalse(shell.runArgs("--conf", confFile.getAbsolutePath()));
-        assertEquals(confFile.getAbsolutePath(), shell.getShellArgs().getConfigFile());
-        assertFalse(shell.getShellArgs().isHelp());
-        verify(conf, times(1)).loadConf(
-            eq(Paths.get(confFile.getAbsolutePath()).toUri().toURL()));
-        verify(commander, times(1)).usage();
-    }
-
-    @Test
-    public void testCmdTestNoSubCommand() throws Exception {
-        assertFalse(shell.runArgs("test"));
-        assertNull(shell.getShellArgs().getConfigFile());
-        assertFalse(shell.getShellArgs().isHelp());
-        verify(conf, times(0)).loadConf(any(URL.class));
-        verify(commander, times(0)).usage();
-        verify(cmdTest, times(1)).run(eq(new String[0]));
-    }
-
-    @Test
-    public void testCmdTestWithUnknownSubCommand() throws Exception {
-        assertFalse(shell.runArgs("test", "unknown"));
-        assertNull(shell.getShellArgs().getConfigFile());
-        assertFalse(shell.getShellArgs().isHelp());
-        verify(commander, times(0)).usage();
-        verify(conf, times(0)).loadConf(any(URL.class));
-        verify(cmdTest, times(1)).run(eq(new String[] { "unknown" }));
-        assertEquals(-1, subCmdTest.value);
-        assertSame(conf, cmdTest.getConf());
-        verify(subCommander, times(1)).usage();
-    }
-
-    @Test
-    public void testCmdTestWithSubCommandNoArgs() throws Exception {
-        assertTrue(shell.runArgs("test", "subtest"));
-        assertNull(shell.getShellArgs().getConfigFile());
-        assertFalse(shell.getShellArgs().isHelp());
-        verify(commander, times(0)).usage();
-        verify(conf, times(0)).loadConf(any(URL.class));
-        verify(cmdTest, times(1)).run(eq(new String[] { "subtest" }));
-        assertEquals(-1, subCmdTest.value);
-        assertSame(conf, cmdTest.getConf());
-        verify(subCommander, times(0)).usage();
-    }
-
-    @Test
-    public void testCmdTestWithSubCommandWithArgs() throws Exception {
-        assertTrue(shell.runArgs("test", "subtest", "--value", "10"));
-        assertNull(shell.getShellArgs().getConfigFile());
-        assertFalse(shell.getShellArgs().isHelp());
-        verify(commander, times(0)).usage();
-        verify(conf, times(0)).loadConf(any(URL.class));
-        verify(cmdTest, times(1)).run(eq(new String[] { "subtest", "--value", "10" }));
-        assertSame(conf, cmdTest.getConf());
-        verify(subCommander, times(0)).usage();
-    }
-}
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/CmdBaseTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/CmdBaseTest.java
deleted file mode 100644
index 8d7deb90b..000000000
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/CmdBaseTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.bookkeeper.tools.cli.commands;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-import com.beust.jcommander.JCommander;
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.Parameters;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.tools.cli.helpers.Command;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Unit test of "bookie" commands.
- */
-public class CmdBaseTest {
-
-    @Parameters(commandDescription = "sub command")
-    static class SubCommand implements Command {
-
-        @Parameter(names = "--value")
-        private int value = -1;
-
-        @Override
-        public String name() {
-            return "subtest";
-        }
-
-        @Override
-        public void run(ServerConfiguration conf) throws Exception {
-        }
-    }
-
-    @Parameters(commandDescription = "test command")
-    static class CmdTest extends CmdBase {
-
-        CmdTest(ServerConfiguration conf) {
-            this(conf, new JCommander(), new SubCommand());
-        }
-
-        CmdTest(ServerConfiguration conf, JCommander commander, SubCommand subCommand) {
-            super("test", conf, commander);
-            addSubCommand(subCommand);
-        }
-    }
-
-    private ServerConfiguration conf;
-    private JCommander commander;
-    private SubCommand subCommand;
-    private CmdTest cmdTest;
-
-    @Before
-    public void setup() {
-        this.conf = new ServerConfiguration();
-        this.commander = spy(new JCommander());
-        this.subCommand = spy(new SubCommand());
-        this.cmdTest = new CmdTest(conf, commander, subCommand);
-    }
-
-    @Test
-    public void testParseFailure() throws Exception {
-        String[] args = new String[] { "--unknown-flag" };
-        assertFalse(cmdTest.run(args));
-        verify(commander, times(1)).parse(args);
-        verify(commander, times(1)).usage();
-        verify(commander, times(0)).getParsedCommand();
-    }
-
-    @Test
-    public void testUnknownSubCommand() throws Exception {
-        String[] args = new String[] { "unknown" };
-        assertFalse(cmdTest.run(args));
-        verify(commander, times(1)).parse(args);
-        verify(commander, times(1)).usage();
-        verify(commander, times(0)).getParsedCommand();
-    }
-
-    @Test
-    public void testSubCommandNoArgs() throws Exception {
-        String[] args = new String[] { "subtest" };
-        assertTrue(cmdTest.run(args));
-        verify(commander, times(1)).parse(args);
-        verify(commander, times(0)).usage();
-        verify(commander, times(1)).getParsedCommand();
-        verify(commander, times(1)).getCommands();
-        verify(subCommand, times(1)).run(eq(conf));
-        assertEquals(-1, subCommand.value);
-    }
-
-    @Test
-    public void testSubCommandWithArgs() throws Exception {
-        String[] args = new String[] { "subtest", "--value", "10" };
-        assertTrue(cmdTest.run(args));
-        verify(commander, times(1)).parse(args);
-        verify(commander, times(0)).usage();
-        verify(commander, times(1)).getParsedCommand();
-        verify(commander, times(1)).getCommands();
-        verify(subCommand, times(1)).run(eq(conf));
-        assertEquals(10, subCommand.value);
-    }
-}
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java
index b2c1d8b34..f338577b4 100644
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java
+++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java
@@ -59,6 +59,10 @@ public LastMarkCommandTest() {
     public void setup() throws Exception {
         super.setup();
 
+        PowerMockito.whenNew(ServerConfiguration.class)
+            .withNoArguments()
+            .thenReturn(conf);
+
         PowerMockito.whenNew(LedgerDirsManager.class)
             .withParameterTypes(
                 ServerConfiguration.class,
@@ -91,9 +95,8 @@ public void setup() throws Exception {
 
     @Test
     public void testCommand() throws Exception {
-        CommandRunner runner = createCommandRunner(new LastMarkCommand());
-
-        runner.runArgs("lastmark");
+        LastMarkCommand cmd = new LastMarkCommand();
+        cmd.apply(bkFlags, new String[] { "" });
 
         PowerMockito.verifyNew(LedgerDirsManager.class, times(1))
             .withArguments(eq(conf), any(File[].class), any(DiskChecker.class));
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java
similarity index 86%
rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommandTest.java
rename to tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java
index 5e7de7d65..f887b3bb8 100644
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommandTest.java
+++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java
@@ -16,11 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.bookkeeper.tools.cli.commands.cluster;
+package org.apache.bookkeeper.tools.cli.commands.bookies;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.value;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Answers.CALLS_REAL_METHODS;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
@@ -104,55 +105,59 @@ private static void verifyPrintBookies(int startPort, int numBookies, int numCal
     @Test
     public void testListReadWriteShortArgs() {
         testCommand(true, false,
-            "listbookies",
+            "list",
             "-rw");
     }
 
     @Test
     public void testListReadWriteLongArgs() {
         testCommand(true, false,
-            "listbookies",
+            "list",
             "--readwrite");
     }
 
     @Test
     public void testListReadOnlyShortArgs() {
         testCommand(false, true,
-            "listbookies",
+            "list",
             "-ro");
     }
 
     @Test
     public void testListReadOnlyLongArgs() {
         testCommand(false, true,
-            "listbookies",
+            "list",
             "--readonly");
     }
 
     @Test
     public void testListNoArgs() {
         testCommand(true, true,
-            "listbookies");
+            "list");
     }
 
     @Test
     public void testListTwoFlagsCoexistsShortArgs() {
         testCommand(true, true,
-            "listbookies", "-rw", "-ro");
+            "list", "-rw", "-ro");
     }
 
     @Test
     public void testListTwoFlagsCoexistsLongArgs() {
         testCommand(true, true,
-            "listbookies", "--readwrite", "--readonly");
+            "list", "--readwrite", "--readonly");
     }
 
     private void testCommand(boolean readwrite,
                              boolean readonly,
                              String... args) {
 
-        CommandRunner runner = createCommandRunner(new ListBookiesCommand());
-        assertTrue(runner.runArgs(args));
+        ListBookiesCommand cmd = new ListBookiesCommand();
+        try {
+            assertTrue(cmd.apply(bkFlags, args));
+        } catch (Exception e) {
+            fail("Should not throw any exception here");
+        }
 
         if (readwrite && !readonly) {
             verifyPrintBookies(3181, 10, 1);
@@ -167,15 +172,22 @@ private void testCommand(boolean readwrite,
     }
 
     @Test
-    public void testListEmptyBookies() {
+    public void testListEmptyBookies() throws Exception {
         // overwrite regClient to return empty bookies
         when(regClient.getWritableBookies())
             .thenReturn(value(new Versioned<>(Collections.emptySet(), new LongVersion(0L))));
         when(regClient.getReadOnlyBookies())
             .thenReturn(value(new Versioned<>(Collections.emptySet(), new LongVersion(0L))));
 
-        CommandRunner runner = createCommandRunner(new ListBookiesCommand());
-        assertTrue(runner.runArgs("listbookies"));
+        ListBookiesCommand cmd = new ListBookiesCommand();
+        assertTrue(cmd.apply(bkFlags, new String[] { "-rw"}));
+
+        PowerMockito.verifyStatic(
+                CommandHelpers.class,
+                times(0));
+        CommandHelpers.getBookieSocketAddrStringRepresentation(any());
+
+        assertTrue(cmd.apply(bkFlags, new String[] { "-ro"}));
 
         PowerMockito.verifyStatic(
                 CommandHelpers.class,
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java
index a901f2182..a63e10806 100644
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java
+++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java
@@ -18,7 +18,6 @@
  */
 package org.apache.bookkeeper.tools.cli.commands.client;
 
-import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.eq;
@@ -43,7 +42,6 @@
     @Test
     public void testCommandShortArgs() throws Exception {
         testCommand(
-            "simpletest",
             "-e", "5",
             "-w", "3",
             "-a", "3",
@@ -53,7 +51,6 @@ public void testCommandShortArgs() throws Exception {
     @Test
     public void testCommandLongArgs() throws Exception {
         testCommand(
-            "simpletest",
             "--ensemble-size", "5",
             "--write-quorum-size", "3",
             "--ack-quorum-size", "3",
@@ -74,8 +71,8 @@ public void testCommand(String... args) throws Exception {
         when(createBuilder.withPassword(any(byte[].class))).thenReturn(createBuilder);
         when(mockBk.newCreateLedgerOp()).thenReturn(createBuilder);
 
-        CommandRunner runner = createCommandRunner(new SimpleTestCommand());
-        assertTrue(runner.runArgs(args));
+        SimpleTestCommand cmd = new SimpleTestCommand();
+        cmd.apply(bkFlags, args);
 
         // verify create builder
         verify(createBuilder, times(1)).withEnsembleSize(eq(5));
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java
index d541db987..2b2d09d1d 100644
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java
+++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java
@@ -18,7 +18,9 @@
  */
 package org.apache.bookkeeper.tools.cli.helpers;
 
+import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.same;
 import static org.mockito.Mockito.CALLS_REAL_METHODS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
@@ -30,6 +32,7 @@
 import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.framework.CliFlags;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -44,12 +47,13 @@
 @PrepareForTest({ ClientCommand.class, BookKeeper.class })
 public class ClientCommandTest {
 
-    private ClientCommand cmd;
+    private ClientCommand<CliFlags> cmd;
     private ServerConfiguration serverConf;
     private ClientConfiguration clientConf;
     private BookKeeperBuilder bkBuilder;
     private BookKeeper bk;
 
+    @SuppressWarnings("unchecked")
     @Before
     public void setup() throws Exception {
         this.cmd = mock(ClientCommand.class, CALLS_REAL_METHODS);
@@ -72,9 +76,10 @@ public void setup() throws Exception {
 
     @Test
     public void testRun() throws Exception {
-        cmd.run(serverConf);
-        verify(cmd, times(1)).run(eq(clientConf));
-        verify(cmd, times(1)).run(eq(bk));
+        CliFlags flags = new CliFlags();
+        assertTrue(cmd.apply(serverConf, flags));
+        verify(cmd, times(1)).apply(eq(clientConf), same(flags));
+        verify(cmd, times(1)).run(eq(bk), same(flags));
         verify(bkBuilder, times(1)).build();
     }
 
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java
index 6b41e5763..af5cfbe1b 100644
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java
+++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java
@@ -55,6 +55,9 @@ public void setup() throws Exception {
         mockBk = mock(BookKeeper.class);
         this.clientConf = spy(new ClientConfiguration(conf));
         this.clientConf.setMetadataServiceUri("zk://127.0.0.1/path/to/ledgers");
+        PowerMockito.whenNew(ClientConfiguration.class)
+            .withNoArguments()
+            .thenReturn(clientConf);
         PowerMockito.whenNew(ClientConfiguration.class)
             .withParameterTypes(AbstractConfiguration.class)
             .withArguments(eq(conf))
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java
index 162d244f5..b88c7719e 100644
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java
+++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java
@@ -20,7 +20,7 @@
 
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.tools.cli.commands.CmdBase;
+import org.apache.bookkeeper.tools.common.BKFlags;
 
 /**
  * A test base providing an environment for run a command.
@@ -28,32 +28,13 @@
 @Slf4j
 public class CommandTestBase {
 
-    /**
-     * Command Runner to run commands inherited from {@link CmdBase}.
-     */
-    protected static class CommandRunner extends CmdBase {
-
-        public CommandRunner(ServerConfiguration conf, Command command) {
-            super("test-runner", conf);
-            commander.addCommand(command.name(), command);
-        }
-
-        public boolean runArgs(String... args) {
-            log.info("Executing command {}", args);
-            return run(args);
-        }
-
-    }
-
-    protected CommandRunner createCommandRunner(Command command) {
-        return new CommandRunner(conf, command);
-    }
-
+    protected final BKFlags bkFlags;
     protected final ServerConfiguration conf;
 
     public CommandTestBase() {
         this.conf = new ServerConfiguration();
         this.conf.setMetadataServiceUri("zk://127.0.0.1/path/to/ledgers");
+        this.bkFlags = new BKFlags();
     }
 
 }
diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java
index 004ba2423..e3c6fdf5a 100644
--- a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java
+++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java
@@ -18,9 +18,11 @@
  */
 package org.apache.bookkeeper.tools.cli.helpers;
 
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Answers.CALLS_REAL_METHODS;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.same;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.powermock.api.mockito.PowerMockito.mock;
@@ -37,6 +39,7 @@
 import org.apache.bookkeeper.meta.MetadataClientDriver;
 import org.apache.bookkeeper.meta.MetadataDrivers;
 import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.tools.framework.CliFlags;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -51,13 +54,14 @@
 @PrepareForTest({ DiscoveryCommand.class, MetadataDrivers.class })
 public class DiscoveryCommandTest {
 
-    private DiscoveryCommand cmd;
+    private DiscoveryCommand<CliFlags> cmd;
     private ServerConfiguration serverConf;
     private ClientConfiguration clientConf;
     private RegistrationClient regClient;
     private MetadataClientDriver clientDriver;
     private ScheduledExecutorService executor;
 
+    @SuppressWarnings("unchecked")
     @Before
     public void setup() throws Exception {
         PowerMockito.mockStatic(Executors.class);
@@ -87,8 +91,9 @@ public void setup() throws Exception {
 
     @Test
     public void testRun() throws Exception {
-        cmd.run(serverConf);
-        verify(cmd, times(1)).run(eq(regClient));
+        CliFlags cliFlags = new CliFlags();
+        assertTrue(cmd.apply(serverConf, cliFlags));
+        verify(cmd, times(1)).run(eq(regClient), same(cliFlags));
         verify(clientDriver, times(1))
             .initialize(eq(clientConf), eq(executor), eq(NullStatsLogger.INSTANCE), eq(Optional.empty()));
         verify(clientDriver, times(1)).getRegistrationClient();
diff --git a/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/package-info.java b/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/package-info.java
index 2268b6324..d6660dee1 100644
--- a/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/package-info.java
+++ b/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/package-info.java
@@ -19,4 +19,4 @@
 /**
  * Common classes used across multiple tools.
  */
-package org.apache.bookkeeper.tools.common;
\ No newline at end of file
+package org.apache.bookkeeper.tools.common;


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services