You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2019/03/15 06:56:38 UTC

[bookkeeper] branch master updated: Issue #1980: Migrate command `ledger` from shell to bkctl

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 fa90f01  Issue #1980: Migrate command `ledger` from shell to bkctl
fa90f01 is described below

commit fa90f0185c51e8f842148945e4958cbc02cbe023
Author: Yong Zhang <zh...@gmail.com>
AuthorDate: Fri Mar 15 14:56:33 2019 +0800

    Issue #1980: Migrate command `ledger` from shell to bkctl
    
    Descriptions of the changes in this PR:
    
    Migrate command `ledger` from shell to bkctl
    
    ### Motivation
    
    #1980
    
    ### Changes
    
    - Add new implement for `ledger`
    
    
    Reviewers: Jia Zhai <zh...@apache.org>, Sijie Guo <si...@apache.org>
    
    This closes #1981 from zymap/command-ledger, closes #1980
---
 .../java/org/apache/bookkeeper/bookie/Bookie.java  |   5 +-
 .../org/apache/bookkeeper/bookie/BookieShell.java  |  97 +---------
 .../tools/cli/commands/bookie/LedgerCommand.java   | 202 +++++++++++++++++++++
 .../tools/cli/commands/BookieCommandGroup.java     |   2 +
 .../cli/commands/bookie/LedgerCommandTest.java     | 166 +++++++++++++++++
 5 files changed, 380 insertions(+), 92 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
index 16b6312..44c9509 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
@@ -625,9 +625,8 @@ public class Bookie extends BookieCriticalThread {
      * @return Passed ledgerStorage instance
      * @throws IOException
      */
-    static LedgerStorage mountLedgerStorageOffline(
-            ServerConfiguration conf,
-            LedgerStorage ledgerStorage) throws IOException {
+    public static LedgerStorage mountLedgerStorageOffline(ServerConfiguration conf, LedgerStorage ledgerStorage)
+            throws IOException {
         StatsLogger statsLogger = NullStatsLogger.INSTANCE;
         DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
 
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 4a75d56..026c900 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
@@ -112,6 +112,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
+import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand;
 import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand;
@@ -144,7 +145,6 @@ import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.mutable.MutableBoolean;
-import org.apache.commons.lang.mutable.MutableLong;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
@@ -717,96 +717,15 @@ public class BookieShell implements Tool {
 
         @Override
         public int runCmd(CommandLine cmdLine) throws Exception {
-            String[] leftArgs = cmdLine.getArgs();
-            if (leftArgs.length <= 0) {
-                printErrorLine("ERROR: missing ledger id");
-                printUsage();
-                return -1;
-            }
-
-            boolean printMeta = false;
+            LedgerCommand cmd = new LedgerCommand(ledgerIdFormatter);
+            cmd.setPrint(BookieShell.this::printInfoLine);
+            LedgerCommand.LedgerFlags flags = new LedgerCommand.LedgerFlags();
             if (cmdLine.hasOption("m")) {
-                printMeta = true;
-            }
-            long ledgerId;
-            try {
-                ledgerId = ledgerIdFormatter.readLedgerId(leftArgs[0]);
-            } catch (IllegalArgumentException iae) {
-                printErrorLine("ERROR: invalid ledger id " + leftArgs[0]);
-                printUsage();
-                return -1;
+                flags.meta(true);
             }
-
-            if (bkConf.getLedgerStorageClass().equals(DbLedgerStorage.class.getName())) {
-                // dump ledger info
-                try {
-                    DbLedgerStorage.readLedgerIndexEntries(ledgerId, bkConf,
-                            (currentEntry, entryLogId, position) -> printInfoLine(
-                                    "entry " + currentEntry + "\t:\t(log: " + entryLogId + ", pos: " + position + ")"));
-                } catch (IOException e) {
-                    System.err.printf("ERROR: initializing dbLedgerStorage %s", e.getMessage());
-                    return -1;
-                }
-            } else if ((bkConf.getLedgerStorageClass().equals(SortedLedgerStorage.class.getName())
-                    || bkConf.getLedgerStorageClass().equals(InterleavedLedgerStorage.class.getName()))) {
-                ServerConfiguration conf = new ServerConfiguration(bkConf);
-                InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage();
-                Bookie.mountLedgerStorageOffline(conf, interleavedStorage);
-
-                if (printMeta) {
-                    // print meta
-                    printInfoLine("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " =====");
-                    LedgerCache.LedgerIndexMetadata meta = interleavedStorage.readLedgerIndexMetadata(ledgerId);
-                    printInfoLine("master key  : " + meta.getMasterKeyHex());
-
-                    long size = meta.size;
-                    if (size % 8 == 0) {
-                        printInfoLine("size        : " + size);
-                    } else {
-                        printInfoLine("size : " + size
-                                + " (not aligned with 8, may be corrupted or under flushing now)");
-                    }
-
-                    printInfoLine("entries     : " + (size / 8));
-                    printInfoLine("isFenced    : " + meta.fenced);
-                }
-
-                try {
-                    // dump ledger info
-                    printInfoLine("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " =====");
-                    for (LedgerCache.PageEntries page : interleavedStorage.getIndexEntries(ledgerId)) {
-                        final MutableLong curEntry = new MutableLong(page.getFirstEntry());
-                        try (LedgerEntryPage lep = page.getLEP()){
-                            lep.getEntries((entry, offset) -> {
-                                while (curEntry.longValue() < entry) {
-                                    printInfoLine("entry " + curEntry + "\t:\tN/A");
-                                    curEntry.increment();
-                                }
-                                long entryLogId = offset >> 32L;
-                                long pos = offset & 0xffffffffL;
-                                printInfoLine("entry " + curEntry + "\t:\t(log:" + entryLogId + ", pos: " + pos + ")");
-                                curEntry.increment();
-                                return true;
-                            });
-                        } catch (IOException ie) {
-                            printInfoLine("Failed to read index page @ " + page.getFirstEntry()
-                                    + ", the index file may be corrupted : "
-                                    + ie.getMessage());
-                            return 1;
-                        }
-
-                        while (curEntry.longValue() < page.getLastEntry()) {
-                            printInfoLine("entry " + curEntry + "\t:\tN/A");
-                            curEntry.increment();
-                        }
-                    }
-                } catch (IOException ie) {
-                    LOG.error("Failed to read index page");
-                    return 1;
-                }
-            }
-
-            return 0;
+            flags.ledgerId(Long.parseLong(cmdLine.getArgs()[0]));
+            boolean result = cmd.apply(bkConf, flags);
+            return (result) ? 0 : 1;
         }
 
         @Override
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java
new file mode 100644
index 0000000..faaefcb
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java
@@ -0,0 +1,202 @@
+/*
+ * 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.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import lombok.Setter;
+import lombok.experimental.Accessors;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
+import org.apache.bookkeeper.bookie.LedgerCache;
+import org.apache.bookkeeper.bookie.LedgerEntryPage;
+import org.apache.bookkeeper.bookie.SortedLedgerStorage;
+import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.apache.bookkeeper.util.LedgerIdFormatter;
+import org.apache.commons.lang.mutable.MutableLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Command to dump ledger index entries into readable format.
+ */
+public class LedgerCommand extends BookieCommand<LedgerCommand.LedgerFlags> {
+
+    static final Logger LOG = LoggerFactory.getLogger(LedgerCommand.class);
+
+    private static final String NAME = "ledger";
+    private static final String DESC = "Dump ledger index entries into readable format";
+
+    private LedgerIdFormatter ledgerIdFormatter;
+
+    private Consumer<String> print = this::printInfoLine;
+
+    public void setPrint(Consumer<String> print) {
+        this.print = print;
+    }
+
+    public LedgerCommand() {
+        this(new LedgerFlags());
+    }
+
+    public LedgerCommand(LedgerIdFormatter ledgerIdFormatter) {
+        this(new LedgerFlags());
+        this.ledgerIdFormatter = ledgerIdFormatter;
+    }
+
+    private LedgerCommand(LedgerFlags flags) {
+        super(CliSpec.<LedgerFlags>newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build());
+    }
+
+    /**
+     * Flags for ledger command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class LedgerFlags extends CliFlags {
+
+        @Parameter(names = { "-id", "--ledgerId" }, description = "Specific ledger id", required = true)
+        private long ledgerId;
+
+        @Parameter(names = { "-m", "--meta" }, description = "Print meta information")
+        private boolean meta;
+
+        @Parameter(names = { "-l", "--ledgeridformatter" }, description = "Set ledger id formatter")
+        private String ledgerIdFormatter = "";
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, LedgerFlags cmdFlags) {
+        initLedgerIdFormatter(conf, cmdFlags);
+        long ledgerId = cmdFlags.ledgerId;
+        if (conf.getLedgerStorageClass().equals(DbLedgerStorage.class.getName())) {
+            // dump ledger info
+            if (!dumpLedgerInfo(ledgerId, conf)) {
+                return false;
+            }
+        } else if (conf.getLedgerStorageClass().equals(SortedLedgerStorage.class.getName())
+                || conf.getLedgerStorageClass().equals(InterleavedLedgerStorage.class.getName())) {
+            ServerConfiguration tConf = new ServerConfiguration(conf);
+            InterleavedLedgerStorage interleavedLedgerStorage = new InterleavedLedgerStorage();
+            try {
+                Bookie.mountLedgerStorageOffline(tConf, interleavedLedgerStorage);
+            } catch (IOException e) {
+                throw new UncheckedExecutionException(e.getMessage(), e);
+            }
+
+            if (cmdFlags.meta) {
+                // print meta
+                printMeta(ledgerId, interleavedLedgerStorage);
+            }
+
+            try {
+                print.accept("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " =====");
+                for (LedgerCache.PageEntries page : interleavedLedgerStorage.getIndexEntries(ledgerId)) {
+                    if (printPageEntries(page)) {
+                        return true;
+                    }
+                }
+            } catch (IOException e) {
+                LOG.error("Failed to read index page");
+                return true;
+            }
+
+        }
+        return true;
+    }
+
+    private void initLedgerIdFormatter(ServerConfiguration conf, LedgerFlags flags) {
+        if (flags.ledgerIdFormatter.equals("")) {
+            this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf);
+        } else {
+            this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(flags.ledgerIdFormatter, conf);
+        }
+    }
+
+    private boolean dumpLedgerInfo(long ledgerId, ServerConfiguration conf) {
+        try {
+            DbLedgerStorage.readLedgerIndexEntries(ledgerId, conf, (currentEntry, entryLodId, position) -> System.out
+                    .println("entry " + currentEntry + "\t:\t(log: " + entryLodId + ", pos: " + position + ")"));
+        } catch (IOException e) {
+            System.err.printf("ERROR: initializing dbLedgerStorage %s", e.getMessage());
+            return false;
+        }
+        return true;
+    }
+
+    private void printMeta(long ledgerId, InterleavedLedgerStorage interleavedLedgerStorage) {
+        print.accept("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " =====");
+        try {
+            LedgerCache.LedgerIndexMetadata meta = interleavedLedgerStorage.readLedgerIndexMetadata(ledgerId);
+            print.accept("master key  : " + meta.getMasterKeyHex());
+            long size = meta.size;
+            if (size % 8 == 0) {
+                print.accept("size         : " + size);
+            } else {
+                print.accept("size : " + size + "(not aligned with 8, may be corrupted or under flushing now)");
+            }
+
+            print.accept("entries      : " + (size / 8));
+            print.accept("isFenced     : " + meta.fenced);
+        } catch (IOException e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean printPageEntries(LedgerCache.PageEntries page) {
+        final MutableLong curEntry = new MutableLong(page.getFirstEntry());
+        try (LedgerEntryPage lep = page.getLEP()) {
+            lep.getEntries((entry, offset) -> {
+                while (curEntry.longValue() < entry) {
+                    print.accept("entry " + curEntry + "\t:\tN/A");
+                    curEntry.increment();
+                }
+                long entryLogId = offset >> 32L;
+                long pos = offset & 0xffffffffL;
+                print.accept("entry " + curEntry + "\t:\t(log:" + entryLogId + ", pos: " + pos + ")");
+                curEntry.increment();
+                return true;
+            });
+        } catch (Exception e) {
+            print.accept(
+                    "Failed to read index page @ " + page.getFirstEntry() + ", the index file may be corrupted : " + e
+                            .getMessage());
+            return true;
+        }
+
+        while (curEntry.longValue() < page.getLastEntry()) {
+            print.accept("entry " + curEntry + "\t:\tN/A");
+            curEntry.increment();
+        }
+
+        return false;
+    }
+
+
+    private void printInfoLine(String mes) {
+        System.out.println(mes);
+    }
+}
diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
index 5680759..e7daee6 100644
--- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
+++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
@@ -24,6 +24,7 @@ import org.apache.bookkeeper.tools.cli.BKCtl;
 import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
+import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand;
 import org.apache.bookkeeper.tools.common.BKFlags;
 import org.apache.bookkeeper.tools.framework.CliCommandGroup;
 import org.apache.bookkeeper.tools.framework.CliSpec;
@@ -44,6 +45,7 @@ public class BookieCommandGroup extends CliCommandGroup<BKFlags> {
         .addCommand(new LastMarkCommand())
         .addCommand(new InitCommand())
         .addCommand(new FormatCommand())
+        .addCommand(new LedgerCommand())
         .build();
 
     public BookieCommandGroup() {
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java
new file mode 100644
index 0000000..1023458
--- /dev/null
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.anyLong;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.verify;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+
+import java.util.Iterator;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
+import org.apache.bookkeeper.bookie.LedgerCache;
+import org.apache.bookkeeper.bookie.LedgerEntryPage;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.SortedLedgerStorage;
+import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
+import org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase;
+import org.junit.Assert;
+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 for {@link LedgerCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ DbLedgerStorage.class, SortedLedgerStorage.class, InterleavedLedgerStorage.class, Bookie.class,
+        LedgerStorage.class, LedgerCache.PageEntries.class, LedgerCache.PageEntriesIterable.class, LedgerCommand.class,
+        LedgerCache.LedgerIndexMetadata.class })
+public class LedgerCommandTest extends BookieCommandTestBase {
+
+    private LedgerCache.LedgerIndexMetadata metadata;
+    private ServerConfiguration tConf;
+
+    public LedgerCommandTest() {
+        super(3, 0);
+    }
+
+    public void setup() throws Exception {
+        super.setup();
+        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
+
+        PowerMockito.mockStatic(DbLedgerStorage.class);
+        PowerMockito.doAnswer(invocationOnMock -> {
+            SingleDirectoryDbLedgerStorage.LedgerLoggerProcessor p = invocationOnMock.getArgument(2);
+            p.process(1L, 1L, 1L);
+            return true;
+        }).when(DbLedgerStorage.class, "readLedgerIndexEntries", anyLong(), any(ServerConfiguration.class),
+                any(SingleDirectoryDbLedgerStorage.LedgerLoggerProcessor.class));
+        PowerMockito.when(DbLedgerStorage.class.getName())
+                .thenReturn("org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage");
+
+        tConf = PowerMockito.mock(ServerConfiguration.class);
+        PowerMockito.whenNew(ServerConfiguration.class).withArguments(ServerConfiguration.class)
+                .thenReturn(tConf);
+
+        InterleavedLedgerStorage interleavedLedgerStorage = PowerMockito.mock(InterleavedLedgerStorage.class);
+        PowerMockito.whenNew(InterleavedLedgerStorage.class).withNoArguments().thenReturn(interleavedLedgerStorage);
+
+        PowerMockito.mockStatic(Bookie.class);
+        PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(tConf), eq(interleavedLedgerStorage)))
+                .thenReturn(PowerMockito.mock(LedgerStorage.class));
+
+        LedgerCache.PageEntries e = PowerMockito.mock(LedgerCache.PageEntries.class);
+        LedgerCache.PageEntriesIterable i = PowerMockito.mock(LedgerCache.PageEntriesIterable.class);
+        PowerMockito.when(interleavedLedgerStorage.getIndexEntries(anyLong())).thenReturn(i);
+        PowerMockito.when(i.iterator()).thenReturn(getPageIterator(e));
+        LedgerEntryPage lep = PowerMockito.mock(LedgerEntryPage.class);
+        PowerMockito.when(e.getLEP()).thenReturn(lep);
+
+        metadata = PowerMockito.mock(LedgerCache.LedgerIndexMetadata.class);
+        PowerMockito.when(interleavedLedgerStorage.readLedgerIndexMetadata(anyLong())).thenReturn(metadata);
+        PowerMockito.when(metadata.getMasterKeyHex()).thenReturn("");
+    }
+
+    public Iterator<LedgerCache.PageEntries> getPageIterator(LedgerCache.PageEntries page) {
+        Iterator<LedgerCache.PageEntries> i = new Iterator<LedgerCache.PageEntries>() {
+            int i = 0;
+
+            @Override
+            public boolean hasNext() {
+                if (i < 2) {
+                    i++;
+                    return true;
+                }
+                return false;
+            }
+
+            @Override
+            public LedgerCache.PageEntries next() {
+                return page;
+            }
+        };
+        return i;
+    }
+
+    // Test without ledger id
+    @Test
+    public void testWithoutLedgerId() {
+        testLedgerCommand("");
+    }
+
+    // test ledger command without args
+    @Test
+    public void testNoArguments() {
+        testLedgerCommand("-id", "1");
+    }
+
+    @Test
+    public void testWithMeta() throws Exception {
+        LedgerCommand cmd = new LedgerCommand();
+        cmd.apply(bkFlags, new String[] { "-id", "1", "-m" });
+
+        PowerMockito.verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf));
+        PowerMockito.verifyNew(InterleavedLedgerStorage.class, times(1)).withNoArguments();
+
+        verify(metadata, times(1)).getMasterKeyHex();
+    }
+
+    @Test
+    public void testDbLedgerStorage() throws Exception {
+        conf.setLedgerStorageClass("org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage");
+        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
+        LedgerCommand cmd = new LedgerCommand();
+        Assert.assertTrue(cmd.apply(bkFlags, new String[]{"-id", "1"}));
+    }
+
+    // test update formatter by flag
+    @Test
+    public void testFormatterFlag() {
+        testLedgerCommand("-id", "1", "-l", "hex");
+    }
+
+    public void testLedgerCommand(String... args) {
+        LedgerCommand ledgerCommand = new LedgerCommand();
+        try {
+            ledgerCommand.apply(bkFlags, args);
+        } catch (IllegalArgumentException iae) {
+            if (!iae.getMessage().equals("No ledger id is specified")) {
+                Assert.fail("exception is not expect ! ");
+            }
+        }
+    }
+}