This is an automated email from the ASF dual-hosted git repository.
sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/master by this push:
new 969e729 BP-27 (part 1): New BookKeeper CLI Skeleton
969e729 is described below
commit 969e7292af5a1b4f26cc06c7cacd06b1746224f4
Author: Sijie Guo <[email protected]>
AuthorDate: Sat Feb 17 01:22:28 2018 +0800
BP-27 (part 1): New BookKeeper CLI Skeleton
Descriptions of the changes in this PR:
This is first implementation of BP-27 (#1000). This change includes:
- introduce a bookkeeper-tools module for keep the commands and also for
developing the new CLI
- introduce 4 command groups: `cluster`, `bookie`, `client`, `metadata`.
- move 3 command: `listbookies` under `cluster` group, `lastmark` under
`bookie` group, `simpletest` under `client` group.
- unit tests for those 3 command.
Author: Sijie Guo <[email protected]>
Reviewers: Charan Reddy Guttapalem <[email protected]>, Yiming Zang
<[email protected]>
This closes #1094 from sijie/bookie_shell_refactor
---
bin/bookkeeper-cli | 155 ++++++++++++
.../src/main/resources/LICENSE-all.bin.txt | 2 +
.../src/main/resources/LICENSE-server.bin.txt | 2 +
.../src/main/resources/NOTICE-all.bin.txt | 5 +
.../src/main/resources/NOTICE-server.bin.txt | 5 +
bookkeeper-server/pom.xml | 4 +
.../org/apache/bookkeeper/bookie/BookieShell.java | 84 ++-----
.../tools/cli/commands/bookie/LastMarkCommand.java | 61 +++++
.../tools/cli/commands/bookie/package-info.java | 23 ++
.../cli/commands/client/SimpleTestCommand.java | 80 +++++++
.../tools/cli/commands/client/package-info.java | 23 ++
.../cli/commands/cluster/ListBookiesCommand.java | 91 ++++++++
.../tools/cli/commands/cluster/package-info.java | 23 ++
.../tools/cli/commands/package-info.java | 23 ++
.../tools/cli/helpers/BookieCommand.java | 25 ++
.../tools/cli/helpers/ClientCommand.java | 48 ++++
.../bookkeeper/tools/cli/helpers/Command.java | 52 +++++
.../tools/cli/helpers/CommandHelpers.java | 53 +++++
.../tools/cli/helpers/DiscoveryCommand.java | 60 +++++
.../bookkeeper/tools/cli/helpers/package-info.java | 24 ++
.../apache/bookkeeper/tools/cli/package-info.java | 26 +++
.../apache/bookkeeper/bookie/BookieShellTest.java | 92 ++++++++
bookkeeper-tools/pom.xml | 55 +++++
.../apache/bookkeeper/tools/cli/BookKeeperCLI.java | 212 +++++++++++++++++
.../bookkeeper/tools/cli/commands/CmdBase.java | 95 ++++++++
.../bookkeeper/tools/cli/commands/CmdBookie.java | 34 +++
.../bookkeeper/tools/cli/commands/CmdClient.java | 35 +++
.../bookkeeper/tools/cli/commands/CmdCluster.java | 34 +++
.../bookkeeper/tools/cli/commands/CmdMetadata.java | 32 +++
.../bookkeeper/tools/cli/BookKeeperCLITest.java | 259 +++++++++++++++++++++
.../bookkeeper/tools/cli/commands/CmdBaseTest.java | 125 ++++++++++
.../cli/commands/bookie/LastMarkCommandTest.java | 109 +++++++++
.../cli/commands/client/SimpleTestCommandTest.java | 92 ++++++++
.../commands/cluster/ListBookiesCommandTest.java | 186 +++++++++++++++
.../tools/cli/helpers/BookieCommandTestBase.java | 64 +++++
.../tools/cli/helpers/ClientCommandTest.java | 80 +++++++
.../tools/cli/helpers/ClientCommandTestBase.java | 65 ++++++
.../tools/cli/helpers/CommandTestBase.java | 55 +++++
.../tools/cli/helpers/DiscoveryCommandTest.java | 91 ++++++++
.../cli/helpers/DiscoveryCommandTestBase.java | 62 +++++
buildtools/src/main/resources/log4j.properties | 42 ++++
conf/bk_cli_env.sh | 51 ++++
conf/log4j.cli.properties | 58 +++++
pom.xml | 12 +
44 files changed, 2742 insertions(+), 67 deletions(-)
diff --git a/bin/bookkeeper-cli b/bin/bookkeeper-cli
new file mode 100755
index 0000000..722fe5d
--- /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 3b2246a..e1d0dcf 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 2261063..74e9861 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 63ae3e9..7357414 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 [email protected]
+
+------------------------------------------------------------------------------------
diff --git a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt
b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt
index c6fd7ab..905c4c1 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 [email protected]
+
+------------------------------------------------------------------------------------
diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml
index ba6f248..548216e 100644
--- a/bookkeeper-server/pom.xml
+++ b/bookkeeper-server/pom.xml
@@ -137,6 +137,10 @@
<version>4.1</version>
</dependency>
<dependency>
+ <groupId>com.beust</groupId>
+ <artifactId>jcommander</artifactId>
+ </dependency>
+ <dependency>
<groupId>net.java.dev.jna</groupId>
<artifactId>jna</artifactId>
<version>3.2.7</version>
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 477f66b..c9e4d93 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.BookieClient;
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 @@ public class BookieShell implements Tool {
@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 @@ public class BookieShell implements Tool {
@Override
public int runCmd(CommandLine c) throws Exception {
- printLastLogMark();
+ LastMarkCommand command = new LastMarkCommand();
+ command.run(bkConf);
return 0;
}
@@ -1482,29 +1471,11 @@ public class BookieShell implements Tool {
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 class BookieShell implements Tool {
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 0000000..a83f195
--- /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 0000000..429cc4e
--- /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 0000000..60633eb
--- /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 0000000..5b321b5
--- /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 0000000..8ec872e
--- /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 0000000..cc32291
--- /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 0000000..737cd34
--- /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 0000000..91967c0
--- /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 0000000..c7a65d8
--- /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 0000000..1ce6093
--- /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 0000000..88ac52e
--- /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 0000000..1ebfd88
--- /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 0000000..68c3d70
--- /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 0000000..90ad08d
--- /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 4a1658e..b77b5e9 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.junit.Assert.fail;
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.conf.ServerConfiguration;
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 @@ public class BookieShellTest {
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 @@ public class BookieShellTest {
.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 0000000..5816e7e
--- /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 0000000..3bc0888
--- /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 0000000..1b3da3e
--- /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 0000000..0c01ccb
--- /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 0000000..d57e1af
--- /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 0000000..4ba1cb4
--- /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 0000000..7fa9794
--- /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 0000000..7a0affe
--- /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 0000000..8d7deb9
--- /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 0000000..c5c366a
--- /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 0000000..946afc5
--- /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 0000000..2574c63
--- /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 0000000..6bd3f54
--- /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 0000000..07f3cb8
--- /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 0000000..1d269f5
--- /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 0000000..6709422
--- /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 0000000..1677309
--- /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 0000000..7f3588e
--- /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 0000000..10ae6bf
--- /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 0000000..323009f
--- /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 0000000..a9d11d8
--- /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 bfe43bc..0171d09 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>
@@ -108,6 +109,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>
@@ -134,6 +136,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 -->
--
To stop receiving notification emails like this one, please contact
[email protected].