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/04/03 06:17:49 UTC

[bookkeeper] branch master updated: Migrate command `readlog`

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 d40b8b6  Migrate command `readlog`
d40b8b6 is described below

commit d40b8b69f1b25f4ad3cd9992db616b847fed54d8
Author: Yong Zhang <zh...@gmail.com>
AuthorDate: Wed Apr 3 14:17:45 2019 +0800

    Migrate command `readlog`
    
    Descriptions of the changes in this PR:
    
    #2044
    
    Reviewers: Sijie Guo <si...@apache.org>
    
    This closes #2045 from zymap/command-readlog
---
 .../org/apache/bookkeeper/bookie/BookieShell.java  | 191 +------------
 .../tools/cli/commands/bookie/ReadLogCommand.java  | 296 +++++++++++++++++++++
 .../tools/cli/commands/BookieCommandGroup.java     |   2 +
 .../cli/commands/bookie/ReadLogCommandTest.java    |  86 ++++++
 4 files changed, 396 insertions(+), 179 deletions(-)

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 725faa7..0d8dea5 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
@@ -27,7 +27,6 @@ import static org.apache.bookkeeper.tools.cli.helpers.CommandHelpers.getBookieSo
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.UncheckedExecutionException;
-import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufUtil;
 import io.netty.buffer.UnpooledByteBufAllocator;
 import io.netty.channel.EventLoopGroup;
@@ -68,7 +67,6 @@ import java.util.stream.LongStream;
 
 import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException;
 import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException;
-import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner;
 import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BKException.MetaStoreException;
@@ -99,13 +97,13 @@ import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryD
 import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand;
-import org.apache.bookkeeper.tools.cli.commands.bookie.FormatUtil;
 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.bookie.ListFilesOnDiscCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand;
+import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand;
@@ -142,7 +140,6 @@ import org.apache.commons.configuration.CompositeConfiguration;
 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.lang3.ArrayUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
@@ -1182,6 +1179,8 @@ public class BookieShell implements Tool {
                 printUsage();
                 return -1;
             }
+            ReadLogCommand cmd = new ReadLogCommand(ledgerIdFormatter, entryFormatter);
+            ReadLogCommand.ReadLogFlags flags = new ReadLogCommand.ReadLogFlags();
 
             boolean printMsg = false;
             if (cmdLine.hasOption("m")) {
@@ -1190,40 +1189,22 @@ public class BookieShell implements Tool {
             long logId;
             try {
                 logId = Long.parseLong(leftArgs[0]);
+                flags.entryLogId(logId);
             } catch (NumberFormatException nfe) {
                 // not a entry log id
-                File f = new File(leftArgs[0]);
-                String name = f.getName();
-                if (!name.endsWith(".log")) {
-                    // not a log file
-                    System.err.println("ERROR: invalid entry log file name " + leftArgs[0]);
-                    printUsage();
-                    return -1;
-                }
-                String idString = name.split("\\.")[0];
-                logId = Long.parseLong(idString, 16);
+                flags.filename(leftArgs[0]);
             }
-
             final long lId = getOptionLedgerIdValue(cmdLine, "ledgerid", -1);
             final long eId = getOptionLongValue(cmdLine, "entryid", -1);
             final long startpos = getOptionLongValue(cmdLine, "startpos", -1);
             final long endpos = getOptionLongValue(cmdLine, "endpos", -1);
-
-            // scan entry log
-            if (startpos != -1) {
-                if ((endpos != -1) && (endpos < startpos)) {
-                    System.err
-                            .println("ERROR: StartPosition of the range should be lesser than or equal to EndPosition");
-                    return -1;
-                }
-                scanEntryLogForPositionRange(logId, startpos, endpos, printMsg);
-            } else if (lId != -1) {
-                scanEntryLogForSpecificEntry(logId, lId, eId, printMsg);
-            } else {
-                scanEntryLog(logId, printMsg);
-            }
-
-            return 0;
+            flags.endPos(endpos);
+            flags.startPos(startpos);
+            flags.entryId(eId);
+            flags.ledgerId(lId);
+            flags.msg(printMsg);
+            boolean result = cmd.apply(bkConf, flags);
+            return (result) ? 0 : -1;
         }
 
         @Override
@@ -2665,30 +2646,6 @@ public class BookieShell implements Tool {
         System.exit(res);
     }
 
-    ///
-    /// Bookie File Operations
-    ///
-
-    /**
-     * Get the ledger file of a specified ledger.
-     *
-     * @param ledgerId Ledger Id
-     *
-     * @return file object.
-     */
-    private File getLedgerFile(long ledgerId) {
-        String ledgerName = IndexPersistenceMgr.getLedgerName(ledgerId);
-        File lf = null;
-        for (File d : indexDirectories) {
-            lf = new File(d, ledgerName);
-            if (lf.exists()) {
-                break;
-            }
-            lf = null;
-        }
-        return lf;
-    }
-
     private synchronized void initEntryLogger() throws IOException {
         if (null == entryLogger) {
             // provide read only entry logger
@@ -2696,17 +2653,6 @@ public class BookieShell implements Tool {
         }
     }
 
-    /**
-     * Scan over entry log.
-     *
-     * @param logId Entry Log Id
-     * @param scanner Entry Log Scanner
-     */
-    protected void scanEntryLog(long logId, EntryLogScanner scanner) throws IOException {
-        initEntryLogger();
-        entryLogger.scanEntryLog(logId, scanner);
-    }
-
     ///
     /// Bookie Shell Commands
     ///
@@ -2721,119 +2667,6 @@ public class BookieShell implements Tool {
         });
     }
 
-    /**
-     * Scan over an entry log file.
-     *
-     * @param logId
-     *          Entry Log File id.
-     * @param printMsg
-     *          Whether printing the entry data.
-     */
-    protected void scanEntryLog(long logId, final boolean printMsg) throws Exception {
-        System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)");
-        scanEntryLog(logId, new EntryLogScanner() {
-            @Override
-            public boolean accept(long ledgerId) {
-                return true;
-            }
-
-            @Override
-            public void process(long ledgerId, long startPos, ByteBuf entry) {
-                FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter);
-            }
-        });
-    }
-
-    /**
-     * Scan over an entry log file for a particular entry.
-     *
-     * @param logId Entry Log File id.
-     * @param ledgerId id of the ledger
-     * @param entryId entryId of the ledger we are looking for (-1 for all of the entries of the ledger)
-     * @param printMsg Whether printing the entry data.
-     * @throws Exception
-     */
-    protected void scanEntryLogForSpecificEntry(long logId, final long ledgerId, final long entryId,
-                                                final boolean printMsg) throws Exception {
-        System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for LedgerId "
-                + ledgerId + ((entryId == -1) ? "" : " for EntryId " + entryId));
-        final MutableBoolean entryFound = new MutableBoolean(false);
-        scanEntryLog(logId, new EntryLogScanner() {
-            @Override
-            public boolean accept(long candidateLedgerId) {
-                return ((candidateLedgerId == ledgerId) && ((!entryFound.booleanValue()) || (entryId == -1)));
-            }
-
-            @Override
-            public void process(long candidateLedgerId, long startPos, ByteBuf entry) {
-                long entrysLedgerId = entry.getLong(entry.readerIndex());
-                long entrysEntryId = entry.getLong(entry.readerIndex() + 8);
-                if ((candidateLedgerId == entrysLedgerId) && (candidateLedgerId == ledgerId)
-                        && ((entrysEntryId == entryId) || (entryId == -1))) {
-                    entryFound.setValue(true);
-                    FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter);
-                }
-            }
-        });
-        if (!entryFound.booleanValue()) {
-            System.out.println("LedgerId " + ledgerId + ((entryId == -1) ? "" : " EntryId " + entryId)
-                    + " is not available in the entry log " + logId + " (" + Long.toHexString(logId) + ".log)");
-        }
-    }
-
-    /**
-     * Scan over an entry log file for entries in the given position range.
-     *
-     * @param logId Entry Log File id.
-     * @param rangeStartPos Start position of the entry we are looking for
-     * @param rangeEndPos End position of the entry we are looking for (-1 for till the end of the entrylog)
-     * @param printMsg Whether printing the entry data.
-     * @throws Exception
-     */
-    protected void scanEntryLogForPositionRange(long logId, final long rangeStartPos, final long rangeEndPos,
-                                                final boolean printMsg) throws Exception {
-        System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for PositionRange: "
-                + rangeStartPos + " - " + rangeEndPos);
-        final MutableBoolean entryFound = new MutableBoolean(false);
-        scanEntryLog(logId, new EntryLogScanner() {
-            private MutableBoolean stopScanning = new MutableBoolean(false);
-
-            @Override
-            public boolean accept(long ledgerId) {
-                return !stopScanning.booleanValue();
-            }
-
-            @Override
-            public void process(long ledgerId, long entryStartPos, ByteBuf entry) {
-                if (!stopScanning.booleanValue()) {
-                    if ((rangeEndPos != -1) && (entryStartPos > rangeEndPos)) {
-                        stopScanning.setValue(true);
-                    } else {
-                        int entrySize = entry.readableBytes();
-                        /**
-                         * entrySize of an entry (inclusive of payload and
-                         * header) value is stored as int value in log file, but
-                         * it is not counted in the entrySize, hence for calculating
-                         * the end position of the entry we need to add additional
-                         * 4 (intsize of entrySize). Please check
-                         * EntryLogger.scanEntryLog.
-                         */
-                        long entryEndPos = entryStartPos + entrySize + 4 - 1;
-                        if (((rangeEndPos == -1) || (entryStartPos <= rangeEndPos)) && (rangeStartPos <= entryEndPos)) {
-                            FormatUtil.formatEntry(entryStartPos, entry, printMsg, ledgerIdFormatter, entryFormatter);
-                            entryFound.setValue(true);
-                        }
-                    }
-                }
-            }
-        });
-        if (!entryFound.booleanValue()) {
-            System.out.println("Entry log " + logId + " (" + Long.toHexString(logId)
-                    + ".log) doesn't has any entry in the range " + rangeStartPos + " - " + rangeEndPos
-                    + ". Probably the position range, you have provided is lesser than the LOGFILE_HEADER_SIZE (1024) "
-                    + "or greater than the current log filesize.");
-        }
-    }
 
     /**
      * Format the entry into a readable format.
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java
new file mode 100644
index 0000000..eae71ca
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java
@@ -0,0 +1,296 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+import java.io.File;
+import java.io.IOException;
+import lombok.Setter;
+import lombok.experimental.Accessors;
+import org.apache.bookkeeper.bookie.EntryLogger;
+import org.apache.bookkeeper.bookie.ReadOnlyEntryLogger;
+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.EntryFormatter;
+import org.apache.bookkeeper.util.LedgerIdFormatter;
+import org.apache.commons.lang.mutable.MutableBoolean;
+
+/**
+ * Command to read entry log files.
+ */
+public class ReadLogCommand extends BookieCommand<ReadLogCommand.ReadLogFlags> {
+
+    private static final String NAME = "readlog";
+    private static final String DESC = "Scan an entry file and format the entries into readable format.";
+
+    private EntryLogger entryLogger;
+    private EntryFormatter entryFormatter;
+    private LedgerIdFormatter ledgerIdFormatter;
+
+    public ReadLogCommand() {
+        this(new ReadLogFlags());
+    }
+
+    public ReadLogCommand(LedgerIdFormatter ledgerIdFormatter, EntryFormatter entryFormatter) {
+        this(new ReadLogFlags());
+        this.ledgerIdFormatter = ledgerIdFormatter;
+        this.entryFormatter = entryFormatter;
+    }
+    private ReadLogCommand(ReadLogFlags flags) {
+        super(CliSpec.<ReadLogFlags>newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build());
+    }
+
+    /**
+     * Flags for read log command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReadLogFlags extends CliFlags {
+
+        @Parameter(names = { "-m", "msg" }, description = "Print message body")
+        private boolean msg;
+
+        @Parameter(names = { "-l", "--ledgerid" }, description = "Ledger ID")
+        private long ledgerId = -1;
+
+        @Parameter(names = { "-e", "--entryid" }, description = "Entry ID")
+        private long entryId = -1;
+
+        @Parameter(names = { "-sp", "--startpos" }, description = "Start Position")
+        private long startPos = -1;
+
+        @Parameter(names = { "-ep", "--endpos" }, description = "End Position")
+        private long endPos = -1;
+
+        @Parameter(names = { "-f", "--filename" }, description = "Entry log filename")
+        private String filename;
+
+        @Parameter(names = { "-li", "--entrylogid" }, description = "Entry log id")
+        private long entryLogId = -1;
+
+        @Parameter(names = {"-lf", "--ledgerIdFormatter"}, description = "Set ledger id formatter")
+        private String ledgerIdFormatter;
+
+        @Parameter(names = {"-ef", "--entryformatter"}, description = "set entry formatter")
+        private String entryFormatter;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReadLogFlags cmdFlags) {
+
+        if (cmdFlags.ledgerIdFormatter != null && this.ledgerIdFormatter == null) {
+            this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(cmdFlags.ledgerIdFormatter, conf);
+        } else if (this.ledgerIdFormatter == null) {
+            this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf);
+        }
+
+        if (cmdFlags.entryFormatter != null && this.entryFormatter == null) {
+            this.entryFormatter = EntryFormatter.newEntryFormatter(cmdFlags.entryFormatter, conf);
+        } else if (this.entryFormatter == null) {
+            this.entryFormatter = EntryFormatter.newEntryFormatter(conf);
+        }
+
+        if (cmdFlags.entryLogId == -1 && cmdFlags.filename == null) {
+            System.err.println("Missing entry log id or entry log file name");
+            usage();
+            return false;
+        }
+        try {
+            return readLog(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean readLog(ServerConfiguration conf, ReadLogFlags flags) throws Exception {
+        long logId = flags.entryLogId;
+        if (logId == -1 && flags.filename != null) {
+            File f = new File(flags.filename);
+            String name = f.getName();
+            if (!name.endsWith(".log")) {
+                System.err.println("Invalid entry log file name " + flags.filename);
+                usage();
+                return false;
+            }
+            String idString = name.split("\\.")[0];
+            logId = Long.parseLong(idString, 16);
+        }
+
+        final long lId = flags.ledgerId;
+        final long eId = flags.entryId;
+        final long startpos = flags.startPos;
+        final long endpos = flags.endPos;
+
+        // scan entry log
+        if (startpos != -1) {
+            if ((endpos != -1) && (endpos < startpos)) {
+                System.err
+                    .println("ERROR: StartPosition of the range should be lesser than or equal to EndPosition");
+                return false;
+            }
+            scanEntryLogForPositionRange(conf, logId, startpos, endpos, flags.msg);
+        } else if (lId != -1) {
+            scanEntryLogForSpecificEntry(conf, logId, lId, eId, flags.msg);
+        } else {
+            scanEntryLog(conf, logId, flags.msg);
+        }
+        return true;
+    }
+
+    /**
+     * Scan over an entry log file for entries in the given position range.
+     *
+     * @param logId Entry Log File id.
+     * @param rangeStartPos Start position of the entry we are looking for
+     * @param rangeEndPos End position of the entry we are looking for (-1 for till the end of the entrylog)
+     * @param printMsg Whether printing the entry data.
+     * @throws Exception
+     */
+    private void scanEntryLogForPositionRange(ServerConfiguration conf, long logId, final long rangeStartPos,
+                                              final long rangeEndPos,
+                                                final boolean printMsg) throws Exception {
+        System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for PositionRange: "
+                           + rangeStartPos + " - " + rangeEndPos);
+        final MutableBoolean entryFound = new MutableBoolean(false);
+        scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() {
+            private MutableBoolean stopScanning = new MutableBoolean(false);
+
+            @Override
+            public boolean accept(long ledgerId) {
+                return !stopScanning.booleanValue();
+            }
+
+            @Override
+            public void process(long ledgerId, long entryStartPos, ByteBuf entry) throws IOException {
+                if (!stopScanning.booleanValue()) {
+                    if ((rangeEndPos != -1) && (entryStartPos > rangeEndPos)) {
+                        stopScanning.setValue(true);
+                    } else {
+                        int entrySize = entry.readableBytes();
+                        /**
+                         * entrySize of an entry (inclusive of payload and
+                         * header) value is stored as int value in log file, but
+                         * it is not counted in the entrySize, hence for calculating
+                         * the end position of the entry we need to add additional
+                         * 4 (intsize of entrySize). Please check
+                         * EntryLogger.scanEntryLog.
+                         */
+                        long entryEndPos = entryStartPos + entrySize + 4 - 1;
+                        if (((rangeEndPos == -1) || (entryStartPos <= rangeEndPos)) && (rangeStartPos <= entryEndPos)) {
+                            FormatUtil.formatEntry(entryStartPos, entry, printMsg, ledgerIdFormatter, entryFormatter);
+                            entryFound.setValue(true);
+                        }
+                    }
+                }
+            }
+        });
+        if (!entryFound.booleanValue()) {
+            System.out.println(
+                "Entry log " + logId + " (" + Long.toHexString(logId) + ".log) doesn't has any entry in the range "
+                + rangeStartPos + " - " + rangeEndPos
+                + ". Probably the position range, you have provided is lesser than the LOGFILE_HEADER_SIZE (1024) "
+                + "or greater than the current log filesize.");
+        }
+    }
+
+    /**
+     * Scan over entry log.
+     *
+     * @param logId   Entry Log Id
+     * @param scanner Entry Log Scanner
+     */
+    private void scanEntryLog(ServerConfiguration conf, long logId, EntryLogger.EntryLogScanner scanner)
+        throws IOException {
+        initEntryLogger(conf);
+        entryLogger.scanEntryLog(logId, scanner);
+    }
+
+    private synchronized void initEntryLogger(ServerConfiguration conf) throws IOException {
+        if (null == entryLogger) {
+            // provide read only entry logger
+            entryLogger = new ReadOnlyEntryLogger(conf);
+        }
+    }
+
+    /**
+     * Scan over an entry log file for a particular entry.
+     *
+     * @param logId Entry Log File id.
+     * @param ledgerId id of the ledger
+     * @param entryId entryId of the ledger we are looking for (-1 for all of the entries of the ledger)
+     * @param printMsg Whether printing the entry data.
+     * @throws Exception
+     */
+    private void scanEntryLogForSpecificEntry(ServerConfiguration conf, long logId, final long ledgerId,
+                                                final long entryId,
+                                                final boolean printMsg) throws Exception {
+        System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for LedgerId "
+                           + ledgerId + ((entryId == -1) ? "" : " for EntryId " + entryId));
+        final MutableBoolean entryFound = new MutableBoolean(false);
+        scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() {
+            @Override
+            public boolean accept(long candidateLedgerId) {
+                return ((candidateLedgerId == ledgerId) && ((!entryFound.booleanValue()) || (entryId == -1)));
+            }
+
+            @Override
+            public void process(long candidateLedgerId, long startPos, ByteBuf entry) {
+                long entrysLedgerId = entry.getLong(entry.readerIndex());
+                long entrysEntryId = entry.getLong(entry.readerIndex() + 8);
+                if ((candidateLedgerId == entrysLedgerId) && (candidateLedgerId == ledgerId)
+                    && ((entrysEntryId == entryId) || (entryId == -1))) {
+                    entryFound.setValue(true);
+                    FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter);
+                }
+            }
+        });
+        if (!entryFound.booleanValue()) {
+            System.out.println("LedgerId " + ledgerId + ((entryId == -1) ? "" : " EntryId " + entryId)
+                               + " is not available in the entry log " + logId + " (" + Long.toHexString(logId)
+                               + ".log)");
+        }
+    }
+
+    /**
+     * Scan over an entry log file.
+     *
+     * @param logId
+     *          Entry Log File id.
+     * @param printMsg
+     *          Whether printing the entry data.
+     */
+    private void scanEntryLog(ServerConfiguration conf, long logId, final boolean printMsg) throws Exception {
+        System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)");
+        scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() {
+            @Override
+            public boolean accept(long ledgerId) {
+                return true;
+            }
+
+            @Override
+            public void process(long ledgerId, long startPos, ByteBuf entry) {
+                FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter);
+            }
+        });
+    }
+}
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 98480ef..2970c8c 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
@@ -30,6 +30,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand;
+import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand;
 import org.apache.bookkeeper.tools.common.BKFlags;
@@ -59,6 +60,7 @@ public class BookieCommandGroup extends CliCommandGroup<BKFlags> {
         .addCommand(new ListLedgersCommand())
         .addCommand(new ConvertToInterleavedStorageCommand())
         .addCommand(new ReadJournalCommand())
+        .addCommand(new ReadLogCommand())
         .addCommand(new ReadLogMetadataCommand())
         .build();
 
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommandTest.java
new file mode 100644
index 0000000..7f18fcb
--- /dev/null
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommandTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.anyLong;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.powermock.api.mockito.PowerMockito.doNothing;
+
+import org.apache.bookkeeper.bookie.EntryLogger;
+import org.apache.bookkeeper.bookie.ReadOnlyEntryLogger;
+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.mockito.Mock;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test for {@link ReadLogCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ ReadLogCommand.class })
+public class ReadLogCommandTest extends BookieCommandTestBase {
+
+    @Mock
+    private ReadOnlyEntryLogger entryLogger;
+
+    @Mock
+    private EntryLogger.EntryLogScanner entryLogScanner;
+
+    public ReadLogCommandTest() {
+        super(3, 0);
+    }
+
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+
+        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
+        PowerMockito.whenNew(ReadOnlyEntryLogger.class).withArguments(eq(conf)).thenReturn(entryLogger);
+        doNothing().when(entryLogger).scanEntryLog(anyLong(), eq(entryLogScanner));
+    }
+
+    @Test
+    public void testWithoutAnyFlags() {
+        ReadLogCommand cmd = new ReadLogCommand();
+        Assert.assertFalse(cmd.apply(bkFlags, new String[] {}));
+    }
+
+    @Test
+    public void testWithEntryId() {
+        ReadLogCommand cmd = new ReadLogCommand();
+        Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-li", "1" }));
+    }
+
+    @Test
+    public void testWithEntryFilename() {
+        ReadLogCommand cmd = new ReadLogCommand();
+        Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-f", "1.log" }));
+    }
+
+    @Test
+    public void testWithErrorPos() {
+        ReadLogCommand cmd = new ReadLogCommand();
+        Assert.assertFalse(cmd.apply(bkFlags, new String[] { "-sp", "1", "-ep", "0", "-li", "1" }));
+    }
+}