You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2017/12/18 21:44:32 UTC

[GitHub] sijie closed pull request #864: ISSUE #849: LedgerIdFormatter and EntryFormatter Config options

sijie closed pull request #864: ISSUE #849: LedgerIdFormatter and EntryFormatter Config options
URL: https://github.com/apache/bookkeeper/pull/864
 
 
   

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

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

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
index ccffb25f7..aacd7a21f 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
@@ -7,7 +7,7 @@
  * "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
+ *      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,
@@ -24,10 +24,8 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.AbstractFuture;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
-
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -58,7 +56,6 @@
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
-
 import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException;
 import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException;
 import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint;
@@ -92,6 +89,7 @@
 import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.util.EntryFormatter;
 import org.apache.bookkeeper.util.IOUtils;
+import org.apache.bookkeeper.util.LedgerIdFormatter;
 import org.apache.bookkeeper.util.MathUtils;
 import org.apache.bookkeeper.util.Tool;
 import org.apache.bookkeeper.versioning.Version;
@@ -121,7 +119,9 @@
 
     static final Logger LOG = LoggerFactory.getLogger(BookieShell.class);
 
-    static final String ENTRY_FORMATTER_CLASS = "entryFormatterClass";
+    static final String CONF_OPT = "conf";
+    static final String ENTRY_FORMATTER_OPT = "entryformat";
+    static final String LEDGERID_FORMATTER_OPT = "ledgeridformat";
 
     static final String CMD_METAFORMAT = "metaformat";
     static final String CMD_BOOKIEFORMAT = "bookieformat";
@@ -160,20 +160,33 @@
 
     EntryLogger entryLogger = null;
     List<Journal> journals = null;
-    EntryFormatter formatter;
+    EntryFormatter entryFormatter;
+    LedgerIdFormatter ledgerIdFormatter;
 
     int pageSize;
     int entriesPerPage;
 
+    public BookieShell() {
+    }
+
+    public BookieShell(LedgerIdFormatter ledgeridFormatter, EntryFormatter entryFormatter) {
+        this.ledgerIdFormatter = ledgeridFormatter;
+        this.entryFormatter = entryFormatter;
+    }
+
     interface Command {
         int runCmd(String[] args) throws Exception;
+
         void printUsage();
     }
 
     abstract class MyCommand implements Command {
         abstract Options getOptions();
+
         abstract String getDescription();
+
         abstract String getUsage();
+
         abstract int runCmd(CommandLine cmdLine) throws Exception;
 
         String cmdName;
@@ -286,7 +299,8 @@ int runCmd(CommandLine cmdLine) throws Exception {
             // delete cookie
             if (cmdLine.hasOption("d")) {
                 RegistrationManager rm = new ZKRegistrationManager();
-                rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE);
+                rm.initialize(conf, () -> {
+                }, NullStatsLogger.INSTANCE);
                 try {
                     Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, conf);
                     cookie.getValue().deleteFromRegistrationManager(rm, conf, cookie.getVersion());
@@ -347,14 +361,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
             boolean skipOpenLedgers = cmdLine.hasOption("sk");
             boolean removeCookies = !dryrun && cmdLine.hasOption("d");
 
-            Long ledgerId = null;
-            if (cmdLine.hasOption("l")) {
-                try {
-                    ledgerId = Long.parseLong(cmdLine.getOptionValue("l"));
-                } catch (NumberFormatException nfe) {
-                    throw new IOException("Invalid ledger id provided : " + cmdLine.getOptionValue("l"));
-                }
-            }
+            Long ledgerId = getOptionLedgerIdValue(cmdLine, "ledger", -1);
 
             // Get bookies list
             final String[] bookieStrs = args[0].split(",");
@@ -386,7 +393,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
                 if (query) {
                     return bkQuery(admin, bookieAddrs);
                 }
-                if (null != ledgerId) {
+                if (-1 != ledgerId) {
                     return bkRecoveryLedger(admin, ledgerId, bookieAddrs, dryrun, skipOpenLedgers, removeCookies);
                 }
                 return bkRecovery(admin, bookieAddrs, dryrun, skipOpenLedgers, removeCookies);
@@ -421,7 +428,7 @@ private int bkQuery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddr
                 ArrayList<BookieSocketAddress> bookieList = ensemble.getValue();
                 System.out.print(ensemble.getKey() + ":\t");
                 int numBookiesToReplace = 0;
-                for (BookieSocketAddress bookie: bookieList) {
+                for (BookieSocketAddress bookie : bookieList) {
                     System.out.print(bookie);
                     if (bookiesToInspect.contains(bookie)) {
                         System.out.print("*");
@@ -517,8 +524,8 @@ public int runCmd(CommandLine cmdLine) throws Exception {
             }
             long ledgerId;
             try {
-                ledgerId = Long.parseLong(leftArgs[0]);
-            } catch (NumberFormatException nfe) {
+                ledgerId = ledgerIdFormatter.readLedgerId(leftArgs[0]);
+            } catch (IllegalArgumentException iae) {
                 System.err.println("ERROR: invalid ledger id " + leftArgs[0]);
                 printUsage();
                 return -1;
@@ -556,6 +563,10 @@ Options getOptions() {
 
         ReadLedgerEntriesCmd() {
             super(CMD_READ_LEDGER_ENTRIES);
+            lOpts.addOption("m", "msg", false, "Print message body");
+            lOpts.addOption("l", "ledgerid", true, "Ledger ID");
+            lOpts.addOption("fe", "firstentryid", true, "First EntryID");
+            lOpts.addOption("le", "lastentryid", true, "Last EntryID");
         }
 
         @Override
@@ -570,34 +581,22 @@ String getDescription() {
 
         @Override
         String getUsage() {
-            return "readledger <ledger_id> [<start_entry_id> [<end_entry_id>]]";
+            return "readledger   [-msg] -ledgerid <ledgerid> "
+                    + "[-firstentryid <firstentryid> [-lastentryid <lastentryid>]]";
         }
 
         @Override
         int runCmd(CommandLine cmdLine) throws Exception {
-            String[] leftArgs = cmdLine.getArgs();
-            if (leftArgs.length <= 0) {
-                System.err.println("ERROR: missing ledger id");
-                printUsage();
+            final long ledgerId = getOptionLedgerIdValue(cmdLine, "ledgerid", -1);
+            if (ledgerId == -1) {
+                System.err.println("Must specify a ledger id");
                 return -1;
             }
 
-            long ledgerId;
-            long firstEntry = 0;
-            long lastEntry = -1;
-            try {
-                ledgerId = Long.parseLong(leftArgs[0]);
-                if (leftArgs.length >= 2) {
-                    firstEntry = Long.parseLong(leftArgs[1]);
-                }
-                if (leftArgs.length >= 3) {
-                    lastEntry = Long.parseLong(leftArgs[2]);
-                }
-            } catch (NumberFormatException nfe) {
-                System.err.println("ERROR: invalid number " + nfe.getMessage());
-                printUsage();
-                return -1;
-            }
+            final long firstEntry = getOptionLongValue(cmdLine, "firstentryid", 0);
+            final long lastEntry = getOptionLongValue(cmdLine, "lastentryid", -1);
+
+            boolean printMsg = cmdLine.hasOption("m");
 
             ClientConfiguration conf = new ClientConfiguration();
             conf.addConfiguration(bkConf);
@@ -608,7 +607,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
                 Iterator<LedgerEntry> entries = bk.readEntries(ledgerId, firstEntry, lastEntry).iterator();
                 while (entries.hasNext()) {
                     LedgerEntry entry = entries.next();
-                    formatEntry(entry, true);
+                    formatEntry(entry, printMsg);
                 }
             } catch (Exception e) {
                 LOG.error("Error reading entries from ledger {}", ledgerId, e.getCause());
@@ -644,13 +643,13 @@ Options getOptions() {
         @Override
         String getDescription() {
             return "List ledgers marked as underreplicated, with optional options to specify missingreplica"
-                + " (BookieId) and to exclude missingreplica.";
+                    + " (BookieId) and to exclude missingreplica.";
         }
 
         @Override
         String getUsage() {
             return "listunderreplicated [[-missingreplica <bookieaddress>]"
-                + " [-excludingmissingreplica <bookieaddress>]]";
+                    + " [-excludingmissingreplica <bookieaddress>]]";
         }
 
         @Override
@@ -679,7 +678,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
                 LedgerUnderreplicationManager underreplicationManager = mFactory.newLedgerUnderreplicationManager();
                 Iterator<Long> iter = underreplicationManager.listLedgersToRereplicate(predicate);
                 while (iter.hasNext()) {
-                    System.out.println(iter.next());
+                    System.out.println(ledgerIdFormatter.formatLedgerId(iter.next()));
                 }
             } finally {
                 if (zk != null) {
@@ -692,6 +691,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
     }
 
     static final int LIST_BATCH_SIZE = 1000;
+
     /**
      * Command to list all ledgers in the cluster.
      */
@@ -741,17 +741,17 @@ public int runCmd(CommandLine cmdLine) throws Exception {
                     while (iter.hasNext()) {
                         LedgerRange r = iter.next();
                         for (Long lid : r.getLedgers()) {
-                            System.out.println(Long.toString(lid));
+                            System.out.println(ledgerIdFormatter.formatLedgerId(lid));
                         }
                     }
                 }
             } finally {
                 if (m != null) {
                     try {
-                      m.close();
-                      mFactory.uninitialize();
+                        m.close();
+                        mFactory.uninitialize();
                     } catch (IOException ioe) {
-                      LOG.error("Failed to close ledger manager : ", ioe);
+                        LOG.error("Failed to close ledger manager : ", ioe);
                     }
                 }
                 if (zk != null) {
@@ -778,14 +778,14 @@ Options getOptions() {
         }
     }
 
-    static void printLedgerMetadata(ReadMetadataCallback cb) throws Exception {
+    void printLedgerMetadata(ReadMetadataCallback cb) throws Exception {
         LedgerMetadata md = cb.get();
-        System.out.println("ledgerID: " + cb.getLedgerId());
+        System.out.println("ledgerID: " + ledgerIdFormatter.formatLedgerId(cb.getLedgerId()));
         System.out.println(new String(md.serialize(), UTF_8));
     }
 
     static class ReadMetadataCallback extends AbstractFuture<LedgerMetadata>
-        implements GenericCallback<LedgerMetadata> {
+            implements GenericCallback<LedgerMetadata> {
         final long ledgerId;
 
         ReadMetadataCallback(long ledgerId) {
@@ -818,7 +818,7 @@ public void operationComplete(int rc, LedgerMetadata result) {
 
         @Override
         public int runCmd(CommandLine cmdLine) throws Exception {
-            final long lid = getOptionLongValue(cmdLine, "ledgerid", -1);
+            final long lid = getOptionLedgerIdValue(cmdLine, "ledgerid", -1);
             if (lid == -1) {
                 System.err.println("Must specify a ledger id");
                 return -1;
@@ -897,13 +897,13 @@ public int runCmd(CommandLine cmdLine) throws Exception {
             conf.addConfiguration(bkConf);
             BookKeeper bk = new BookKeeper(conf);
             LedgerHandle lh = bk.createLedger(ensemble, writeQuorum, ackQuorum,
-                                              BookKeeper.DigestType.MAC, new byte[0]);
+                    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) {
+                        TimeUnit.NANOSECONDS) > 1) {
                     System.out.println(i + " entries written");
                     lastReport = System.nanoTime();
                 }
@@ -1065,7 +1065,7 @@ public int runCmd(CommandLine cmdLine) throws Exception {
                 logId = Long.parseLong(idString, 16);
             }
 
-            final long lId = getOptionLongValue(cmdLine, "ledgerid", -1);
+            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);
@@ -1411,9 +1411,9 @@ Options getOptions() {
         public AutoRecoveryCmd() {
             super(CMD_AUTORECOVERY);
             opts.addOption("e", "enable", false,
-                           "Enable auto recovery of underreplicated ledgers");
+                    "Enable auto recovery of underreplicated ledgers");
             opts.addOption("d", "disable", false,
-                           "Disable auto recovery of underreplicated ledgers");
+                    "Disable auto recovery of underreplicated ledgers");
         }
 
         @Override
@@ -1642,7 +1642,8 @@ private int updateBookieIdInCookie(final String bookieId, final boolean useHostn
                 InterruptedException {
             RegistrationManager rm = new ZKRegistrationManager();
             try {
-                rm.initialize(bkConf, () -> {}, NullStatsLogger.INSTANCE);
+                rm.initialize(bkConf, () -> {
+                }, NullStatsLogger.INSTANCE);
                 ServerConfiguration conf = new ServerConfiguration(bkConf);
                 String newBookieId = Bookie.getBookieAddress(conf).toString();
                 // read oldcookie
@@ -1743,7 +1744,7 @@ Options getOptions() {
         @Override
         String getDescription() {
             return "Add new empty ledger/index directories. Update the directories"
-                   + "info in the conf file before running the command.";
+                    + "info in the conf file before running the command.";
         }
 
         @Override
@@ -1756,7 +1757,8 @@ int runCmd(CommandLine cmdLine) {
             ServerConfiguration conf = new ServerConfiguration(bkConf);
             try (RegistrationManager rm = new ZKRegistrationManager()) {
                 try {
-                    rm.initialize(bkConf, () -> {}, NullStatsLogger.INSTANCE);
+                    rm.initialize(bkConf, () -> {
+                    }, NullStatsLogger.INSTANCE);
                 } catch (BookieException e) {
                     LOG.error("Exception while establishing zookeeper connection.", e);
                     return -1;
@@ -1771,10 +1773,10 @@ int runCmd(CommandLine cmdLine) {
                 try {
                     conf.setAllowStorageExpansion(true);
                     Bookie.checkEnvironmentWithStorageExpansion(conf, rm,
-                        Lists.newArrayList(journalDirectories), allLedgerDirs);
+                            Lists.newArrayList(journalDirectories), allLedgerDirs);
                 } catch (BookieException e) {
                     LOG.error(
-                        "Exception while updating cookie for storage expansion", e);
+                            "Exception while updating cookie for storage expansion", e);
                     return -1;
                 }
                 return 0;
@@ -1811,7 +1813,7 @@ String getDescription() {
         @Override
         String getUsage() {
             return "updateledger -bookieId <hostname|ip> [-updatespersec N] [-limit N] [-verbose true/false] "
-                   + "[-printprogress N]";
+                    + "[-printprogress N]";
         }
 
         @Override
@@ -1904,26 +1906,17 @@ public void progress(long updated, long issued) {
 
         @Override
         public int runCmd(CommandLine cmdLine) throws Exception {
-            final String lidStr = cmdLine.getOptionValue("ledgerid");
-            if (StringUtils.isBlank(lidStr)) {
-                LOG.error("Invalid argument list!");
-                this.printUsage();
-                return -1;
-            }
-
-            final long lid;
-            try {
-                lid = Long.parseLong(lidStr);
-            } catch (NumberFormatException nfe) {
-                System.err.println("ERROR: invalid ledger id " + lidStr);
-                printUsage();
+            final long lid = getOptionLedgerIdValue(cmdLine, "ledgerid", -1);
+            if (lid == -1) {
+                System.err.println("Must specify a ledger id");
                 return -1;
             }
 
             boolean force = cmdLine.hasOption("f");
             boolean confirm = false;
             if (!force) {
-                confirm = IOUtils.confirmPrompt("Are you sure to delete Ledger : " + lid + "?");
+                confirm = IOUtils.confirmPrompt(
+                        "Are you sure to delete Ledger : " + ledgerIdFormatter.formatLedgerId(lid) + "?");
             }
 
             BookKeeper bk = null;
@@ -1986,7 +1979,7 @@ Options getOptions() {
         }
 
         String getReadable(long val) {
-            String unit[] = {"", "KB", "MB", "GB", "TB" };
+            String unit[] = {"", "KB", "MB", "GB", "TB"};
             int cnt = 0;
             double d = val;
             while (d >= 1000 && cnt < unit.length - 1) {
@@ -2082,7 +2075,7 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         @Override
         String getDescription() {
             return "Force trigger the Audittask and make sure all the ledgers stored in the decommissioning bookie"
-                + " are replicated.";
+                    + " are replicated.";
         }
 
         @Override
@@ -2184,7 +2177,7 @@ public void startCheckpoint(Checkpoint checkpoint) {
             int convertedLedgers = 0;
             for (long ledgerId : interleavedStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)) {
                 if (LOG.isDebugEnabled()) {
-                    LOG.debug("Converting ledger {}", ledgerId);
+                    LOG.debug("Converting ledger {}", ledgerIdFormatter.formatLedgerId(ledgerId));
                 }
 
                 FileInfo fi = getFileInfo(ledgerId);
@@ -2277,7 +2270,7 @@ public void startCheckpoint(Checkpoint checkpoint) {
             int convertedLedgers = 0;
             for (long ledgerId : dbStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)) {
                 if (LOG.isDebugEnabled()) {
-                    LOG.debug("Converting ledger {}", ledgerId);
+                    LOG.debug("Converting ledger {}", ledgerIdFormatter.formatLedgerId(ledgerId));
                 }
 
                 interleavedStorage.setMasterKey(ledgerId, dbStorage.readMasterKey(ledgerId));
@@ -2357,6 +2350,7 @@ int runCmd(CommandLine cmdLine) throws Exception {
     }
 
     final Map<String, MyCommand> commands = new HashMap<String, MyCommand>();
+
     {
         commands.put(CMD_METAFORMAT, new MetaFormatCmd());
         commands.put(CMD_BOOKIEFORMAT, new BookieFormatCmd());
@@ -2399,18 +2393,14 @@ public void setConf(CompositeConfiguration conf) throws Exception {
         } else {
             indexDirectories = Bookie.getCurrentDirectories(bkConf.getIndexDirs());
         }
-        formatter = EntryFormatter.newEntryFormatter(bkConf, ENTRY_FORMATTER_CLASS);
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Using entry formatter {}", formatter.getClass().getName());
-        }
         pageSize = bkConf.getPageSize();
         entriesPerPage = pageSize / 8;
     }
 
     private void printShellUsage() {
-        System.err.println(
-                "Usage: BookieShell [-conf configuration] <command>");
-        System.err.println();
+        System.err.println("Usage: bookkeeper shell [-ledgeridformat <hex/long/uuid>] "
+                + "[-entryformat <hex/string>] [-conf configuration] <command>");
+        System.err.println("where command is one of:");
         List<String> commandNames = new ArrayList<String>();
         for (MyCommand c : commands.values()) {
             commandNames.add("       " + c.getUsage());
@@ -2497,29 +2487,43 @@ public int compare(File file1, File file2) {
 
     public static void main(String argv[]) throws Exception {
         BookieShell shell = new BookieShell();
-        if (argv.length <= 0) {
-            shell.printShellUsage();
-            System.exit(-1);
-        }
 
-        CompositeConfiguration conf = new CompositeConfiguration();
+        // handle some common options for multiple cmds
+        Options opts = new Options();
+        opts.addOption(CONF_OPT, true, "configuration file");
+        opts.addOption(LEDGERID_FORMATTER_OPT, true, "format of ledgerId");
+        opts.addOption(ENTRY_FORMATTER_OPT, true, "format of entries");
+        BasicParser parser = new BasicParser();
+        CommandLine cmdLine = parser.parse(opts, argv, true);
+
         // load configuration
-        if ("-conf".equals(argv[0])) {
-            if (argv.length <= 1) {
-                shell.printShellUsage();
-                System.exit(-1);
-            }
+        CompositeConfiguration conf = new CompositeConfiguration();
+        if (cmdLine.hasOption(CONF_OPT)) {
+            String val = cmdLine.getOptionValue(CONF_OPT);
             conf.addConfiguration(new PropertiesConfiguration(
-                                  new File(argv[1]).toURI().toURL()));
+                    new File(val).toURI().toURL()));
+        }
+        shell.setConf(conf);
 
-            String[] newArgv = new String[argv.length - 2];
-            System.arraycopy(argv, 2, newArgv, 0, newArgv.length);
-            argv = newArgv;
+        // ledgerid format
+        if (cmdLine.hasOption(LEDGERID_FORMATTER_OPT)) {
+            String val = cmdLine.getOptionValue(LEDGERID_FORMATTER_OPT);
+            shell.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(val, shell.bkConf);
+        } else {
+            shell.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(shell.bkConf);
         }
+        LOG.debug("Using ledgerIdFormatter {}", shell.ledgerIdFormatter.getClass());
 
+        // entry format
+        if (cmdLine.hasOption(ENTRY_FORMATTER_OPT)) {
+            String val = cmdLine.getOptionValue(ENTRY_FORMATTER_OPT);
+            shell.entryFormatter = EntryFormatter.newEntryFormatter(val, shell.bkConf);
+        } else {
+            shell.entryFormatter = EntryFormatter.newEntryFormatter(shell.bkConf);
+        }
+        LOG.debug("Using entry formatter {}", shell.entryFormatter.getClass());
 
-        shell.setConf(conf);
-        int res = shell.run(argv);
+        int res = shell.run(cmdLine.getArgs());
         System.exit(res);
     }
 
@@ -2587,7 +2591,7 @@ protected void scanEntryLog(long logId, EntryLogScanner scanner) throws IOExcept
             journals = Lists.newArrayListWithCapacity(bkConf.getJournalDirs().length);
             for (File journalDir : bkConf.getJournalDirs()) {
                 journals.add(new Journal(journalDir, bkConf, new LedgerDirsManager(bkConf, bkConf.getLedgerDirs(),
-                    new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold()))));
+                        new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold()))));
             }
         }
         return journals;
@@ -2613,7 +2617,7 @@ protected void scanJournal(Journal journal, long journalId, JournalScanner scann
      * @param ledgerId Ledger Id
      */
     protected void readLedgerMeta(long ledgerId) throws Exception {
-        System.out.println("===== LEDGER: " + ledgerId + " =====");
+        System.out.println("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " =====");
         FileInfo fi = getFileInfo(ledgerId);
         byte[] masterKey = fi.getMasterKey();
         if (null == masterKey) {
@@ -2638,7 +2642,7 @@ protected void readLedgerMeta(long ledgerId) throws Exception {
      * @throws IOException
      */
     protected void readLedgerIndexEntries(long ledgerId) throws IOException {
-        System.out.println("===== LEDGER: " + ledgerId + " =====");
+        System.out.println("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " =====");
         FileInfo fi = getFileInfo(ledgerId);
         long size = fi.size();
         System.out.println("size        : " + size);
@@ -2753,6 +2757,7 @@ protected void scanEntryLog(long logId, final boolean printMsg) throws Exception
             public boolean accept(long ledgerId) {
                 return true;
             }
+
             @Override
             public void process(long ledgerId, long startPos, ByteBuf entry) {
                 formatEntry(startPos, entry, printMsg);
@@ -2860,6 +2865,7 @@ protected void scanJournal(Journal journal, long journalId, final boolean printM
         System.out.println("Scan journal " + journalId + " (" + Long.toHexString(journalId) + ".txn)");
         scanJournal(journal, journalId, new JournalScanner() {
             boolean printJournalVersion = false;
+
             @Override
             public void process(int journalVersion, long offset, ByteBuffer entry) throws IOException {
                 if (!printJournalVersion) {
@@ -2895,10 +2901,10 @@ private void formatEntry(LedgerEntry entry, boolean printMsg) {
         long ledgerId = entry.getLedgerId();
         long entryId = entry.getEntryId();
         long entrySize = entry.getLength();
-        System.out
-                .println("--------- Lid=" + ledgerId + ", Eid=" + entryId + ", EntrySize=" + entrySize + " ---------");
+        System.out.println("--------- Lid=" + ledgerIdFormatter.formatLedgerId(ledgerId) + ", Eid=" + entryId
+                + ", EntrySize=" + entrySize + " ---------");
         if (printMsg) {
-            formatter.formatEntry(entry.getEntry());
+            entryFormatter.formatEntry(entry.getEntry());
         }
     }
 
@@ -2917,8 +2923,8 @@ private void formatEntry(long pos, ByteBuf recBuff, boolean printMsg) {
         long ledgerId = recBuff.readLong();
         long entryId = recBuff.readLong();
 
-        System.out.println("--------- Lid=" + ledgerId + ", Eid=" + entryId
-                         + ", ByteOffset=" + pos + ", EntrySize=" + entrySize + " ---------");
+        System.out.println("--------- Lid=" + ledgerIdFormatter.formatLedgerId(ledgerId) + ", Eid=" + entryId
+                + ", ByteOffset=" + pos + ", EntrySize=" + entrySize + " ---------");
         if (entryId == Bookie.METAENTRY_ID_LEDGER_KEY) {
             int masterKeyLen = recBuff.readInt();
             byte[] masterKey = new byte[masterKeyLen];
@@ -2949,7 +2955,7 @@ private void formatEntry(long pos, ByteBuf recBuff, boolean printMsg) {
         try {
             byte[] ret = new byte[recBuff.readableBytes()];
             recBuff.readBytes(ret);
-            formatter.formatEntry(ret);
+            entryFormatter.formatEntry(ret);
         } catch (Exception e) {
             System.out.println("N/A. Corrupted.");
         }
@@ -2992,6 +2998,19 @@ private static long getOptionLongValue(CommandLine cmdLine, String option, long
         return defaultVal;
     }
 
+    private long getOptionLedgerIdValue(CommandLine cmdLine, String option, long defaultVal) {
+        if (cmdLine.hasOption(option)) {
+            String val = cmdLine.getOptionValue(option);
+            try {
+                return ledgerIdFormatter.readLedgerId(val);
+            } catch (IllegalArgumentException iae) {
+                System.err.println("ERROR: invalid value for option " + option + " : " + val);
+                return defaultVal;
+            }
+        }
+        return defaultVal;
+    }
+
     private static boolean getOptionBooleanValue(CommandLine cmdLine, String option, boolean defaultVal) {
         if (cmdLine.hasOption(option)) {
             String val = cmdLine.getOptionValue(option);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollector.java
index 87116ded4..f9f9a7b8a 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollector.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollector.java
@@ -43,7 +43,7 @@
          *
          * @param ledgerId Ledger ID to be cleaned
          */
-        void clean(final long ledgerId);
+        void clean(long ledgerId);
     }
 
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteAdvHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteAdvHandle.java
index a6fb2526b..037f7dd72 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteAdvHandle.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteAdvHandle.java
@@ -82,6 +82,6 @@
      * @param data array of bytes to be written
      * @return an handle to the result, in case of success it will return the same value of param entryId
      */
-    CompletableFuture<Long> write(final long entryId, final ByteBuf data);
+    CompletableFuture<Long> write(long entryId, ByteBuf data);
 
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
index b197b1ecf..f963f8b63 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
@@ -22,10 +22,15 @@
 import java.net.URL;
 import java.util.Iterator;
 import java.util.List;
+
 import javax.net.ssl.SSLEngine;
+
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.util.EntryFormatter;
+import org.apache.bookkeeper.util.LedgerIdFormatter;
 import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.bookkeeper.util.StringEntryFormatter;
 import org.apache.commons.configuration.CompositeConfiguration;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
@@ -77,6 +82,9 @@
     // TLS provider factory class name
     protected static final String TLS_PROVIDER_FACTORY_CLASS = "tlsProviderFactoryClass";
 
+    protected static final String LEDGERID_FORMATTER_CLASS = "ledgerIdFormatterClass";
+    protected static final String ENTRY_FORMATTER_CLASS = "entryFormatterClass";
+
     // Enable authentication of the other connection end point (mutual authentication)
     protected static final String TLS_CLIENT_AUTHENTICATION = "tlsClientAuthentication";
 
@@ -373,6 +381,50 @@ public Feature getFeature(String configProperty, Feature defaultValue) {
         }
     }
 
+    /**
+     * Set Ledger id formatter Class.
+     *
+     * @param formatterClass
+     *          LedgerIdFormatter Class
+     */
+    public void setLedgerIdFormatterClass(Class<? extends LedgerIdFormatter> formatterClass) {
+        setProperty(LEDGERID_FORMATTER_CLASS, formatterClass.getName());
+    }
+
+    /**
+     * Get ledger id formatter class.
+     *
+     * @return LedgerIdFormatter class
+     */
+    public Class<? extends LedgerIdFormatter> getLedgerIdFormatterClass()
+        throws ConfigurationException {
+        return ReflectionUtils.getClass(this, LEDGERID_FORMATTER_CLASS,
+                                        null, LedgerIdFormatter.UUIDLedgerIdFormatter.class,
+                                        LedgerIdFormatter.class.getClassLoader());
+    }
+
+    /**
+     * Set entry formatter Class.
+     *
+     * @param formatterClass
+     *          EntryFormatter Class
+     */
+    public void setEntryFormatterClass(Class<? extends EntryFormatter> formatterClass) {
+        setProperty(ENTRY_FORMATTER_CLASS, formatterClass.getName());
+    }
+
+    /**
+     * Get entry formatter class.
+     *
+     * @return EntryFormatter class
+     */
+    public Class<? extends EntryFormatter> getEntryFormatterClass()
+        throws ConfigurationException {
+        return ReflectionUtils.getClass(this, ENTRY_FORMATTER_CLASS,
+                                        null, StringEntryFormatter.class,
+                                        EntryFormatter.class.getClassLoader());
+    }
+
     /**
      * Set the client authentication provider factory class name.
      * If this is not set, no authentication will be used
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java
index 651076fdb..73db12271 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java
@@ -59,9 +59,9 @@
      * @return ledger manager factory instance
      * @throws IOException when fail to initialize the factory.
      */
-    public abstract LedgerManagerFactory initialize(final AbstractConfiguration conf,
-                                                    final ZooKeeper zk,
-                                                    final int factoryVersion)
+    public abstract LedgerManagerFactory initialize(AbstractConfiguration conf,
+                                                    ZooKeeper zk,
+                                                    int factoryVersion)
     throws IOException;
 
     /**
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
index ffaa50a52..6e64b37ee 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
@@ -1323,7 +1323,7 @@ protected int logAndConvertStatus(StatusCode status, int defaultStatus,
 
 
         public abstract void errorOut();
-        public abstract void errorOut(final int rc);
+        public abstract void errorOut(int rc);
 
         protected void errorOutAndRunCallback(final Runnable callback) {
             executor.submitOrdered(ledgerId,
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EntryFormatter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EntryFormatter.java
index 86a5f0e3f..c24e375d7 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EntryFormatter.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EntryFormatter.java
@@ -21,8 +21,7 @@
 
 package org.apache.bookkeeper.util;
 
-import org.apache.commons.configuration.Configuration;
-
+import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,12 +32,6 @@
 
     private static final Logger LOG = LoggerFactory.getLogger(EntryFormatter.class);
 
-    protected Configuration conf;
-
-    public void setConf(Configuration conf) {
-        this.conf = conf;
-    }
-
     /**
      * Format an entry into a readable format.
      *
@@ -54,20 +47,30 @@ public void setConf(Configuration conf) {
      *          Input Stream
      */
     public abstract void formatEntry(java.io.InputStream input);
-
     public static final EntryFormatter STRING_FORMATTER = new StringEntryFormatter();
 
-    public static EntryFormatter newEntryFormatter(Configuration conf, String clsProperty) {
-        String cls = conf.getString(clsProperty, StringEntryFormatter.class.getName());
+    public static EntryFormatter newEntryFormatter(AbstractConfiguration<?> conf) {
         EntryFormatter formatter;
         try {
-            Class<? extends EntryFormatter> aCls = ReflectionUtils.forName(cls, EntryFormatter.class);
-            formatter = ReflectionUtils.newInstance(aCls);
-            formatter.setConf(conf);
+            Class<? extends EntryFormatter> entryFormatterClass = conf.getEntryFormatterClass();
+            formatter = ReflectionUtils.newInstance(entryFormatterClass);
         } catch (Exception e) {
-            LOG.warn("No formatter class found : " + cls, e);
+            LOG.warn("No formatter class found", e);
             LOG.warn("Using Default String Formatter.");
-            formatter = STRING_FORMATTER;
+            formatter = new StringEntryFormatter();
+        }
+        return formatter;
+    }
+
+    public static EntryFormatter newEntryFormatter(String opt, AbstractConfiguration conf) {
+        EntryFormatter formatter;
+        if ("hex".equals(opt)) {
+            formatter = new HexDumpEntryFormatter();
+        } else if ("string".equals(opt)) {
+            formatter = new StringEntryFormatter();
+        } else {
+            LOG.warn("specified unexpected entryformat {}, so default EntryFormatter is used", opt);
+            formatter = newEntryFormatter(conf);
         }
         return formatter;
     }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java
new file mode 100644
index 000000000..ec52cdf5c
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java
@@ -0,0 +1,133 @@
+/*
+ *
+ * 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.util;
+
+import java.util.UUID;
+
+import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Formatter to format a ledgerId.
+ */
+public abstract class LedgerIdFormatter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(LedgerIdFormatter.class);
+
+    /**
+     * Formats the LedgerId according to the type of the Formatter and return it
+     * in String format.
+     *
+     * @param ledgerId
+     * @return
+     */
+    public abstract String formatLedgerId(long ledgerId);
+
+    /**
+     * converts the ledgeridString, which is in format of the type of formatter,
+     * to the long value.
+     *
+     * @param ledgerIdString
+     * @return
+     */
+    public abstract long readLedgerId(String ledgerIdString);
+
+    // Used by BKExtentIdByteArray
+    public static final LedgerIdFormatter LONG_LEDGERID_FORMATTER = new LongLedgerIdFormatter();
+
+    public static LedgerIdFormatter newLedgerIdFormatter(AbstractConfiguration<?> conf) {
+        LedgerIdFormatter formatter;
+        try {
+            Class<? extends LedgerIdFormatter> ledgerIdFormatterClass = conf.getLedgerIdFormatterClass();
+            formatter = ReflectionUtils.newInstance(ledgerIdFormatterClass);
+        } catch (Exception e) {
+            LOG.warn("No formatter class found", e);
+            LOG.warn("Using Default UUID Formatter.");
+            formatter = new UUIDLedgerIdFormatter();
+        }
+        return formatter;
+    }
+
+    public static LedgerIdFormatter newLedgerIdFormatter(String opt, AbstractConfiguration conf) {
+        LedgerIdFormatter formatter;
+        if ("hex".equals(opt)) {
+            formatter = new LedgerIdFormatter.HexLedgerIdFormatter();
+        } else if ("uuid".equals(opt)) {
+            formatter = new LedgerIdFormatter.UUIDLedgerIdFormatter();
+        } else if ("long".equals(opt)) {
+            formatter = new LedgerIdFormatter.LongLedgerIdFormatter();
+        } else {
+            LOG.warn("specified unexpected ledgeridformat {}, so default LedgerIdFormatter is used", opt);
+            formatter = newLedgerIdFormatter(conf);
+        }
+        return formatter;
+    }
+
+    /**
+     * long ledgerId formatter.
+     */
+    public static class LongLedgerIdFormatter extends LedgerIdFormatter {
+
+        @Override
+        public String formatLedgerId(long ledgerId) {
+            return Long.toString(ledgerId);
+        }
+
+        @Override
+        public long readLedgerId(String ledgerIdString) {
+            return Long.parseLong(ledgerIdString.trim());
+        }
+    }
+
+    /**
+     * hex ledgerId formatter.
+     */
+    public static class HexLedgerIdFormatter extends LedgerIdFormatter {
+
+        @Override
+        public String formatLedgerId(long ledgerId) {
+            return Long.toHexString(ledgerId);
+        }
+
+        @Override
+        public long readLedgerId(String ledgerIdString) {
+            return Long.valueOf(ledgerIdString.trim(), 16);
+        }
+    }
+
+    /**
+     * uuid ledgerId formatter.
+     */
+    public static class UUIDLedgerIdFormatter extends LedgerIdFormatter {
+
+        @Override
+        public String formatLedgerId(long ledgerId) {
+            return (new UUID(0, ledgerId)).toString();
+        }
+
+        @Override
+        public long readLedgerId(String ledgerIdString) {
+            return UUID.fromString(ledgerIdString.trim()).getLeastSignificantBits();
+        }
+    }
+}
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 f42ec9e2d..4a1658e1e 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
@@ -43,6 +43,8 @@
 import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener;
 import org.apache.bookkeeper.discover.ZKRegistrationManager;
 import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.EntryFormatter;
+import org.apache.bookkeeper.util.LedgerIdFormatter;
 import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -73,7 +75,7 @@
 
     @Before
     public void setup() throws Exception {
-        this.shell = new BookieShell();
+        this.shell = new BookieShell(LedgerIdFormatter.LONG_LEDGERID_FORMATTER, EntryFormatter.STRING_FORMATTER);
         this.admin = PowerMockito.mock(BookKeeperAdmin.class);
         whenNew(BookKeeperAdmin.class)
             .withParameterTypes(ClientConfiguration.class)
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java
index 49784c60f..2a0e38454 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java
@@ -685,7 +685,7 @@ public void testReadWriteWithV2WireProtocol() throws Exception {
                 try (LedgerHandle lh2 = bkc.createLedger(digestType, "testPasswd".getBytes())) {
                     ledgerId = lh2.getId();
                     lh2.addEntry(data);
-                    try (LedgerHandle lh2_fence = bkc.openLedger(ledgerId, digestType, "testPasswd".getBytes())) {
+                    try (LedgerHandle lh2Fence = bkc.openLedger(ledgerId, digestType, "testPasswd".getBytes())) {
                     }
                     try {
                         lh2.addEntry(data);


 

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


With regards,
Apache Git Services