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/02/16 17:22:46 UTC

[GitHub] sijie closed pull request #1094: BP-27 (part 1): New BookKeeper CLI Skeleton

sijie closed pull request #1094: BP-27 (part 1): New BookKeeper CLI Skeleton
URL: https://github.com/apache/bookkeeper/pull/1094
 
 
   

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/bookkeeper-cli
new file mode 100755
index 000000000..722fe5df0
--- /dev/null
+++ b/bin/bookkeeper-cli
@@ -0,0 +1,155 @@
+#!/usr/bin/env bash
+#
+#/**
+# * Copyright 2007 The Apache Software Foundation
+# *
+# * 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.
+# */
+
+BINDIR=`dirname "$0"`
+BK_HOME=`cd $BINDIR/..;pwd`
+
+# TODO: change the default conf after moving conf to root module
+#       {@link https://github.com/apache/bookkeeper/issues/1147}
+DEFAULT_CONF=$BK_HOME/bookkeeper-server/conf/bk_server.conf
+DEFAULT_LOG_CONF=$BK_HOME/conf/log4j.cli.properties
+TOOLS_HOME=${BK_HOME}/bookkeeper-tools
+
+source $BK_HOME/conf/bk_cli_env.sh
+
+# Check for the java to use
+if [[ -z $JAVA_HOME ]]; then
+  JAVA=$(which java)
+  if [ $? = 0 ]; then
+    echo "JAVA_HOME not set, using java from PATH. ($JAVA)"
+  else
+    echo "Error: JAVA_HOME not set, and no java executable found in $PATH." 1>&2
+    exit 1
+  fi
+else
+  JAVA=$JAVA_HOME/bin/java
+fi
+
+find_cli_jar() {
+  DIR=$1
+  if [ -d $DIR ]; then
+    cd $DIR
+    for f in *.jar; do
+      if [[ $f =~ ^(org.apache.bookkeeper-)?bookkeeper-tools-[0-9\\.]*(-SNAPSHOT)?.jar$ ]]; then
+        echo $DIR/$f
+          return
+      fi
+    done
+  fi
+}
+
+RELEASE_JAR=$(find_cli_jar ${BK_HOME})
+if [ -n "${RELEASE_JAR}" ]; then
+  CLI_JAR=${RELEASE_JAR}
+else
+  RELEASE_JAR=$(find_cli_jar ${BK_HOME}/lib)
+  if [ -n "${RELEASE_JAR}" ]; then
+    CLI_JAR=${RELEASE_JAR}
+  fi
+fi
+
+BUILT_JAR=$(find_cli_jar ${TOOLS_HOME}/target)
+
+if [ -z "${BUILT_JAR}" ] && [ -z "${CLI_JAR}" ]; then
+  echo "Couldn't find bookkeeper jar."
+  read -p "Do you want me to run \`mvn package -DskiptTests\` for you ? " answer
+  case "${answer:0:1}" in
+    y|Y )
+      mvn package -DskipTests
+      ;;
+    * )
+      exit 1
+      ;;
+  esac
+
+  BUILT_JAR=$(find_cli_jar ${TOOLS_HOME}/target)
+  if [ -n "${BUILT_JAR}" ]; then
+    CLI_JAR=$BUILT_JAR
+  fi
+fi
+
+if [ -e "${BUILT_JAR}" ]; then
+  CLI_JAR="${BUILT_JAR}"
+fi
+
+if [ ! -e "${CLI_JAR}" ]; then
+  echo "Could not find bookkeeper cli jar."
+  exit 1
+fi
+
+add_maven_deps_to_classpath() {
+  MVN="mvn"
+  if [ "$MAVEN_HOME" != "" ]; then
+    MVN=${MAVEN_HOME}/bin/mvn
+  fi
+
+  # Need to generate classpath from maven pom. This is costly so generate it
+  # and cache it. Save the file into our target dir so a mvn clean will get
+  # clean it up and force us create a new one.
+  f="${TOOLS_HOME}/target/cached_classpath.txt"
+  if [ ! -f "${f}" ]
+  then
+    ${MVN} -f "${TOOLS_HOME}/pom.xml" dependency:build-classpath -Dmdep.outputFile="${f}" &> /dev/null
+  fi
+  CLI_CLASSPATH=${CLASSPATH}:`cat "${f}"`
+}
+
+if [ -d "$BK_HOME/lib" ]; then
+  for i in $BK_HOME/lib/*.jar; do
+    CLI_CLASSPATH=$CLI_CLASSPATH:$i
+  done
+else
+  add_maven_deps_to_classpath
+fi
+
+if [ -z "$CLI_CONF" ]; then
+  CLI_CONF=$DEFAULT_CONF
+fi
+
+if [ -z "$CLI_LOG_CONF" ]; then
+  CLI_LOG_CONF=$DEFAULT_LOG_CONF
+fi
+
+CLI_CLASSPATH="$CLI_JAR:$CLI_CLASSPATH:$CLI_EXTRA_CLASSPATH"
+CLI_CLASSPATH="`dirname $CLI_LOG_CONF`:$CLI_CLASSPATH"
+OPTS="$OPTS -Dlog4j.configuration=`basename $CLI_LOG_CONF`"
+
+OPTS="-cp $CLI_CLASSPATH $OPTS"
+
+OPTS="$OPTS $CLI_EXTRA_OPTS"
+
+# Disable ipv6 as it can cause issues
+OPTS="$OPTS -Djava.net.preferIPv4Stack=true"
+
+# log directory & file
+CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,CONSOLE"}
+CLI_LOG_DIR=${CLI_LOG_DIR:-"$BK_HOME/logs"}
+CLI_LOG_FILE=${CLI_LOG_FILE:-"bookkeeper-cli.log"}
+
+#Configure log configuration system properties
+OPTS="$OPTS -Dbookkeeper.cli.root.logger=$CLI_ROOT_LOGGER"
+OPTS="$OPTS -Dbookkeeper.cli.log.dir=$CLI_LOG_DIR"
+OPTS="$OPTS -Dbookkeeper.cli.log.file=$CLI_LOG_FILE"
+
+#Change to BK_HOME to support relative paths
+cd "$BK_HOME"
+exec $JAVA $OPTS org.apache.bookkeeper.tools.cli.BookieShell --conf $CLI_CONF $@
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
index 3b2246a5e..e1d0dcf9b 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
@@ -270,6 +270,7 @@ Apache Software License, Version 2.
 - lib/org.eclipse.jetty-jetty-servlet-9.4.5.v20170502.jar [34]
 - lib/org.eclipse.jetty-jetty-util-9.4.5.v20170502.jar [34]
 - lib/org.rocksdb-rocksdbjni-5.8.6.jar [35]
+- lib/com.beust-jcommander-1.48.jar [36]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
@@ -306,6 +307,7 @@ Apache Software License, Version 2.
 [33] Source available at https://github.com/apache/zookeeper/tree/release-3.5.3
 [34] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502
 [35] Source available at https://github.com/facebook/rocksdb/tree/v5.8.6
+[36] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-3.10.1.Final.jar contains the extensions to Java Collections Framework which has
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
index 226106305..74e9861a0 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
@@ -235,6 +235,7 @@ Apache Software License, Version 2.
 - lib/org.eclipse.jetty-jetty-servlet-9.4.5.v20170502.jar [21]
 - lib/org.eclipse.jetty-jetty-util-9.4.5.v20170502.jar [21]
 - lib/org.rocksdb-rocksdbjni-5.8.6.jar [22]
+- lib/com.beust-jcommander-1.48.jar [23]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
@@ -258,6 +259,7 @@ Apache Software License, Version 2.
 [20] Source available at https://github.com/apache/zookeeper/tree/release-3.5.3
 [21] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502
 [22] Source available at https://github.com/facebook/rocksdb/tree/v5.8.6
+[23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-all-4.1.12.Final.jar bundles some 3rd party dependencies
diff --git a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt
index 63ae3e922..735741451 100644
--- a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt
+++ b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt
@@ -122,3 +122,8 @@ Permission to use, copy, modify and distribute UnixCrypt
 for non-commercial or commercial purposes and without fee is
 granted provided that the copyright notice appears in all copies.
 ------------------------------------------------------------------------------------
+- lib/com.beust-jcommander-1.48.jar
+
+Copyright 2010 Cedric Beust cedric@beust.com
+
+------------------------------------------------------------------------------------
diff --git a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt
index c6fd7abe0..905c4c1a5 100644
--- a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt
+++ b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt
@@ -82,3 +82,8 @@ Permission to use, copy, modify and distribute UnixCrypt
 for non-commercial or commercial purposes and without fee is
 granted provided that the copyright notice appears in all copies.
 ------------------------------------------------------------------------------------
+- lib/com.beust-jcommander-1.48.jar
+
+Copyright 2010 Cedric Beust cedric@beust.com
+
+------------------------------------------------------------------------------------
diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml
index f0721b6ab..ed0353e73 100644
--- a/bookkeeper-server/pom.xml
+++ b/bookkeeper-server/pom.xml
@@ -136,6 +136,10 @@
       <artifactId>commons-collections4</artifactId>
       <version>4.1</version>
     </dependency>
+    <dependency>
+      <groupId>com.beust</groupId>
+      <artifactId>jcommander</artifactId>
+    </dependency>
     <dependency>
       <groupId>net.java.dev.jna</groupId>
       <artifactId>jna</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 477f66bca..c9e4d9373 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
@@ -19,11 +19,11 @@
 package org.apache.bookkeeper.bookie;
 
 import static com.google.common.base.Charsets.UTF_8;
+import static org.apache.bookkeeper.tools.cli.helpers.CommandHelpers.getBookieSocketAddrStringRepresentation;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.net.InetAddresses;
 import com.google.common.util.concurrent.AbstractFuture;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufUtil;
@@ -97,6 +97,9 @@
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
 import org.apache.bookkeeper.replication.AuditorElector;
 import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
+import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand;
+import org.apache.bookkeeper.tools.cli.commands.cluster.ListBookiesCommand;
 import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.util.EntryFormatter;
@@ -1065,33 +1068,18 @@ Options getOptions() {
 
         @Override
         public int runCmd(CommandLine cmdLine) throws Exception {
-            byte[] data = new byte[100]; // test data
-
             int ensemble = getOptionIntValue(cmdLine, "ensemble", 3);
             int writeQuorum = getOptionIntValue(cmdLine, "writeQuorum", 2);
             int ackQuorum = getOptionIntValue(cmdLine, "ackQuorum", 2);
             int numEntries = getOptionIntValue(cmdLine, "numEntries", 1000);
 
-            ClientConfiguration conf = new ClientConfiguration();
-            conf.addConfiguration(bkConf);
-            BookKeeper bk = new BookKeeper(conf);
-            LedgerHandle lh = bk.createLedger(ensemble, writeQuorum, ackQuorum,
-                    BookKeeper.DigestType.MAC, new byte[0]);
-            System.out.println("Ledger ID: " + lh.getId());
-            long lastReport = System.nanoTime();
-            for (int i = 0; i < numEntries; i++) {
-                lh.addEntry(data);
-                if (TimeUnit.SECONDS.convert(System.nanoTime() - lastReport,
-                        TimeUnit.NANOSECONDS) > 1) {
-                    System.out.println(i + " entries written");
-                    lastReport = System.nanoTime();
-                }
-            }
-
-            lh.close();
-            bk.close();
-            System.out.println(numEntries + " entries written to ledger " + lh.getId());
+            SimpleTestCommand command = new SimpleTestCommand()
+                .ensembleSize(ensemble)
+                .writeQuorumSize(writeQuorum)
+                .ackQuorumSize(ackQuorum)
+                .numEntries(numEntries);
 
+            command.run(bkConf);
             return 0;
         }
 
@@ -1438,7 +1426,8 @@ Options getOptions() {
 
         @Override
         public int runCmd(CommandLine c) throws Exception {
-            printLastLogMark();
+            LastMarkCommand command = new LastMarkCommand();
+            command.run(bkConf);
             return 0;
         }
 
@@ -1482,29 +1471,11 @@ public int runCmd(CommandLine cmdLine) throws Exception {
                 printUsage();
                 return 1;
             }
-            ClientConfiguration clientConf = new ClientConfiguration(bkConf);
-            clientConf.setZkServers(bkConf.getZkServers());
-            BookKeeperAdmin bka = new BookKeeperAdmin(clientConf);
-
-            int count = 0;
-            Collection<BookieSocketAddress> bookies = new ArrayList<BookieSocketAddress>();
-            if (cmdLine.hasOption("rw")) {
-                Collection<BookieSocketAddress> availableBookies = bka
-                        .getAvailableBookies();
-                bookies.addAll(availableBookies);
-            } else if (cmdLine.hasOption("ro")) {
-                Collection<BookieSocketAddress> roBookies = bka
-                        .getReadOnlyBookies();
-                bookies.addAll(roBookies);
-            }
-            for (BookieSocketAddress b : bookies) {
-                System.out.println(getBookieSocketAddrStringRepresentation(b));
-                count++;
-            }
-            if (count == 0) {
-                System.err.println("No bookie exists!");
-                return 1;
-            }
+
+            ListBookiesCommand command = new ListBookiesCommand()
+                .readwrite(readwrite)
+                .readonly(readonly);
+            command.run(bkConf);
             return 0;
         }
 
@@ -2793,27 +2764,6 @@ public int run(String[] args) throws Exception {
         return cmd.runCmd(newArgs);
     }
 
-    /*
-     * The string returned is of the form:
-     * 'hostname'('otherformofhostname'):'port number'
-     *
-     * where hostname and otherformofhostname are ipaddress and
-     * canonicalhostname or viceversa
-     */
-    private static String getBookieSocketAddrStringRepresentation(BookieSocketAddress bookieId) {
-        String hostname = bookieId.getHostName();
-        boolean isHostNameIpAddress = InetAddresses.isInetAddress(hostname);
-        String otherFormOfHostname = null;
-        if (isHostNameIpAddress) {
-            otherFormOfHostname = bookieId.getSocketAddress().getAddress().getCanonicalHostName();
-        } else {
-            otherFormOfHostname = bookieId.getSocketAddress().getAddress().getHostAddress();
-        }
-        String bookieSocketAddrStringRepresentation = hostname + "(" + otherFormOfHostname + ")" + ":"
-                + bookieId.getSocketAddress().getPort();
-        return bookieSocketAddrStringRepresentation;
-    }
-
     /**
      * Returns the sorted list of the files in the given folders with the given file extensions.
      * Sorting is done on the basis of CreationTime if the CreationTime is not available or if they are equal
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
new file mode 100644
index 000000000..a83f195d5
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java
@@ -0,0 +1,61 @@
+/*
+ * 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.bookie;
+
+import com.beust.jcommander.Parameters;
+import com.google.common.collect.Lists;
+import java.util.List;
+import org.apache.bookkeeper.bookie.Journal;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LogMark;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
+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 {
+
+    @Override
+    public String name() {
+        return "lastmark";
+    }
+
+    @Override
+    public void run(ServerConfiguration conf) throws Exception {
+        LedgerDirsManager dirsManager = new LedgerDirsManager(
+            conf, conf.getJournalDirs(),
+            new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
+        List<Journal> journals = Lists.transform(
+            Lists.newArrayList(conf.getJournalDirs()),
+            dir -> new Journal(
+                dir,
+                conf,
+                dirsManager)
+        );
+        for (Journal journal : journals) {
+            LogMark lastLogMark = journal.getLastLogMark().getCurMark();
+            System.out.println("LastLogMark : Journal Id - " + lastLogMark.getLogFileId() + "("
+                + Long.toHexString(lastLogMark.getLogFileId()) + ".txn), Pos - "
+                + lastLogMark.getLogFileOffset());
+        }
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/package-info.java
new file mode 100644
index 000000000..429cc4e9d
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * Commands to operate a single bookie.
+ */
+package org.apache.bookkeeper.tools.cli.commands.bookie;
\ 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
new file mode 100644
index 000000000..60633ebaf
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommand.java
@@ -0,0 +1,80 @@
+/*
+ * 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.client;
+
+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.helpers.ClientCommand;
+
+/**
+ * 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 {
+
+    @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;
+
+    @Override
+    public String name() {
+        return "simpletest";
+    }
+
+    @Override
+    protected void run(BookKeeper bk) throws Exception {
+        byte[] data = new byte[100]; // test data
+
+        try (WriteHandle wh = result(bk.newCreateLedgerOp()
+            .withEnsembleSize(ensembleSize)
+            .withWriteQuorumSize(writeQuorumSize)
+            .withAckQuorumSize(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++) {
+                result(wh.append(data));
+                if (TimeUnit.SECONDS.convert(System.nanoTime() - lastReport,
+                        TimeUnit.NANOSECONDS) > 1) {
+                    System.out.println(i + " entries written");
+                    lastReport = System.nanoTime();
+                }
+            }
+            System.out.println(numEntries + " entries written to ledger " + wh.getId());
+        }
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/package-info.java
new file mode 100644
index 000000000..5b321b5da
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains all client related commands.
+ */
+package org.apache.bookkeeper.tools.cli.commands.client;
\ No newline at end of file
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/cluster/ListBookiesCommand.java
new file mode 100644
index 000000000..8ec872e4b
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommand.java
@@ -0,0 +1,91 @@
+/*
+ * 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.cluster;
+
+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.helpers.DiscoveryCommand;
+
+/**
+ * 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 {
+
+    @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) {
+            // case: no args is provided. list all the bookies by default.
+            readwrite = true;
+            readonly = true;
+        }
+
+        boolean hasBookies = false;
+        if (readwrite) {
+            Set<BookieSocketAddress> bookies = result(
+                regClient.getWritableBookies()
+            ).getValue();
+            if (!bookies.isEmpty()) {
+                System.out.println("ReadWrite Bookies :");
+                printBookies(bookies);
+                hasBookies = true;
+            }
+        }
+        if (readonly) {
+            Set<BookieSocketAddress> bookies = result(
+                regClient.getReadOnlyBookies()
+            ).getValue();
+            if (!bookies.isEmpty()) {
+                System.out.println("Readonly Bookies :");
+                printBookies(bookies);
+                hasBookies = true;
+            }
+        }
+        if (!hasBookies) {
+            System.err.println("No bookie exists!");
+        }
+    }
+
+    private static void printBookies(Collection<BookieSocketAddress> bookies) {
+        for (BookieSocketAddress b : bookies) {
+            System.out.println(getBookieSocketAddrStringRepresentation(b));
+        }
+    }
+
+    @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/cluster/package-info.java
new file mode 100644
index 000000000..cc3229150
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cluster/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * Commands on operating a cluster.
+ */
+package org.apache.bookkeeper.tools.cli.commands.cluster;
\ No newline at end of file
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/package-info.java
new file mode 100644
index 000000000..737cd3405
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package provides all the actual commands in bookie shell.
+ */
+package org.apache.bookkeeper.tools.cli.commands;
\ No newline at end of file
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
new file mode 100644
index 000000000..91967c0d1
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommand.java
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/**
+ * This is a mixin for bookie related commands to extends.
+ */
+public abstract class BookieCommand implements Command {
+}
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
new file mode 100644
index 000000000..c7a65d83f
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java
@@ -0,0 +1,48 @@
+/*
+ * 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 lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.api.BookKeeper;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * This is a mixin class for commands that needs a bookkeeper client.
+ */
+@Slf4j
+public abstract class ClientCommand implements Command {
+
+    @Override
+    public void run(ServerConfiguration conf) throws Exception {
+        // cast the server configuration to a client configuration object.
+        ClientConfiguration clientConf = new ClientConfiguration(conf);
+        run(clientConf);
+    }
+
+    protected void run(ClientConfiguration conf) throws Exception {
+        try (BookKeeper bk = BookKeeper.newBuilder(conf).build()) {
+            run(bk);
+        }
+    }
+
+    protected abstract void run(BookKeeper bk) 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
new file mode 100644
index 000000000..1ce60933c
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/Command.java
@@ -0,0 +1,52 @@
+/*
+ * 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/CommandHelpers.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java
new file mode 100644
index 000000000..88ac52e4a
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.net.InetAddresses;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+
+/**
+ * Helper classes used by the cli commands.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CommandHelpers {
+
+    /*
+     * The string returned is of the form:
+     * 'hostname'('otherformofhostname'):'port number'
+     *
+     * where hostname and otherformofhostname are ipaddress and
+     * canonicalhostname or viceversa
+     */
+    public static String getBookieSocketAddrStringRepresentation(BookieSocketAddress bookieId) {
+        String hostname = bookieId.getHostName();
+        boolean isHostNameIpAddress = InetAddresses.isInetAddress(hostname);
+        String otherFormOfHostname = null;
+        if (isHostNameIpAddress) {
+            otherFormOfHostname = bookieId.getSocketAddress().getAddress().getCanonicalHostName();
+        } else {
+            otherFormOfHostname = bookieId.getSocketAddress().getAddress().getHostAddress();
+        }
+        String bookieSocketAddrStringRepresentation = hostname + "(" + otherFormOfHostname + ")" + ":"
+                + bookieId.getSocketAddress().getPort();
+        return bookieSocketAddrStringRepresentation;
+    }
+
+}
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
new file mode 100644
index 000000000..1ebfd88c5
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommand.java
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationClient;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.ReflectionUtils;
+
+/**
+ * This is a mixin for commands that talks to discovery service.
+ */
+@Slf4j
+public abstract class DiscoveryCommand implements Command {
+
+    @Override
+    public void run(ServerConfiguration conf) throws Exception {
+        // cast the server configuration to a client configuration object.
+        ClientConfiguration clientConf = new ClientConfiguration(conf);
+        run(clientConf);
+    }
+
+    protected void run(ClientConfiguration conf) throws Exception {
+        Class<? extends RegistrationClient> regClientCls = conf.getRegistrationClientClass();
+        @Cleanup("shutdown") ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+        try (RegistrationClient regClient = ReflectionUtils.newInstance(regClientCls)) {
+            regClient.initialize(
+                conf,
+                executor,
+                NullStatsLogger.INSTANCE,
+                Optional.empty());
+            run(regClient);
+        }
+    }
+
+    protected abstract void run(RegistrationClient regClient) throws Exception;
+
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/package-info.java
new file mode 100644
index 000000000..68c3d70dd
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/**
+ * This classes provides some helper mixins for easily to add commands
+ * to bookie shell.
+ */
+package org.apache.bookkeeper.tools.cli.helpers;
\ No newline at end of file
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/package-info.java
new file mode 100644
index 000000000..90ad08d49
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/**
+ * BookKeeper CLI that interacts with bookkeeper cluster.
+ *
+ * <p>TODO: this package will be moved to `bookkeeper-tools` module
+ * after refactoring.
+ */
+package org.apache.bookkeeper.tools.cli;
\ No newline at end of file
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 4a1658e1e..b77b5e9dd 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
@@ -25,10 +25,15 @@
 import static org.mockito.ArgumentMatchers.any;
 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.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.spy;
+import static org.powermock.api.mockito.PowerMockito.verifyNew;
 import static org.powermock.api.mockito.PowerMockito.whenNew;
 
 import com.google.common.collect.Maps;
@@ -43,6 +48,9 @@
 import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener;
 import org.apache.bookkeeper.discover.ZKRegistrationManager;
 import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
+import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand;
+import org.apache.bookkeeper.tools.cli.commands.cluster.ListBookiesCommand;
 import org.apache.bookkeeper.util.EntryFormatter;
 import org.apache.bookkeeper.util.LedgerIdFormatter;
 import org.apache.bookkeeper.versioning.LongVersion;
@@ -73,8 +81,30 @@
     private Cookie cookie;
     private Version version;
 
+    // commands
+    private LastMarkCommand mockLastMarkCommand;
+    private SimpleTestCommand mockSimpleTestCommand;
+    private ListBookiesCommand mockListBookiesCommand;
+
     @Before
     public void setup() throws Exception {
+        // setup the required mocks before constructing bookie shell.
+        this.mockLastMarkCommand = mock(LastMarkCommand.class);
+        whenNew(LastMarkCommand.class)
+            .withNoArguments()
+            .thenReturn(mockLastMarkCommand);
+        this.mockSimpleTestCommand = spy(new SimpleTestCommand());
+        doNothing().when(mockSimpleTestCommand).run(any(ServerConfiguration.class));
+        whenNew(SimpleTestCommand.class)
+            .withNoArguments()
+            .thenReturn(mockSimpleTestCommand);
+        this.mockListBookiesCommand = spy(new ListBookiesCommand());
+        doNothing().when(mockListBookiesCommand).run(any(ServerConfiguration.class));
+        whenNew(ListBookiesCommand.class)
+            .withNoArguments()
+            .thenReturn(mockListBookiesCommand);
+
+        // construct the bookie shell.
         this.shell = new BookieShell(LedgerIdFormatter.LONG_LEDGERID_FORMATTER, EntryFormatter.STRING_FORMATTER);
         this.admin = PowerMockito.mock(BookKeeperAdmin.class);
         whenNew(BookKeeperAdmin.class)
@@ -280,4 +310,66 @@ void testRecoverCmdRecover(boolean dryrun,
                 .withNoArguments();
         }
     }
+
+    @Test
+    public void testLastMarkCmd() throws Exception {
+        shell.run(new String[] { "lastmark"});
+        verifyNew(LastMarkCommand.class, times(1)).withNoArguments();
+        verify(mockLastMarkCommand, times(1)).run(same(shell.bkConf));
+    }
+
+    @Test
+    public void testSimpleTestCmd() throws Exception {
+        shell.run(new String[] {
+            "simpletest",
+            "-e", "10",
+            "-w", "5",
+            "-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));
+    }
+
+    @Test
+    public void testListBookiesCmdNoArgs() throws Exception {
+        assertEquals(1, shell.run(new String[] {
+            "listbookies"
+        }));
+        verifyNew(ListBookiesCommand.class, times(0)).withNoArguments();
+    }
+
+    @Test
+    public void testListBookiesCmdConflictArgs() throws Exception {
+        assertEquals(1, shell.run(new String[] {
+            "listbookies", "-rw", "-ro"
+        }));
+        verifyNew(ListBookiesCommand.class, times(0)).withNoArguments();
+    }
+
+    @Test
+    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);
+    }
+
+    @Test
+    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);
+    }
 }
diff --git a/bookkeeper-tools/pom.xml b/bookkeeper-tools/pom.xml
new file mode 100644
index 000000000..5816e7e16
--- /dev/null
+++ b/bookkeeper-tools/pom.xml
@@ -0,0 +1,55 @@
+<?xml version="1.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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>bookkeeper</artifactId>
+    <groupId>org.apache.bookkeeper</groupId>
+    <version>4.7.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>bookkeeper-tools</artifactId>
+  <name>Apache BookKeeper :: Tools</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.libdir>${basedir}/lib</project.libdir>
+  </properties>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-server</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.beust</groupId>
+      <artifactId>jcommander</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>buildtools</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-server</artifactId>
+      <type>test-jar</type>
+      <version>${project.parent.version}</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>
diff --git a/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/BookKeeperCLI.java b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/BookKeeperCLI.java
new file mode 100644
index 000000000..3bc0888ca
--- /dev/null
+++ b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/BookKeeperCLI.java
@@ -0,0 +1,212 @@
+/*
+ * 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/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBase.java b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBase.java
new file mode 100644
index 000000000..1b3da3e0b
--- /dev/null
+++ b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBase.java
@@ -0,0 +1,95 @@
+/*
+ * 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/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBookie.java b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBookie.java
new file mode 100644
index 000000000..0c01ccb24
--- /dev/null
+++ b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdBookie.java
@@ -0,0 +1,34 @@
+/*
+ * 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;
+import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
+
+/**
+ * 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());
+    }
+}
diff --git a/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdClient.java b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdClient.java
new file mode 100644
index 000000000..d57e1afbe
--- /dev/null
+++ b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdClient.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand;
+
+/**
+ * Client operations that interacts with a cluster, such as simpletest.
+ */
+@Parameters(commandDescription = "Commands that interact with a cluster")
+public class CmdClient extends CmdBase {
+
+    public CmdClient(ServerConfiguration conf) {
+        super("client", conf);
+        addSubCommand(new SimpleTestCommand());
+    }
+}
diff --git a/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdCluster.java b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdCluster.java
new file mode 100644
index 000000000..4ba1cb4c8
--- /dev/null
+++ b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdCluster.java
@@ -0,0 +1,34 @@
+/*
+ * 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;
+import org.apache.bookkeeper.tools.cli.commands.cluster.ListBookiesCommand;
+
+/**
+ * Commands that interact with a cluster, such as format.
+ */
+@Parameters(commandDescription = "Commands that operate a cluster")
+public class CmdCluster extends CmdBase {
+    public CmdCluster(ServerConfiguration conf) {
+        super("cluster", conf);
+        addSubCommand(new ListBookiesCommand());
+    }
+}
diff --git a/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdMetadata.java b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdMetadata.java
new file mode 100644
index 000000000..7fa97947d
--- /dev/null
+++ b/bookkeeper-tools/src/main/java/org/apache/bookkeeper/tools/cli/commands/CmdMetadata.java
@@ -0,0 +1,32 @@
+/*
+ * 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/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/BookKeeperCLITest.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/BookKeeperCLITest.java
new file mode 100644
index 000000000..7a0affe60
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/BookKeeperCLITest.java
@@ -0,0 +1,259 @@
+/*
+ * 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/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/CmdBaseTest.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/CmdBaseTest.java
new file mode 100644
index 000000000..8d7deb90b
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/CmdBaseTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java
new file mode 100644
index 000000000..c5c366a8e
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.bookie;
+
+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.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+import java.io.File;
+import org.apache.bookkeeper.bookie.Journal;
+import org.apache.bookkeeper.bookie.Journal.LastLogMark;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LogMark;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test of {@link LastMarkCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ LastMarkCommand.class })
+public class LastMarkCommandTest extends BookieCommandTestBase {
+
+    private Journal journal;
+    private LastLogMark lastLogMark;
+    private LogMark logMark;
+
+    public LastMarkCommandTest() {
+        super(3, 0);
+    }
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+
+        PowerMockito.whenNew(LedgerDirsManager.class)
+            .withParameterTypes(
+                ServerConfiguration.class,
+                File[].class,
+                DiskChecker.class)
+            .withArguments(
+                eq(conf),
+                any(File[].class),
+                any(DiskChecker.class))
+            .thenReturn(mock(LedgerDirsManager.class));
+
+        this.journal = mock(Journal.class);
+        this.lastLogMark = mock(LastLogMark.class);
+        this.logMark = mock(LogMark.class);
+        when(lastLogMark.getCurMark()).thenReturn(logMark);
+        when(journal.getLastLogMark()).thenReturn(lastLogMark);
+        PowerMockito.whenNew(Journal.class)
+            .withParameterTypes(
+                File.class,
+                ServerConfiguration.class,
+                LedgerDirsManager.class)
+            .withArguments(
+                any(File.class),
+                eq(conf),
+                any(LedgerDirsManager.class))
+            .thenReturn(journal);
+    }
+
+    @Test
+    public void testCommand() throws Exception {
+        CommandRunner runner = createCommandRunner(new LastMarkCommand());
+
+        runner.runArgs("lastmark");
+
+        PowerMockito.verifyNew(LedgerDirsManager.class, times(1))
+            .withArguments(eq(conf), any(File[].class), any(DiskChecker.class));
+        PowerMockito.verifyNew(Journal.class, times(3))
+            .withArguments(any(File.class), eq(conf), any(LedgerDirsManager.class));
+        verify(journal, times(3)).getLastLogMark();
+        verify(lastLogMark, times(3)).getCurMark();
+        verify(logMark, times(3 * 2)).getLogFileId();
+        verify(logMark, times(3)).getLogFileOffset();
+    }
+
+
+
+
+}
diff --git a/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java
new file mode 100644
index 000000000..946afc552
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.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;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.bookkeeper.client.api.CreateBuilder;
+import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.WriteHandle;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.tools.cli.helpers.ClientCommandTestBase;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link SimpleTestCommand}.
+ */
+public class SimpleTestCommandTest extends ClientCommandTestBase {
+
+    @Test
+    public void testCommandShortArgs() {
+        testCommand(
+            "simpletest",
+            "-e", "5",
+            "-w", "3",
+            "-a", "3",
+            "-n", "10");
+    }
+
+    @Test
+    public void testCommandLongArgs() {
+        testCommand(
+            "simpletest",
+            "--ensemble-size", "5",
+            "--write-quorum-size", "3",
+            "--ack-quorum-size", "3",
+            "--num-entries", "10");
+    }
+
+    public void testCommand(String... args) {
+        WriteHandle wh = mock(WriteHandle.class);
+        AtomicLong counter = new AtomicLong(0L);
+        when(wh.append(any(byte[].class))).thenReturn(FutureUtils.value(counter.get()));
+        CreateBuilder createBuilder = mock(CreateBuilder.class);
+        when(createBuilder.execute())
+            .thenReturn(FutureUtils.value(wh));
+        when(createBuilder.withEnsembleSize(anyInt())).thenReturn(createBuilder);
+        when(createBuilder.withWriteQuorumSize(anyInt())).thenReturn(createBuilder);
+        when(createBuilder.withAckQuorumSize(anyInt())).thenReturn(createBuilder);
+        when(createBuilder.withDigestType(any(DigestType.class))).thenReturn(createBuilder);
+        when(createBuilder.withPassword(any(byte[].class))).thenReturn(createBuilder);
+        when(mockBk.newCreateLedgerOp()).thenReturn(createBuilder);
+
+        CommandRunner runner = createCommandRunner(new SimpleTestCommand());
+        assertTrue(runner.runArgs(args));
+
+        // verify create builder
+        verify(createBuilder, times(1)).withEnsembleSize(eq(5));
+        verify(createBuilder, times(1)).withWriteQuorumSize(eq(3));
+        verify(createBuilder, times(1)).withAckQuorumSize(eq(3));
+        verify(createBuilder, times(1)).withDigestType(eq(DigestType.CRC32C));
+        verify(createBuilder, times(1)).withPassword(eq(new byte[0]));
+        verify(createBuilder, times(1)).execute();
+
+        // verify appends
+        verify(wh, times(10)).append(eq(new byte[100]));
+    }
+
+}
diff --git a/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommandTest.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommandTest.java
new file mode 100644
index 000000000..2574c6368
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/commands/cluster/ListBookiesCommandTest.java
@@ -0,0 +1,186 @@
+/*
+ * 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.cluster;
+
+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.mockito.Answers.CALLS_REAL_METHODS;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+
+import com.google.common.primitives.UnsignedBytes;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.tools.cli.helpers.CommandHelpers;
+import org.apache.bookkeeper.tools.cli.helpers.DiscoveryCommandTestBase;
+import org.apache.bookkeeper.versioning.LongVersion;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test of {@link ListBookiesCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ ListBookiesCommand.class, CommandHelpers.class })
+public class ListBookiesCommandTest extends DiscoveryCommandTestBase {
+
+    private static class BookieAddressComparator implements Comparator<BookieSocketAddress> {
+
+        @Override
+        public int compare(BookieSocketAddress o1, BookieSocketAddress o2) {
+            int ret = UnsignedBytes.lexicographicalComparator()
+                .compare(o1.getHostName().getBytes(UTF_8), o2.getHostName().getBytes(UTF_8));
+            if (ret == 0) {
+                return Integer.compare(o1.getPort(), o2.getPort());
+            } else {
+                return ret;
+            }
+        }
+    }
+
+    private Set<BookieSocketAddress> writableBookies;
+    private Set<BookieSocketAddress> readonlyBookies;
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+
+        writableBookies = createBookies(3181, 10);
+        readonlyBookies = createBookies(4181, 10);
+
+        when(regClient.getWritableBookies())
+            .thenReturn(value(new Versioned<>(writableBookies, new LongVersion(0L))));
+        when(regClient.getReadOnlyBookies())
+            .thenReturn(value(new Versioned<>(readonlyBookies, new LongVersion(0L))));
+
+        PowerMockito.mockStatic(CommandHelpers.class, CALLS_REAL_METHODS);
+    }
+
+    private static Set<BookieSocketAddress> createBookies(int startPort, int numBookies) {
+        Set<BookieSocketAddress> bookies = new TreeSet<>(new BookieAddressComparator());
+        for (int i = 0; i < numBookies; i++) {
+            bookies.add(new BookieSocketAddress("127.0.0.1", startPort + i));
+        }
+        return bookies;
+    }
+
+    private static void verifyPrintBookies(int startPort, int numBookies, int numCalls) {
+        for (int i = 0; i < numBookies; i++) {
+            PowerMockito.verifyStatic(
+                CommandHelpers.class,
+                times(numCalls));
+            CommandHelpers.getBookieSocketAddrStringRepresentation(
+                eq(new BookieSocketAddress("127.0.0.1", startPort + 1)));
+        }
+    }
+
+    @Test
+    public void testListReadWriteShortArgs() {
+        testCommand(true, false,
+            "listbookies",
+            "-rw");
+    }
+
+    @Test
+    public void testListReadWriteLongArgs() {
+        testCommand(true, false,
+            "listbookies",
+            "--readwrite");
+    }
+
+    @Test
+    public void testListReadOnlyShortArgs() {
+        testCommand(false, true,
+            "listbookies",
+            "-ro");
+    }
+
+    @Test
+    public void testListReadOnlyLongArgs() {
+        testCommand(false, true,
+            "listbookies",
+            "--readonly");
+    }
+
+    @Test
+    public void testListNoArgs() {
+        testCommand(true, true,
+            "listbookies");
+    }
+
+    @Test
+    public void testListTwoFlagsCoexistsShortArgs() {
+        testCommand(true, true,
+            "listbookies", "-rw", "-ro");
+    }
+
+    @Test
+    public void testListTwoFlagsCoexistsLongArgs() {
+        testCommand(true, true,
+            "listbookies", "--readwrite", "--readonly");
+    }
+
+    private void testCommand(boolean readwrite,
+                             boolean readonly,
+                             String... args) {
+
+        CommandRunner runner = createCommandRunner(new ListBookiesCommand());
+        assertTrue(runner.runArgs(args));
+
+        if (readwrite && !readonly) {
+            verifyPrintBookies(3181, 10,1);
+            verifyPrintBookies(4181, 10,0);
+        } else if (readonly && !readwrite) {
+            verifyPrintBookies(3181, 10,0);
+            verifyPrintBookies(4181, 10,1);
+        } else {
+            verifyPrintBookies(3181, 10,1);
+            verifyPrintBookies(4181, 10,1);
+        }
+    }
+
+    @Test
+    public void testListEmptyBookies() {
+        // 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"));
+
+        PowerMockito.verifyStatic(
+                CommandHelpers.class,
+                times(0));
+        CommandHelpers.getBookieSocketAddrStringRepresentation(any());
+    }
+
+}
diff --git a/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommandTestBase.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommandTestBase.java
new file mode 100644
index 000000000..6bd3f5453
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommandTestBase.java
@@ -0,0 +1,64 @@
+/*
+ * 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 java.io.File;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * A test base for testing bookie commands.
+ */
+public abstract class BookieCommandTestBase extends CommandTestBase {
+
+    @Rule
+    public final TemporaryFolder testDir = new TemporaryFolder();
+
+    protected final int numJournalDirs;
+    protected final int numLedgerDirs;
+
+    protected BookieCommandTestBase(int numJournalDirs, int numLedgerDirs) {
+        this.numJournalDirs = numJournalDirs;
+        this.numLedgerDirs = numLedgerDirs;
+    }
+
+    @Before
+    public void setup() throws Exception {
+        String[] journalDirs = new String[numJournalDirs];
+        if (numJournalDirs > 0) {
+            for (int i = 0; i < numJournalDirs; i++) {
+                File dir = testDir.newFile();
+                dir.mkdirs();
+                journalDirs[i] = dir.getAbsolutePath();
+            }
+            conf.setJournalDirsName(journalDirs);
+        }
+        String[] ledgerDirs = new String[numLedgerDirs];
+        if (numLedgerDirs > 0) {
+            for (int i = 0; i < numLedgerDirs; i++) {
+                File dir = testDir.newFile();
+                dir.mkdirs();
+                ledgerDirs[i] = dir.getAbsolutePath();
+            }
+            conf.setLedgerDirNames(ledgerDirs);
+        }
+    }
+
+}
diff --git a/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java
new file mode 100644
index 000000000..07f3cb8dd
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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 static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.CALLS_REAL_METHODS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.bookkeeper.client.api.BookKeeper;
+import org.apache.bookkeeper.client.api.BookKeeperBuilder;
+import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test of {@link ClientCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ ClientCommand.class, BookKeeper.class })
+public class ClientCommandTest {
+
+    private ClientCommand cmd;
+    private ServerConfiguration serverConf;
+    private ClientConfiguration clientConf;
+    private BookKeeperBuilder bkBuilder;
+    private BookKeeper bk;
+
+    @Before
+    public void setup() throws Exception {
+        this.cmd = mock(ClientCommand.class, CALLS_REAL_METHODS);
+
+        this.serverConf = new ServerConfiguration();
+        this.clientConf = new ClientConfiguration(serverConf);
+        PowerMockito.whenNew(ClientConfiguration.class)
+            .withParameterTypes(AbstractConfiguration.class)
+            .withArguments(eq(serverConf))
+            .thenReturn(clientConf);
+        PowerMockito.mockStatic(BookKeeper.class);
+        this.bkBuilder = mock(BookKeeperBuilder.class, CALLS_REAL_METHODS);
+        this.bk = mock(BookKeeper.class);
+        PowerMockito.when(
+            BookKeeper.class, "newBuilder", eq(clientConf))
+            .thenReturn(bkBuilder);
+        when(bkBuilder.build()).thenReturn(bk);
+    }
+
+    @Test
+    public void testRun() throws Exception {
+        cmd.run(serverConf);
+        verify(cmd, times(1)).run(eq(clientConf));
+        verify(cmd, times(1)).run(eq(bk));
+        verify(bkBuilder, times(1)).build();
+    }
+
+}
diff --git a/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java
new file mode 100644
index 000000000..1d269f5ec
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java
@@ -0,0 +1,65 @@
+/*
+ * 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 static org.mockito.Answers.CALLS_REAL_METHODS;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.spy;
+
+import org.apache.bookkeeper.client.api.BookKeeper;
+import org.apache.bookkeeper.client.api.BookKeeperBuilder;
+import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.junit.Before;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * A test base for testing client commands.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ ClientCommand.class, BookKeeper.class })
+public abstract class ClientCommandTestBase extends CommandTestBase {
+
+    protected ClientConfiguration clientConf;
+    protected BookKeeperBuilder mockBkBuilder;
+    protected BookKeeper mockBk;
+
+    @Before
+    public void setup() throws Exception {
+        mockBk = mock(BookKeeper.class);
+        this.clientConf = spy(new ClientConfiguration(conf));
+        PowerMockito.whenNew(ClientConfiguration.class)
+            .withParameterTypes(AbstractConfiguration.class)
+            .withArguments(eq(conf))
+            .thenReturn(clientConf);
+        PowerMockito.mockStatic(BookKeeper.class);
+        this.mockBkBuilder = mock(BookKeeperBuilder.class, CALLS_REAL_METHODS);
+        this.mockBk = mock(BookKeeper.class);
+        PowerMockito.when(
+            BookKeeper.class, "newBuilder", eq(clientConf))
+            .thenReturn(mockBkBuilder);
+        when(mockBkBuilder.build()).thenReturn(mockBk);
+    }
+
+}
diff --git a/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java
new file mode 100644
index 000000000..670942227
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java
@@ -0,0 +1,55 @@
+/*
+ * 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 lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.cli.commands.CmdBase;
+
+/**
+ * A test base providing an environment for run a command.
+ */
+@Slf4j
+public class CommandTestBase {
+
+    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 ServerConfiguration conf;
+
+    public CommandTestBase() {
+        this.conf = new ServerConfiguration();
+    }
+
+}
diff --git a/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java
new file mode 100644
index 000000000..1677309e1
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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 static org.mockito.Answers.CALLS_REAL_METHODS;
+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.mock;
+
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.bookkeeper.client.api.BookKeeper;
+import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationClient;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test of {@link DiscoveryCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ DiscoveryCommand.class, ReflectionUtils.class })
+public class DiscoveryCommandTest {
+
+    private DiscoveryCommand cmd;
+    private ServerConfiguration serverConf;
+    private ClientConfiguration clientConf;
+    private RegistrationClient regClient;
+    private ScheduledExecutorService executor;
+
+    @Before
+    public void setup() throws Exception {
+        PowerMockito.mockStatic(Executors.class);
+        PowerMockito.mockStatic(ReflectionUtils.class);
+
+        this.cmd = mock(DiscoveryCommand.class, CALLS_REAL_METHODS);
+
+        this.serverConf = new ServerConfiguration();
+        this.clientConf = new ClientConfiguration(serverConf);
+        PowerMockito.whenNew(ClientConfiguration.class)
+            .withParameterTypes(AbstractConfiguration.class)
+            .withArguments(eq(serverConf))
+            .thenReturn(clientConf);
+
+        this.executor = mock(ScheduledExecutorService.class);
+        PowerMockito.when(Executors.newSingleThreadScheduledExecutor())
+            .thenReturn(executor);
+
+        this.regClient = mock(RegistrationClient.class);
+        PowerMockito.when(ReflectionUtils.newInstance(any()))
+            .thenReturn(regClient);
+    }
+
+    @Test
+    public void testRun() throws Exception {
+        cmd.run(serverConf);
+        verify(cmd, times(1)).run(eq(regClient));
+        verify(regClient, times(1))
+            .initialize(eq(clientConf), eq(executor), eq(NullStatsLogger.INSTANCE), eq(Optional.empty()));
+        verify(regClient, times(1)).close();
+        verify(executor, times(1)).shutdown();
+    }
+
+}
diff --git a/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTestBase.java b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTestBase.java
new file mode 100644
index 000000000..7f3588ebc
--- /dev/null
+++ b/bookkeeper-tools/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTestBase.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.helpers;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.RegistrationClient;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * A test base for discovery related commands.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ DiscoveryCommand.class, ReflectionUtils.class })
+public abstract class DiscoveryCommandTestBase extends ClientCommandTestBase {
+
+    protected RegistrationClient regClient;
+    protected ScheduledExecutorService executor;
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+
+        PowerMockito.mockStatic(Executors.class);
+        PowerMockito.mockStatic(ReflectionUtils.class);
+
+        this.executor = mock(ScheduledExecutorService.class);
+        PowerMockito.when(Executors.newSingleThreadScheduledExecutor())
+            .thenReturn(executor);
+
+        this.regClient = mock(RegistrationClient.class);
+        PowerMockito.when(ReflectionUtils.newInstance(any()))
+            .thenReturn(regClient);
+    }
+
+}
diff --git a/buildtools/src/main/resources/log4j.properties b/buildtools/src/main/resources/log4j.properties
new file mode 100644
index 000000000..10ae6bfcb
--- /dev/null
+++ b/buildtools/src/main/resources/log4j.properties
@@ -0,0 +1,42 @@
+#
+#
+# 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.
+#
+#
+
+#
+# Bookkeeper Logging Configuration
+#
+
+# Format is "<default threshold> (, <appender>)+
+
+# DEFAULT: console appender only, level INFO
+bookkeeper.root.logger=INFO,CONSOLE
+log4j.rootLogger=${bookkeeper.root.logger}
+
+#
+# Log INFO level and above messages to the console
+#
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+#disable zookeeper logging
+log4j.logger.org.apache.zookeeper=OFF
+log4j.logger.org.apache.bookkeeper.bookie=INFO
+log4j.logger.org.apache.bookkeeper.meta=INFO
diff --git a/conf/bk_cli_env.sh b/conf/bk_cli_env.sh
new file mode 100644
index 000000000..323009ff9
--- /dev/null
+++ b/conf/bk_cli_env.sh
@@ -0,0 +1,51 @@
+#!/bin/sh
+#
+#/**
+# * Copyright 2007 The Apache Software Foundation
+# *
+# * 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.
+# */
+
+# Set JAVA_HOME here to override the environment setting
+# JAVA_HOME=
+
+# default settings for bookkeeper cli
+
+# Configuration file of settings used in bookkeeper cli
+# CLI_CONF=
+
+# Extra options to be passed to the jvm
+# CLI_EXTRA_OPTS=
+
+# Add extra paths to the bookkeeper classpath
+# CLI_EXTRA_CLASSPATH=
+
+#
+# CLI Logging Options
+#
+
+# Log4j configuration file
+# CLI_LOG_CONF=
+
+# Logs location
+# CLI_LOG_DIR=
+
+# Log file name
+# CLI_LOG_FILE="bookkeeper-cli.log"
+
+# Log level & appender
+# CLI_ROOT_LOGGER="INFO,CONSOLE"
diff --git a/conf/log4j.cli.properties b/conf/log4j.cli.properties
new file mode 100644
index 000000000..a9d11d80c
--- /dev/null
+++ b/conf/log4j.cli.properties
@@ -0,0 +1,58 @@
+#
+#
+# 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.
+#
+#
+
+#
+# BookKeeper CLI configuration
+
+# DEFAULT: console appender only
+# Define some default values that can be overridden by system properties
+bookkeeper.cli.root.logger=ERROR,CONSOLE
+bookkeeper.cli.log.dir=logs
+bookkeeper.cli.log.file=bookkeeper-cli.log
+
+log4j.rootLogger=${bookkeeper.cli.root.logger}
+
+#
+# Log INFO level and above messages to the console
+#
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.Threshold=INFO
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ABSOLUTE} %-5p %m%n
+
+# verbose console logging
+log4j.appender.VERBOSECONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.VERBOSECONSOLE.Threshold=INFO
+log4j.appender.VERBOSECONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.VERBOSECONSOLE.layout.ConversionPattern=%m%n
+
+# file logging
+log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.ROLLINGFILE.Threshold=INFO
+log4j.appender.ROLLINGFILE.File=${bookkeeper.cli.log.dir}/${bookkeeper.cli.log.file}
+log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
+log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+log4j.logger.verbose=INFO,VERBOSECONSOLE
+log4j.logger.org.apache.zookeeper=ERROR
+log4j.logger.org.apache.bookkeeper=ERROR
+log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO
+log4j.logger.org.apache.bookkeeper.client.BookKeeperAdmin=INFO
diff --git a/pom.xml b/pom.xml
index 8565fef5f..4c1c851d9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -57,6 +57,7 @@
     <module>bookkeeper-stats</module>
     <module>bookkeeper-proto</module>
     <module>bookkeeper-server</module>
+    <module>bookkeeper-tools</module>
     <module>bookkeeper-benchmark</module>
     <module>bookkeeper-stats-providers</module>
     <module>bookkeeper-http</module>
@@ -107,6 +108,7 @@
     <google.errorprone.version>2.1.2</google.errorprone.version>
     <guava.version>20.0</guava.version>
     <hamcrest.version>1.3</hamcrest.version>
+    <jcommander.version>1.48</jcommander.version>
     <jmh.version>1.19</jmh.version>
     <junit.version>4.12</junit.version>
     <lombok.version>1.16.18</lombok.version>
@@ -133,6 +135,16 @@
     <maven-assembly-plugin.version>2.2.1</maven-assembly-plugin.version>
   </properties>
 
+  <dependencyManagement>
+    <dependencies>
+      <dependency>
+        <groupId>com.beust</groupId>
+        <artifactId>jcommander</artifactId>
+        <version>${jcommander.version}</version>
+      </dependency>
+    </dependencies>
+  </dependencyManagement>
+
   <!-- dependencies for all modules -->
   <dependencies>
     <!-- compilation dependencies -->


 

----------------------------------------------------------------
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