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/11/12 07:14:09 UTC

[GitHub] jiazhai closed pull request #714: Issue-612 BOOKKEEPER-818: Make bookie recovery work with recovering multiple bookies

jiazhai closed pull request #714: Issue-612 BOOKKEEPER-818: Make bookie recovery work with recovering multiple bookies
URL: https://github.com/apache/bookkeeper/pull/714
 
 
   

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 2d261453e..d564254e1 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
@@ -20,6 +20,7 @@
 
 import static com.google.common.base.Charsets.UTF_8;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.AbstractFuture;
 import java.io.File;
@@ -42,9 +43,13 @@
 import java.util.Enumeration;
 import java.util.Formatter;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException;
@@ -92,6 +97,7 @@
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.mutable.MutableBoolean;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -287,6 +293,11 @@ int runCmd(CommandLine cmdLine) throws Exception {
 
         public RecoverCmd() {
             super(CMD_RECOVER);
+            opts.addOption("q", "query", false, "Query the ledgers that contain given bookies");
+            opts.addOption("dr", "dryrun", false, "Printing the recovery plan w/o doing actual recovery");
+            opts.addOption("f", "force", false, "Force recovery without confirmation");
+            opts.addOption("l", "ledger", true, "Recover a specific ledger");
+            opts.addOption("sk", "skipOpenLedgers", false, "Skip recovering open ledgers");
             opts.addOption("d", "deleteCookie", false, "Delete cookie node for the bookie.");
         }
 
@@ -302,7 +313,7 @@ String getDescription() {
 
         @Override
         String getUsage() {
-            return "recover [-deleteCookie] <bookieSrc> [bookieDest]";
+            return "recover [-deleteCookie] <bookieSrc[:bookieSrc]>";
         }
 
         @Override
@@ -312,54 +323,160 @@ int runCmd(CommandLine cmdLine) throws Exception {
                 throw new MissingArgumentException(
                         "'bookieSrc' argument required");
             }
+            if (args.length > 1) {
+                System.err.println("The provided bookie dest " + args[1] + " will be ignored!");
+            }
+            boolean query = cmdLine.hasOption("q");
+            boolean dryrun = cmdLine.hasOption("dr");
+            boolean force = cmdLine.hasOption("f");
+            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"));
+                }
+            }
+
+            // Get bookies list
+            final String[] bookieStrs = args[0].split(",");
+            final Set<BookieSocketAddress> bookieAddrs = new HashSet<>();
+            for (String bookieStr : bookieStrs) {
+                final String bookieStrParts[] = bookieStr.split(":");
+                if (bookieStrParts.length != 2) {
+                    System.err.println("BookieSrcs has invalid bookie address format (host:port expected) : "
+                            + bookieStr);
+                    return -1;
+                }
+                bookieAddrs.add(new BookieSocketAddress(bookieStrParts[0],
+                        Integer.parseInt(bookieStrParts[1])));
+            }
+
+            if (!force) {
+                System.err.println("Bookies : " + bookieAddrs);
+                if (!IOUtils.confirmPrompt("Are you sure to recover them : (Y/N)")) {
+                    System.err.println("Give up!");
+                    return -1;
+                }
+            }
 
+            LOG.info("Constructing admin");
             ClientConfiguration adminConf = new ClientConfiguration(bkConf);
             BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+            LOG.info("Construct admin : {}", admin);
             try {
-                return bkRecovery(adminConf, admin, args, cmdLine.hasOption("d"));
+                if (query) {
+                    return bkQuery(admin, bookieAddrs);
+                }
+                if (null != ledgerId) {
+                    return bkRecoveryLedger(admin, ledgerId, bookieAddrs, dryrun, skipOpenLedgers, removeCookies);
+                }
+                return bkRecovery(admin, bookieAddrs, dryrun, skipOpenLedgers, removeCookies);
             } finally {
                 admin.close();
             }
         }
 
-        private int bkRecovery(ClientConfiguration conf, BookKeeperAdmin bkAdmin,
-                               String[] args, boolean deleteCookie)
-                throws InterruptedException, BKException, BookieException, IOException {
-            final String bookieSrcString[] = args[0].split(":");
-            if (bookieSrcString.length != 2) {
-                System.err.println("BookieSrc inputted has invalid format"
-                        + "(host:port expected): " + args[0]);
-                return -1;
-            }
-            final BookieSocketAddress bookieSrc = new BookieSocketAddress(
-                    bookieSrcString[0], Integer.parseInt(bookieSrcString[1]));
-            BookieSocketAddress bookieDest = null;
-            if (args.length >= 2) {
-                final String bookieDestString[] = args[1].split(":");
-                if (bookieDestString.length < 2) {
-                    System.err.println("BookieDest inputted has invalid format"
-                            + "(host:port expected): " + args[1]);
-                    return -1;
+        private int bkQuery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs)
+                throws InterruptedException, BKException {
+            SortedMap<Long, LedgerMetadata> ledgersContainBookies =
+                    bkAdmin.getLedgersContainBookies(bookieAddrs);
+            System.err.println("NOTE: Bookies in inspection list are marked with '*'.");
+            for (Map.Entry<Long, LedgerMetadata> ledger : ledgersContainBookies.entrySet()) {
+                System.out.println("ledger " + ledger.getKey() + " : " + ledger.getValue().getState());
+                Map<Long, Integer> numBookiesToReplacePerEnsemble =
+                        inspectLedger(ledger.getValue(), bookieAddrs);
+                System.out.print("summary: [");
+                for (Map.Entry<Long, Integer> entry : numBookiesToReplacePerEnsemble.entrySet()) {
+                    System.out.print(entry.getKey() + "=" + entry.getValue() + ", ");
                 }
-                bookieDest = new BookieSocketAddress(bookieDestString[0],
-                        Integer.parseInt(bookieDestString[1]));
+                System.out.println("]");
+                System.out.println();
             }
+            System.err.println("Done");
+            return 0;
+        }
 
-            bkAdmin.recoverBookieData(bookieSrc, bookieDest);
-            if (deleteCookie) {
-                ServerConfiguration serverConf = new ServerConfiguration();
-                serverConf.addConfiguration(conf);
-                RegistrationManager rm = new ZKRegistrationManager();
-                try {
-                    rm.initialize(serverConf, () -> {}, NullStatsLogger.INSTANCE);
-                    Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, bookieSrc);
-                    cookie.getValue().deleteFromRegistrationManager(rm, bookieSrc, cookie.getVersion());
-                } catch (CookieNotFoundException nne) {
-                    LOG.warn("No cookie to remove for {} : ", bookieSrc, nne);
+        private Map<Long, Integer> inspectLedger(LedgerMetadata metadata, Set<BookieSocketAddress> bookiesToInspect) {
+            Map<Long, Integer> numBookiesToReplacePerEnsemble = new TreeMap<Long, Integer>();
+            for (Map.Entry<Long, ArrayList<BookieSocketAddress>> ensemble : metadata.getEnsembles().entrySet()) {
+                ArrayList<BookieSocketAddress> bookieList = ensemble.getValue();
+                System.out.print(ensemble.getKey() + ":\t");
+                int numBookiesToReplace = 0;
+                for (BookieSocketAddress bookie: bookieList) {
+                    System.out.print(bookie);
+                    if (bookiesToInspect.contains(bookie)) {
+                        System.out.print("*");
+                        ++numBookiesToReplace;
+                    } else {
+                        System.out.print(" ");
+                    }
+                    System.out.print(" ");
                 }
+                System.out.println();
+                numBookiesToReplacePerEnsemble.put(ensemble.getKey(), numBookiesToReplace);
+            }
+            return numBookiesToReplacePerEnsemble;
+        }
+
+        private int bkRecoveryLedger(BookKeeperAdmin bkAdmin,
+                                     long lid,
+                                     Set<BookieSocketAddress> bookieAddrs,
+                                     boolean dryrun,
+                                     boolean skipOpenLedgers,
+                                     boolean removeCookies)
+                throws InterruptedException, BKException, KeeperException {
+            bkAdmin.recoverBookieData(lid, bookieAddrs, dryrun, skipOpenLedgers);
+            if (removeCookies) {
+                deleteCookies(bkAdmin.getConf(), bookieAddrs);
             }
             return 0;
         }
+
+        private int bkRecovery(BookKeeperAdmin bkAdmin,
+                               Set<BookieSocketAddress> bookieAddrs,
+                               boolean dryrun,
+                               boolean skipOpenLedgers,
+                               boolean removeCookies)
+                throws InterruptedException, BKException, KeeperException {
+            bkAdmin.recoverBookieData(bookieAddrs, dryrun, skipOpenLedgers);
+            if (removeCookies) {
+                deleteCookies(bkAdmin.getConf(), bookieAddrs);
+            }
+            return 0;
+        }
+
+        private void deleteCookies(ClientConfiguration conf,
+                                   Set<BookieSocketAddress> bookieAddrs) throws BKException {
+            ServerConfiguration serverConf = new ServerConfiguration(conf);
+            RegistrationManager rm = new ZKRegistrationManager();
+            try {
+                rm.initialize(serverConf, () -> {}, NullStatsLogger.INSTANCE);
+                for (BookieSocketAddress addr : bookieAddrs) {
+                    deleteCookie(rm, addr);
+                }
+            } catch (BookieException be) {
+                BKException bke = new BKException.MetaStoreException();
+                bke.initCause(be);
+                throw bke;
+            } finally {
+                rm.close();
+            }
+        }
+
+        private void deleteCookie(RegistrationManager rm,
+                                  BookieSocketAddress bookieSrc) throws BookieException {
+            try {
+                Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, bookieSrc);
+                cookie.getValue().deleteFromRegistrationManager(rm, bookieSrc, cookie.getVersion());
+            } catch (CookieNotFoundException nne) {
+                LOG.warn("No cookie to remove for {} : ", bookieSrc, nne);
+            }
+        }
+
     }
 
     /**
@@ -2058,6 +2175,11 @@ private void printShellUsage() {
         }
     }
 
+    @VisibleForTesting
+    public int execute(String... args) throws Exception {
+        return run(args);
+    }
+
     @Override
     public int run(String[] args) throws Exception {
         if (args.length <= 0) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
index aae36b872..e761222f6 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
@@ -42,10 +42,13 @@
 import java.util.Optional;
 import java.util.Random;
 import java.util.Set;
+import java.util.SortedMap;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Predicate;
 import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
 import org.apache.bookkeeper.client.AsyncCallback.RecoverCallback;
@@ -182,6 +185,10 @@ public BookKeeperAdmin(final BookKeeper bkc) {
         this(bkc, NullStatsLogger.INSTANCE);
     }
 
+    public ClientConfiguration getConf() {
+        return bkc.getConf();
+    }
+
     /**
      * Gracefully release resources that this client uses.
      *
@@ -442,6 +449,69 @@ public SyncObject() {
         }
     }
 
+    public SortedMap<Long, LedgerMetadata> getLedgersContainBookies(Set<BookieSocketAddress> bookies)
+            throws InterruptedException, BKException {
+        final SyncObject sync = new SyncObject();
+        final AtomicReference<SortedMap<Long, LedgerMetadata>> resultHolder =
+                new AtomicReference<SortedMap<Long, LedgerMetadata>>(null);
+        asyncGetLedgersContainBookies(bookies, new GenericCallback<SortedMap<Long, LedgerMetadata>>() {
+            @Override
+            public void operationComplete(int rc, SortedMap<Long, LedgerMetadata> result) {
+                LOG.info("GetLedgersContainBookies completed with rc : {}", rc);
+                synchronized (sync) {
+                    sync.rc = rc;
+                    sync.value = true;
+                    resultHolder.set(result);
+                    sync.notify();
+                }
+            }
+        });
+        synchronized (sync) {
+            while (sync.value == false) {
+                sync.wait();
+            }
+        }
+        if (sync.rc != BKException.Code.OK) {
+            throw BKException.create(sync.rc);
+        }
+        return resultHolder.get();
+    }
+
+    public void asyncGetLedgersContainBookies(final Set<BookieSocketAddress> bookies,
+                                              final GenericCallback<SortedMap<Long, LedgerMetadata>> callback) {
+        final SortedMap<Long, LedgerMetadata> ledgers = new ConcurrentSkipListMap<Long, LedgerMetadata>();
+        bkc.getLedgerManager().asyncProcessLedgers(new Processor<Long>() {
+            @Override
+            public void process(final Long lid, final AsyncCallback.VoidCallback cb) {
+                bkc.getLedgerManager().readLedgerMetadata(lid, new GenericCallback<LedgerMetadata>() {
+                    @Override
+                    public void operationComplete(int rc, LedgerMetadata metadata) {
+                        if (BKException.Code.NoSuchLedgerExistsException == rc) {
+                            // the ledger was deleted during this iteration.
+                            cb.processResult(BKException.Code.OK, null, null);
+                            return;
+                        } else if (BKException.Code.OK != rc) {
+                            cb.processResult(rc, null, null);
+                            return;
+                        }
+                        Set<BookieSocketAddress> bookiesInLedger = metadata.getBookiesInThisLedger();
+                        Sets.SetView<BookieSocketAddress> intersection =
+                                Sets.intersection(bookiesInLedger, bookies);
+                        if (!intersection.isEmpty()) {
+                            ledgers.put(lid, metadata);
+                        }
+                        cb.processResult(BKException.Code.OK, null, null);
+                    }
+                });
+            }
+        }, new AsyncCallback.VoidCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx) {
+                callback.operationComplete(rc, ledgers);
+            }
+        }, null, BKException.Code.OK, BKException.Code.MetaStoreException);
+    }
+
     /**
      * Synchronous method to rebuild and recover the ledger fragments data that
      * was stored on the source bookie. That bookie could have failed completely
@@ -456,11 +526,8 @@ public SyncObject() {
      * @param bookieSrc
      *            Source bookie that had a failure. We want to replicate the
      *            ledger fragments that were stored there.
-     * @param bookieDest
-     *            Optional destination bookie that if passed, we will copy all
-     *            of the ledger fragments from the source bookie over to it.
      */
-    public void recoverBookieData(final BookieSocketAddress bookieSrc, final BookieSocketAddress bookieDest)
+    public void recoverBookieData(final BookieSocketAddress bookieSrc)
             throws InterruptedException, BKException {
         Set<BookieSocketAddress> bookiesSrc = Sets.newHashSet(bookieSrc);
         recoverBookieData(bookiesSrc);
@@ -490,6 +557,34 @@ public void recoverComplete(int rc, Object ctx) {
 
         // Wait for the async method to complete.
         synchronized (sync) {
+            while (sync.value == false) {
+                sync.wait();
+            }
+        }
+        if (sync.rc != BKException.Code.OK) {
+            throw BKException.create(sync.rc);
+        }
+    }
+
+    public void recoverBookieData(final long lid,
+                                  final Set<BookieSocketAddress> bookiesSrc,
+                                  boolean dryrun,
+                                  boolean skipOpenLedgers)
+            throws InterruptedException, BKException {
+        SyncObject sync = new SyncObject();
+        // Call the async method to recover bookie data.
+        asyncRecoverBookieData(lid, bookiesSrc, dryrun, skipOpenLedgers, (rc, ctx) -> {
+            LOG.info("Recover bookie for {} completed with rc : {}", lid, rc);
+            SyncObject syncObject = (SyncObject) ctx;
+            synchronized (syncObject) {
+                syncObject.rc = rc;
+                syncObject.value = true;
+                syncObject.notify();
+            }
+        }, sync);
+
+        // Wait for the async method to complete.
+        synchronized (sync) {
             while (!sync.value) {
                 sync.wait();
             }
@@ -498,7 +593,7 @@ public void recoverComplete(int rc, Object ctx) {
             throw BKException.create(sync.rc);
         }
     }
-	
+
     /**
      * Async method to rebuild and recover the ledger fragments data that was
      * stored on the source bookie. That bookie could have failed completely and
@@ -536,6 +631,30 @@ public void asyncRecoverBookieData(final Set<BookieSocketAddress> bookieSrc, boo
     }
 
     /**
+     * Recover a specific ledger.
+     *
+     * @param lid
+     *          ledger to recover
+     * @param bookieSrc
+     *          Source bookies that had a failure. We want to replicate the ledger fragments that were stored there.
+     * @param dryrun
+     *          dryrun the recover procedure.
+     * @param skipOpenLedgers
+     *            Skip recovering open ledgers.
+     * @param callback
+     *          RecoverCallback to invoke once all of the data on the dead
+     *          bookie has been recovered and replicated.
+     * @param context
+     *          Context for the RecoverCallback to call.
+     */
+    public void asyncRecoverBookieData(long lid, final Set<BookieSocketAddress> bookieSrc, boolean dryrun,
+                                       boolean skipOpenLedgers, final RecoverCallback callback, final Object context) {
+        AsyncCallback.VoidCallback callbackWrapper = (rc, path, ctx)
+            -> callback.recoverComplete(bkc.getReturnRc(rc), context);
+        recoverLedger(bookieSrc, lid, dryrun, skipOpenLedgers, callbackWrapper);
+    }
+
+    /**
      * This method asynchronously polls ZK to get the current set of active
      * ledgers. From this, we can open each ledger and look at the metadata to
      * determine if any of the ledger fragments for it were stored at the dead
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
index 5683105de..76a8b6142 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
@@ -17,33 +17,32 @@
  */
 package org.apache.bookkeeper.client;
 
+import static com.google.common.base.Charsets.UTF_8;
+
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.collect.Maps;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.TextFormat;
-
-import org.apache.bookkeeper.net.BookieSocketAddress;
-import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat;
-import org.apache.bookkeeper.versioning.Version;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.StringReader;
 import java.nio.CharBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Maps;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat;
+import org.apache.bookkeeper.versioning.Version;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class encapsulates all the ledger metadata that is persistently stored
@@ -227,7 +226,7 @@ public boolean isInRecovery() {
         return state == LedgerMetadataFormat.State.IN_RECOVERY;
     }
 
-    LedgerMetadataFormat.State getState() {
+    public LedgerMetadataFormat.State getState() {
         return state;
     }
 
@@ -633,4 +632,12 @@ void mergeEnsembles(SortedMap<Long, ArrayList<BookieSocketAddress>> newEnsembles
         }
     }
 
+    Set<BookieSocketAddress> getBookiesInThisLedger() {
+        Set<BookieSocketAddress> bookies = new HashSet<BookieSocketAddress>();
+        for (ArrayList<BookieSocketAddress> ensemble : ensembles.values()) {
+            bookies.addAll(ensemble);
+        }
+        return bookies;
+    }
+
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/RecoveryBookieService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/RecoveryBookieService.java
index b079f1b36..cfbb92134 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/RecoveryBookieService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/RecoveryBookieService.java
@@ -25,7 +25,6 @@
 import java.util.concurrent.ExecutorService;
 import org.apache.bookkeeper.bookie.Cookie;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
-import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.http.service.HttpEndpointService;
@@ -69,13 +68,11 @@ public RecoveryBookieService(ServerConfiguration conf, BookKeeperAdmin bka, Exec
      * Example body as this:
      * {
      *   "bookie_src": [ "bookie_src1", "bookie_src2"... ],
-     *   "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ],
      *   "delete_cookie": <bool_value>
      * }
      */
     static class RecoveryRequestJsonBody {
         public List<String> bookie_src;
-        public List<String> bookie_dest;
         public boolean delete_cookie;
     }
 
@@ -94,8 +91,6 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception {
         try {
             requestJsonBody = JsonUtil.fromJson(requestBody, RecoveryRequestJsonBody.class);
             LOG.debug("bookie_src: [" + requestJsonBody.bookie_src.get(0)
-                + "],  bookie_dest: ["
-                + ((requestJsonBody.bookie_dest == null) ? "null" : requestJsonBody.bookie_dest.get(0))
                 + "],  delete_cookie: [" + requestJsonBody.delete_cookie + "]");
         } catch (JsonUtil.ParseJsonException e) {
             LOG.error("Meet Exception: ", e);
@@ -114,19 +109,11 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception {
             String bookieSrcString[] = requestJsonBody.bookie_src.get(0).split(":");
             BookieSocketAddress bookieSrc = new BookieSocketAddress(
               bookieSrcString[0], Integer.parseInt(bookieSrcString[1]));
-            final BookieSocketAddress bookieDest;
-            if ((requestJsonBody.bookie_dest != null) && !requestJsonBody.bookie_dest.isEmpty()) {
-                String bookieDestString[] = requestJsonBody.bookie_dest.get(0).split(":");
-                bookieDest = new BookieSocketAddress(bookieDestString[0],
-                  Integer.parseInt(bookieDestString[1]));
-            } else {
-                bookieDest = null;
-            }
             boolean deleteCookie = requestJsonBody.delete_cookie;
             executor.execute(() -> {
                 try {
                     LOG.info("Start recovering bookie.");
-                    bka.recoverBookieData(bookieSrc, bookieDest);
+                    bka.recoverBookieData(bookieSrc);
                     if (deleteCookie) {
                         Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, bookieSrc);
                         cookie.getValue().deleteFromRegistrationManager(rm, bookieSrc, cookie.getVersion());
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
new file mode 100644
index 000000000..f42ec9e2d
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
@@ -0,0 +1,281 @@
+/*
+ * 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.bookie;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+import com.google.common.collect.Maps;
+import java.util.Set;
+import java.util.SortedMap;
+import org.apache.bookkeeper.bookie.BookieShell.MyCommand;
+import org.apache.bookkeeper.bookie.BookieShell.RecoverCmd;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener;
+import org.apache.bookkeeper.discover.ZKRegistrationManager;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.versioning.LongVersion;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.MissingArgumentException;
+import org.apache.commons.cli.ParseException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test for {@link BookieShell}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(BookieShell.class)
+public class BookieShellTest {
+
+    private ClientConfiguration clientConf;
+    private BookieShell shell;
+    private BookKeeperAdmin admin;
+    private ZKRegistrationManager rm;
+    private Cookie cookie;
+    private Version version;
+
+    @Before
+    public void setup() throws Exception {
+        this.shell = new BookieShell();
+        this.admin = PowerMockito.mock(BookKeeperAdmin.class);
+        whenNew(BookKeeperAdmin.class)
+            .withParameterTypes(ClientConfiguration.class)
+            .withArguments(any(ClientConfiguration.class))
+            .thenReturn(admin);
+        this.clientConf = new ClientConfiguration();
+        when(admin.getConf()).thenReturn(this.clientConf);
+        this.rm = PowerMockito.mock(ZKRegistrationManager.class);
+        this.cookie = Cookie.newBuilder()
+            .setBookieHost("127.0.0.1:3181")
+            .setInstanceId("xyz")
+            .setJournalDirs("/path/to/journal/dir")
+            .setLedgerDirs("/path/to/journal/dir")
+            .setLayoutVersion(Cookie.CURRENT_COOKIE_LAYOUT_VERSION)
+            .build();
+        this.version = new LongVersion(1L);
+        when(rm.readCookie(anyString()))
+            .thenReturn(new Versioned<>(cookie.toString().getBytes(UTF_8), version));
+        whenNew(ZKRegistrationManager.class)
+            .withNoArguments()
+            .thenReturn(rm);
+    }
+
+    private static CommandLine parseCommandLine(MyCommand cmd, String... args) throws ParseException {
+        BasicParser parser = new BasicParser();
+        return parser.parse(cmd.getOptions(), args);
+    }
+
+    @Test
+    public void testRecoverCmdMissingArgument() throws Exception {
+        RecoverCmd cmd = (RecoverCmd) shell.commands.get("recover");
+        CommandLine cmdLine = parseCommandLine(cmd);
+        try {
+            cmd.runCmd(cmdLine);
+            fail("should fail running command when the arguments are missing");
+        } catch (MissingArgumentException e) {
+            // expected
+        }
+        PowerMockito.verifyNew(BookKeeperAdmin.class, never()).withArguments(any(ClientConfiguration.class));
+    }
+
+    @Test
+    public void testRecoverCmdInvalidBookieAddress() throws Exception {
+        RecoverCmd cmd = (RecoverCmd) shell.commands.get("recover");
+        CommandLine cmdLine = parseCommandLine(cmd, "127.0.0.1");
+        assertEquals(-1, cmd.runCmd(cmdLine));
+        PowerMockito.verifyNew(BookKeeperAdmin.class, never()).withArguments(any(ClientConfiguration.class));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testRecoverCmdQuery() throws Exception {
+        SortedMap<Long, LedgerMetadata> ledgersContainBookies = Maps.newTreeMap();
+        when(admin.getLedgersContainBookies(any(Set.class)))
+            .thenReturn(ledgersContainBookies);
+
+        RecoverCmd cmd = (RecoverCmd) shell.commands.get("recover");
+        CommandLine cmdLine = parseCommandLine(cmd, "-force", "-q", "127.0.0.1:3181");
+        assertEquals(0, cmd.runCmd(cmdLine));
+        PowerMockito
+            .verifyNew(BookKeeperAdmin.class, times(1))
+            .withArguments(any(ClientConfiguration.class));
+        verify(admin, times(1)).getLedgersContainBookies(any(Set.class));
+        verify(admin, times(1)).close();
+    }
+
+    @Test
+    public void testRecoverCmdRecoverLedgerDefault() throws Exception {
+        // default behavior
+        testRecoverCmdRecoverLedger(
+            12345, false, false, false,
+            "-force", "-l", "12345", "127.0.0.1:3181");
+    }
+
+    @Test
+    public void testRecoverCmdRecoverLedgerDeleteCookie() throws Exception {
+        // dryrun
+        testRecoverCmdRecoverLedger(
+            12345, false, false, true,
+            "-force", "-l", "12345", "-deleteCookie", "127.0.0.1:3181");
+    }
+
+    @Test
+    public void testRecoverCmdRecoverLedgerSkipOpenLedgersDeleteCookie() throws Exception {
+        // dryrun
+        testRecoverCmdRecoverLedger(
+            12345, false, true, true,
+            "-force", "-l", "12345", "-deleteCookie", "-skipOpenLedgers", "127.0.0.1:3181");
+    }
+
+    @Test
+    public void testRecoverCmdRecoverLedgerDryrun() throws Exception {
+        // dryrun
+        testRecoverCmdRecoverLedger(
+            12345, true, false, false,
+            "-force", "-l", "12345", "-dryrun", "127.0.0.1:3181");
+    }
+
+    @Test
+    public void testRecoverCmdRecoverLedgerDryrunDeleteCookie() throws Exception {
+        // dryrun & removeCookie : removeCookie should be false
+        testRecoverCmdRecoverLedger(
+            12345, true, false, false,
+            "-force", "-l", "12345", "-dryrun", "-deleteCookie", "127.0.0.1:3181");
+    }
+
+    @SuppressWarnings("unchecked")
+    void testRecoverCmdRecoverLedger(long ledgerId,
+                                     boolean dryrun,
+                                     boolean skipOpenLedgers,
+                                     boolean removeCookies,
+                                     String... args) throws Exception {
+        RecoverCmd cmd = (RecoverCmd) shell.commands.get("recover");
+        CommandLine cmdLine = parseCommandLine(cmd, args);
+        assertEquals(0, cmd.runCmd(cmdLine));
+        PowerMockito
+            .verifyNew(BookKeeperAdmin.class, times(1))
+            .withArguments(any(ClientConfiguration.class));
+        verify(admin, times(1))
+            .recoverBookieData(eq(ledgerId), any(Set.class), eq(dryrun), eq(skipOpenLedgers));
+        verify(admin, times(1)).close();
+        if (removeCookies) {
+            PowerMockito
+                .verifyNew(ZKRegistrationManager.class, times(1))
+                .withNoArguments();
+            verify(rm, times(1)).initialize(
+                any(ServerConfiguration.class), any(RegistrationListener.class), eq(NullStatsLogger.INSTANCE));
+            verify(rm, times(1)).readCookie(anyString());
+            verify(rm, times(1)).removeCookie(anyString(), eq(version));
+        } else {
+            PowerMockito
+                .verifyNew(ZKRegistrationManager.class, never())
+                .withNoArguments();
+        }
+    }
+
+    @Test
+    public void testRecoverCmdRecoverDefault() throws Exception {
+        // default behavior
+        testRecoverCmdRecover(
+            false, false, false,
+            "-force", "127.0.0.1:3181");
+    }
+
+    @Test
+    public void testRecoverCmdRecoverDeleteCookie() throws Exception {
+        // dryrun
+        testRecoverCmdRecover(
+            false, false, true,
+            "-force", "-deleteCookie", "127.0.0.1:3181");
+    }
+
+    @Test
+    public void testRecoverCmdRecoverSkipOpenLedgersDeleteCookie() throws Exception {
+        // dryrun
+        testRecoverCmdRecover(
+            false, true, true,
+            "-force", "-deleteCookie", "-skipOpenLedgers", "127.0.0.1:3181");
+    }
+
+    @Test
+    public void testRecoverCmdRecoverDryrun() throws Exception {
+        // dryrun
+        testRecoverCmdRecover(
+            true, false, false,
+            "-force", "-dryrun", "127.0.0.1:3181");
+    }
+
+    @Test
+    public void testRecoverCmdRecoverDryrunDeleteCookie() throws Exception {
+        // dryrun & removeCookie : removeCookie should be false
+        testRecoverCmdRecover(
+            true, false, false,
+            "-force", "-dryrun", "-deleteCookie", "127.0.0.1:3181");
+    }
+
+    @SuppressWarnings("unchecked")
+    void testRecoverCmdRecover(boolean dryrun,
+                               boolean skipOpenLedgers,
+                               boolean removeCookies,
+                               String... args) throws Exception {
+        RecoverCmd cmd = (RecoverCmd) shell.commands.get("recover");
+        CommandLine cmdLine = parseCommandLine(cmd, args);
+        assertEquals(0, cmd.runCmd(cmdLine));
+        PowerMockito
+            .verifyNew(BookKeeperAdmin.class, times(1))
+            .withArguments(any(ClientConfiguration.class));
+        verify(admin, times(1))
+            .recoverBookieData(any(Set.class), eq(dryrun), eq(skipOpenLedgers));
+        verify(admin, times(1)).close();
+        if (removeCookies) {
+            PowerMockito
+                .verifyNew(ZKRegistrationManager.class, times(1))
+                .withNoArguments();
+            verify(rm, times(1)).initialize(
+                any(ServerConfiguration.class), any(RegistrationListener.class), eq(NullStatsLogger.INSTANCE));
+            verify(rm, times(1)).readCookie(anyString());
+            verify(rm, times(1)).removeCookie(anyString(), eq(version));
+        } else {
+            PowerMockito
+                .verifyNew(ZKRegistrationManager.class, never())
+                .withNoArguments();
+        }
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
index f415152c4..e4c5dccc8 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
@@ -476,7 +476,6 @@ public void testBookKeeperAdmin() throws Exception {
             BookieSocketAddress bookieToKill = getBookie(0);
             killBookie(bookieToKill);
             startNewBookie();
-            BookieSocketAddress newBookie = getBookie(2);
 
             CheckerCb checkercb = new CheckerCb();
             LedgerChecker lc = new LedgerChecker(bk);
@@ -507,7 +506,7 @@ public void testBookKeeperAdmin() throws Exception {
             }
 
             try {
-                bkadmin.recoverBookieData(bookieToKill, newBookie);
+                bkadmin.recoverBookieData(bookieToKill);
                 fail("Shouldn't be able to recover with a closed client");
             } catch (BKException.BKClientClosedException cce) {
                 // correct behaviour
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
index a9ccf89d1..1a4a3ef76 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
@@ -271,7 +271,7 @@ void metadataConflictWithRecovery(BookKeeper bkc) throws Exception {
         for (int i = 0; i < numEntries; i++) {
             lh.addEntry(data);
         }
-        bkAdmin.recoverBookieData(bookieToKill, null);
+        bkAdmin.recoverBookieData(bookieToKill);
         // fail another bookie to cause ensemble change again
         bookieToKill = lh.getLedgerMetadata().getEnsemble(2 * numEntries - 1).get(1);
         ServerConfiguration confOfKilledBookie = killBookie(bookieToKill);
@@ -433,7 +433,7 @@ public void testSyncBookieRecoveryToSpecificBookie() throws Exception {
           newBookiePort);
         LOG.info("Now recover the data on the killed bookie (" + bookieSrc + ") and replicate it to the new one ("
           + bookieDest + ")");
-        bkAdmin.recoverBookieData(bookieSrc, bookieDest);
+        bkAdmin.recoverBookieData(bookieSrc);
 
         // Verify the recovered ledger entries are okay.
         verifyRecoveredLedgers(lhs, 0, 2 * numMsgs - 1);
@@ -477,10 +477,9 @@ public void testSyncBookieRecoveryToRandomBookies() throws Exception {
         // Call the sync recover bookie method.
         BookieSocketAddress bookieSrc = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(),
           initialPort);
-        BookieSocketAddress bookieDest = null;
         LOG.info("Now recover the data on the killed bookie (" + bookieSrc
           + ") and replicate it to a random available one");
-        bkAdmin.recoverBookieData(bookieSrc, bookieDest);
+        bkAdmin.recoverBookieData(bookieSrc);
 
         // Verify the recovered ledger entries are okay.
         verifyRecoveredLedgers(lhs, 0, 2 * numMsgs - 1);
@@ -639,11 +638,10 @@ public void testBookieRecoveryOnClosedLedgers() throws Exception {
         // start a new bookie
         startNewBookie();
 
-        BookieSocketAddress bookieDest = null;
         LOG.info("Now recover the data on the killed bookie (" + bookieToKill
           + ") and replicate it to a random available one");
 
-        bkAdmin.recoverBookieData(bookieToKill, bookieDest);
+        bkAdmin.recoverBookieData(bookieToKill);
         for (LedgerHandle lh : lhs) {
             assertTrue("Not fully replicated", verifyFullyReplicated(lh, numMsgs));
             lh.close();
@@ -669,11 +667,10 @@ public void testBookieRecoveryOnOpenedLedgers() throws Exception {
         // start a new bookie
         startNewBookie();
 
-        BookieSocketAddress bookieDest = null;
         LOG.info("Now recover the data on the killed bookie (" + bookieToKill
           + ") and replicate it to a random available one");
 
-        bkAdmin.recoverBookieData(bookieToKill, bookieDest);
+        bkAdmin.recoverBookieData(bookieToKill);
 
         for (LedgerHandle lh : lhs) {
             assertTrue("Not fully replicated", verifyFullyReplicated(lh, numMsgs));
@@ -720,7 +717,7 @@ public void testBookieRecoveryOnInRecoveryLedger() throws Exception {
         }
 
         try {
-            bkAdmin.recoverBookieData(bookieToKill, null);
+            bkAdmin.recoverBookieData(bookieToKill);
             fail("Should have thrown exception");
         } catch (BKException.BKLedgerRecoveryException bke) {
             // correct behaviour
@@ -731,7 +728,7 @@ public void testBookieRecoveryOnInRecoveryLedger() throws Exception {
         bsConfs.add(conf2);
 
         // recover them
-        bkAdmin.recoverBookieData(bookieToKill, null);
+        bkAdmin.recoverBookieData(bookieToKill);
 
         for (LedgerHandle lh : lhs) {
             assertTrue("Not fully replicated", verifyFullyReplicated(lh, numMsgs));
@@ -773,7 +770,7 @@ public void testAsyncBookieRecoveryToRandomBookiesNotEnoughBookies() throws Exce
         // Initiate the sync object
         sync.value = false;
         try {
-            bkAdmin.recoverBookieData(bookieSrc, null);
+            bkAdmin.recoverBookieData(bookieSrc);
             fail("Should have thrown exception");
         } catch (BKException.BKLedgerRecoveryException bke) {
             // correct behaviour
@@ -811,7 +808,7 @@ public void testSyncBookieRecoveryToRandomBookiesCheckForDupes() throws Exceptio
           + ") and replicate it to a random available one");
         // Initiate the sync object
         sync.value = false;
-        bkAdmin.recoverBookieData(bookieSrc, null);
+        bkAdmin.recoverBookieData(bookieSrc);
 
         assertFalse("Dupes exist in ensembles", findDupesInEnsembles(lhs));
 
@@ -857,7 +854,7 @@ public void recoverWithoutPasswordInConf() throws Exception {
         setMetastoreImplClass(adminConf);
 
         BookKeeperAdmin bka = new BookKeeperAdmin(adminConf);
-        bka.recoverBookieData(bookieSrc, null);
+        bka.recoverBookieData(bookieSrc);
         bka.close();
 
         lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect);
@@ -880,7 +877,7 @@ public void recoverWithoutPasswordInConf() throws Exception {
         setMetastoreImplClass(adminConf);
 
         bka = new BookKeeperAdmin(adminConf);
-        bka.recoverBookieData(bookieSrc, null);
+        bka.recoverBookieData(bookieSrc);
         bka.close();
 
         lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java
index ea6205f15..53cd87e1f 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java
@@ -274,7 +274,7 @@ public void testFencingInteractionWithBookieRecovery() throws Exception {
             writelh.addEntry(tmp.getBytes());
         }
 
-        admin.recoverBookieData(bookieToKill, null);
+        admin.recoverBookieData(bookieToKill);
 
         for (int i = 0; i < numEntries; i++) {
             writelh.addEntry(tmp.getBytes());
@@ -321,7 +321,7 @@ public void testFencingInteractionWithBookieRecovery2() throws Exception {
         BookieSocketAddress bookieToKill
             = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0);
         killBookie(bookieToKill);
-        admin.recoverBookieData(bookieToKill, null);
+        admin.recoverBookieData(bookieToKill);
 
         try {
             writelh.addEntry(tmp.getBytes());
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/http/TestHttpService.java
index 7a49b3050..5be680f47 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/http/TestHttpService.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/http/TestHttpService.java
@@ -575,21 +575,12 @@ public void testRecoveryBookieService() throws Exception {
 
         //3, body with bookie_src, bookie_dest and delete_cookie of PUT, should success.
         String bookieSrc = getBookie(0).toString();
-        String bookieDest = getBookie(1).toString();
-        String putBody = "{\"bookie_src\": [ \"" + bookieSrc + "\" ],"
-          + "\"bookie_dest\": [ \"" + bookieDest + "\" ],"
-          + "\"delete_cookie\": true }";
-        HttpServiceRequest request3 = new HttpServiceRequest(putBody, HttpServer.Method.PUT, null);
+        String putBody3 = "{\"bookie_src\": [ \"" + bookieSrc + "\" ],"
+          + "\"delete_cookie\": false }";
+        HttpServiceRequest request3 = new HttpServiceRequest(putBody3, HttpServer.Method.PUT, null);
         HttpServiceResponse response3 = recoveryBookieService.handle(request3);
         assertEquals(HttpServer.StatusCode.OK.getValue(), response3.getStatusCode());
 
-        //4, body with bookie_src, and delete_cookie of PUT, should success.
-        String putBody4 = "{\"bookie_src\": [ \"" + bookieSrc + "\" ],"
-          + "\"delete_cookie\": false }";
-        HttpServiceRequest request4 = new HttpServiceRequest(putBody4, HttpServer.Method.PUT, null);
-        HttpServiceResponse response4 = recoveryBookieService.handle(request4);
-        assertEquals(HttpServer.StatusCode.OK.getValue(), response4.getStatusCode());
-
         //5, body with bookie_src of PUT, should success.
         String putBody5 = "{\"bookie_src\": [ \"" + bookieSrc + "\" ] }";
         HttpServiceRequest request5 = new HttpServiceRequest(putBody5, HttpServer.Method.PUT, null);
diff --git a/tests/backward/src/test/java/org/apache/bookkeeper/tests/backward/TestBookieRecovery.java b/tests/backward/src/test/java/org/apache/bookkeeper/tests/backward/TestBookieRecovery.java
index a009154fe..64b42bbf4 100644
--- a/tests/backward/src/test/java/org/apache/bookkeeper/tests/backward/TestBookieRecovery.java
+++ b/tests/backward/src/test/java/org/apache/bookkeeper/tests/backward/TestBookieRecovery.java
@@ -318,7 +318,7 @@ public void ensurePasswordUsedForOldLedgers() throws Exception {
 
             bka = new BookKeeperAdmin(adminConf);
             try {
-                bka.recoverBookieData(bookieSrc, null);
+                bka.recoverBookieData(bookieSrc);
                 fail("Shouldn't be able to recover with wrong password");
             } catch (BKException bke) {
                 // correct behaviour
@@ -336,7 +336,7 @@ public void ensurePasswordUsedForOldLedgers() throws Exception {
 
         bka = new BookKeeperAdmin(adminConf);
         try {
-            bka.recoverBookieData(bookieSrc, null);
+            bka.recoverBookieData(bookieSrc);
             fail("Shouldn't be able to recover with wrong digest");
         } catch (BKException bke) {
             // correct behaviour
@@ -353,7 +353,7 @@ public void ensurePasswordUsedForOldLedgers() throws Exception {
         adminConf.setBookieRecoveryPasswd(passwdCorrect);
 
         bka = new BookKeeperAdmin(adminConf);
-        bka.recoverBookieData(bookieSrc, null);
+        bka.recoverBookieData(bookieSrc);
         bka.close();
 
         lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect);


 

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