You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by iv...@apache.org on 2018/01/24 12:06:09 UTC

[bookkeeper] branch master updated: Ledgers without masterkey in metadata can't be recovered

This is an automated email from the ASF dual-hosted git repository.

ivank pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 519743e  Ledgers without masterkey in metadata can't be recovered
519743e is described below

commit 519743e51cdc40c3e3ddbfabd83640fc30838a10
Author: Ivan Kelly <iv...@apache.org>
AuthorDate: Wed Jan 24 13:06:01 2018 +0100

    Ledgers without masterkey in metadata can't be recovered
    
    Test that ledgers, created without a master key in the metadata, cannot be recovered until
    the master key is known.
    
    Replaces TestBookieRecovery from the old tests.
    
    Master Issue: #903
    
    Author: Ivan Kelly <iv...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
    
    This closes #1035 from ivankelly/bc-recovery
---
 tests/backward-compat-recovery-no-password/pom.xml |  43 +++
 .../tests/TestCompatRecoveryNoPassword.groovy      | 248 ++++++++++++++
 .../src/test/resources/arquillian.xml              |  28 ++
 .../tests/backward/TestBookieRecovery.java         | 364 ---------------------
 tests/pom.xml                                      |   1 +
 5 files changed, 320 insertions(+), 364 deletions(-)

diff --git a/tests/backward-compat-recovery-no-password/pom.xml b/tests/backward-compat-recovery-no-password/pom.xml
new file mode 100644
index 0000000..4e1c528
--- /dev/null
+++ b/tests/backward-compat-recovery-no-password/pom.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="
+  http://maven.apache.org/POM/4.0.0
+  http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.bookkeeper.tests</groupId>
+    <artifactId>integration-tests-base-groovy</artifactId>
+    <version>4.7.0-SNAPSHOT</version>
+    <relativePath>../integration-tests-base-groovy</relativePath>
+  </parent>
+
+  <groupId>org.apache.bookkeeper.tests</groupId>
+  <artifactId>backward-compat-recovery-no-password</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache BookKeeper :: Tests :: Test recovery does not work when password no in metadata</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-server</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
+</project>
diff --git a/tests/backward-compat-recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/TestCompatRecoveryNoPassword.groovy b/tests/backward-compat-recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/TestCompatRecoveryNoPassword.groovy
new file mode 100644
index 0000000..e522d13
--- /dev/null
+++ b/tests/backward-compat-recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/TestCompatRecoveryNoPassword.groovy
@@ -0,0 +1,248 @@
+/*
+* 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.tests
+
+import com.github.dockerjava.api.DockerClient
+
+import io.netty.buffer.ByteBuf
+import java.util.ArrayList
+import java.util.HashMap
+import java.util.concurrent.CompletableFuture
+import java.util.concurrent.CountDownLatch
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.bookkeeper.client.BKException
+import org.apache.bookkeeper.client.BookKeeper
+import org.apache.bookkeeper.client.BookKeeperAdmin
+import org.apache.bookkeeper.client.LedgerHandle
+import org.apache.bookkeeper.client.LedgerMetadata
+import org.apache.bookkeeper.conf.ClientConfiguration
+import org.apache.bookkeeper.net.BookieSocketAddress
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback
+
+import org.jboss.arquillian.junit.Arquillian
+import org.jboss.arquillian.test.api.ArquillianResource
+
+import org.junit.Assert
+import org.junit.Test
+import org.junit.runner.RunWith
+
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+@RunWith(Arquillian.class)
+class TestCompatRecoveryNoPassword {
+    private static final Logger LOG = LoggerFactory.getLogger(TestCompatRecoveryNoPassword.class)
+    private static byte[] PASSWD = "foobar".getBytes()
+
+    @ArquillianResource
+    DockerClient docker
+
+    private LedgerMetadata getLedgerMetadata(BookKeeper bookkeeper, long ledgerId) throws Exception {
+        CompletableFuture<LedgerMetadata> future = new CompletableFuture<LedgerMetadata>()
+        bookkeeper.getLedgerManager().readLedgerMetadata(
+            ledgerId, { rc, result ->
+                if (rc != BKException.Code.OK) {
+                    future.completeExceptionally(BKException.create(rc))
+                } else {
+                    future.complete(result)
+                }
+            })
+        return future.get()
+    }
+
+    private static class ReplicationVerificationCallback implements ReadEntryCallback {
+        final CountDownLatch latch;
+        final AtomicLong numSuccess;
+
+        ReplicationVerificationCallback(int numRequests) {
+            latch = new CountDownLatch(numRequests)
+            numSuccess = new AtomicLong(0)
+        }
+
+        @Override
+        public void readEntryComplete(int rc, long ledgerId, long entryId,
+                                      ByteBuf buffer, Object ctx) {
+            LOG.debug("Got {} for ledger {} entry {} from {}", rc, ledgerId, entryId, ctx)
+            if (rc == BKException.Code.OK) {
+                numSuccess.incrementAndGet()
+            }
+            latch.countDown()
+        }
+
+        long await() throws InterruptedException {
+            if (!latch.await(60, TimeUnit.SECONDS)) {
+                LOG.warn("Didn't get all responses in verification");
+                return 0;
+            } else {
+                return numSuccess.get();
+            }
+        }
+    }
+
+    private boolean verifyFullyReplicated(BookKeeper bookkeeper,
+                                          LedgerHandle lh,
+                                          long untilEntry) throws Exception {
+        LedgerMetadata md = getLedgerMetadata(bookkeeper, lh.getId())
+
+        def ensembles = md.getEnsembles()
+
+        def ranges = new HashMap<Long, Long>()
+        def keyList = new ArrayList(ensembles.keySet())
+        Collections.sort(keyList)
+        for (int i = 0; i < keyList.size() - 1; i++) {
+            ranges.put(keyList.get(i), keyList.get(i + 1))
+        }
+        ranges.put(keyList.get(keyList.size() - 1), untilEntry)
+
+        for (def e : ensembles.entrySet()) {
+            int quorum = md.getAckQuorumSize()
+            long startEntryId = e.getKey()
+            long endEntryId = ranges.get(startEntryId)
+            long expectedSuccess = quorum * (endEntryId - startEntryId)
+            int numRequests = e.getValue().size() * ((int) (endEntryId - startEntryId))
+
+            def cb = new ReplicationVerificationCallback(numRequests)
+            for (long i = startEntryId; i < endEntryId; i++) {
+                for (BookieSocketAddress addr : e.getValue()) {
+                    bookkeeper.getBookieClient()
+                        .readEntry(addr, lh.getId(), i, cb, addr)
+                }
+            }
+
+            long numSuccess = cb.await();
+            if (numSuccess < expectedSuccess) {
+                LOG.warn("Fragment not fully replicated ledgerId = {} startEntryId = {}"
+                         + " endEntryId = {} expectedSuccess = {} gotSuccess = {}",
+                         lh.getId(), startEntryId, endEntryId, expectedSuccess, numSuccess);
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * Test that when we try to recover a ledger which doesn't have
+     * the password stored in the configuration, we don't succeed.
+     */
+    @Test
+    public void testRecoveryWithoutPasswordInMetadata() throws Exception {
+        int numEntries = 10
+        byte[] passwdCorrect = "AAAAAA".getBytes()
+        byte[] passwdBad = "BBBBBB".getBytes()
+
+        String currentVersion = System.getProperty("currentVersion")
+        String zookeeper = BookKeeperClusterUtils.zookeeperConnectString(docker)
+
+        BookKeeperClusterUtils.legacyMetadataFormat(docker)
+
+        // Create a 4.1.0 client, will update /ledgers/LAYOUT
+        def v410CL = MavenClassLoader.forBookKeeperVersion("4.1.0")
+        def v410Conf = v410CL.newInstance("org.apache.bookkeeper.conf.ClientConfiguration")
+        v410Conf.setZkServers(zookeeper).setLedgerManagerType("hierarchical")
+        def v410BK = v410CL.newInstance("org.apache.bookkeeper.client.BookKeeper", v410Conf)
+
+        // Start bookies
+        def bookieContainers = new ArrayList<>(DockerUtils.cubeIdsMatching("bookkeeper"))
+        Assert.assertTrue(bookieContainers.size() >= 3)
+        Assert.assertTrue(BookKeeperClusterUtils.startBookieWithVersion(
+                docker, bookieContainers.get(0), currentVersion))
+        Assert.assertTrue(BookKeeperClusterUtils.startBookieWithVersion(
+                docker, bookieContainers.get(1), currentVersion))
+
+        // recreate bk client so that it reads bookie list
+        v410BK.close()
+        v410BK = v410CL.newBookKeeper(zookeeper)
+
+        // Write a ledger
+        def ledger0 = v410BK.createLedger(2, 2,
+                                          v410CL.digestType("MAC"), passwdCorrect)
+        for (int i = 0; i < numEntries; i++) {
+            ledger0.addEntry("foobar".getBytes())
+        }
+        ledger0.close()
+        v410BK.close()
+
+        // start a new bookie, and kill one of the initial 2
+        def failedBookieId = new BookieSocketAddress(
+            DockerUtils.getContainerIP(docker, bookieContainers.get(0)), 3181)
+        Assert.assertTrue(BookKeeperClusterUtils.stopBookie(
+                docker, bookieContainers.get(0)))
+        Assert.assertTrue(BookKeeperClusterUtils.startBookieWithVersion(
+                docker, bookieContainers.get(2), currentVersion))
+
+        def bkCur = new BookKeeper(zookeeper)
+
+        LedgerHandle lh = bkCur.openLedgerNoRecovery(
+            ledger0.getId(), BookKeeper.DigestType.MAC, passwdCorrect)
+        Assert.assertFalse("Should be entries missing",
+                           verifyFullyReplicated(bkCur, lh, numEntries))
+        lh.close()
+
+        ClientConfiguration adminConf = new ClientConfiguration()
+        adminConf.setZkServers(zookeeper)
+        adminConf.setBookieRecoveryDigestType(BookKeeper.DigestType.MAC)
+        adminConf.setBookieRecoveryPasswd(passwdBad)
+
+        def bka = new BookKeeperAdmin(adminConf)
+        try {
+            bka.recoverBookieData(failedBookieId)
+            Assert.fail("Shouldn't be able to recover with wrong password")
+        } catch (BKException bke) {
+            // correct behaviour
+        } finally {
+            bka.close();
+        }
+
+        adminConf.setBookieRecoveryDigestType(BookKeeper.DigestType.CRC32)
+        adminConf.setBookieRecoveryPasswd(passwdCorrect)
+
+        bka = new BookKeeperAdmin(adminConf)
+        try {
+            bka.recoverBookieData(failedBookieId)
+            Assert.fail("Shouldn't be able to recover with wrong digest")
+        } catch (BKException bke) {
+            // correct behaviour
+        } finally {
+            bka.close();
+        }
+
+        // Check that entries are still missing
+        lh = bkCur.openLedgerNoRecovery(ledger0.getId(),
+                                        BookKeeper.DigestType.MAC, passwdCorrect)
+        Assert.assertFalse("Should be entries missing",
+                           verifyFullyReplicated(bkCur, lh, numEntries))
+        lh.close()
+
+
+        // Set correct password and mac, recovery will work
+        adminConf.setBookieRecoveryDigestType(BookKeeper.DigestType.MAC)
+        adminConf.setBookieRecoveryPasswd(passwdCorrect)
+
+        bka = new BookKeeperAdmin(adminConf)
+        bka.recoverBookieData(failedBookieId)
+        bka.close()
+
+        lh = bkCur.openLedgerNoRecovery(ledger0.getId(),
+                                        BookKeeper.DigestType.MAC, passwdCorrect)
+        Assert.assertTrue("Should have recovered everything",
+                          verifyFullyReplicated(bkCur, lh, numEntries))
+        lh.close()
+    }
+}
diff --git a/tests/backward-compat-recovery-no-password/src/test/resources/arquillian.xml b/tests/backward-compat-recovery-no-password/src/test/resources/arquillian.xml
new file mode 100644
index 0000000..f914ff2
--- /dev/null
+++ b/tests/backward-compat-recovery-no-password/src/test/resources/arquillian.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<arquillian xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+            xmlns="http://jboss.org/schema/arquillian"
+            xsi:schemaLocation="http://jboss.org/schema/arquillian
+                                http://jboss.org/schema/arquillian/arquillian_1_0.xsd">
+
+  <extension qualifier="docker">
+    <property name="definitionFormat">CUBE</property>
+    <property name="dockerContainersResource">cube-definitions/3-node-all-version-unstarted.yaml</property>
+  </extension>
+
+</arquillian>
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
deleted file mode 100644
index 64b42bb..0000000
--- a/tests/backward/src/test/java/org/apache/bookkeeper/tests/backward/TestBookieRecovery.java
+++ /dev/null
@@ -1,364 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- *
- */
-package org.apache.bookkeeper.tests.backward;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import io.netty.buffer.ByteBuf;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.bookkeeper.client.AsyncCallback.RecoverCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper.DigestType;
-import org.apache.bookkeeper.client.BookKeeperAdmin;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory;
-import org.apache.bookkeeper.meta.MSLedgerManagerFactory;
-import org.apache.bookkeeper.net.BookieSocketAddress;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
-import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test backward compat issue on bookie recovery.
- */
-public class TestBookieRecovery extends BookKeeperClusterTestCase {
-
-    private static final Logger LOG = LoggerFactory.getLogger(TestBookieRecovery.class);
-
-    // Object used for synchronizing async method calls
-    class SyncObject {
-        boolean value;
-
-        public SyncObject() {
-            value = false;
-        }
-    }
-
-    // Object used for implementing the Bookie RecoverCallback for this jUnit
-    // test. This verifies that the operation completed successfully.
-    class BookieRecoverCallback implements RecoverCallback {
-        boolean success = false;
-        @Override
-        public void recoverComplete(int rc, Object ctx) {
-            LOG.info("Recovered bookie operation completed with rc: " + rc);
-            success = rc == BKException.Code.OK;
-            SyncObject sync = (SyncObject) ctx;
-            synchronized (sync) {
-                sync.value = true;
-                sync.notify();
-            }
-        }
-    }
-
-    // Objects to use for this jUnit test.
-    DigestType digestType;
-    String ledgerManagerFactory;
-    SyncObject sync;
-    BookieRecoverCallback bookieRecoverCb;
-    BookKeeperAdmin bkAdmin;
-
-    // Constructor
-    public TestBookieRecovery() {
-        super(3);
-
-        this.digestType = DigestType.CRC32;
-        this.ledgerManagerFactory = "org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory";
-        LOG.info("Using ledger manager " + ledgerManagerFactory);
-        // set ledger manager
-        baseConf.setLedgerManagerFactoryClassName(ledgerManagerFactory);
-        baseConf.setOpenFileLimit(200); // Limit the number of open files to avoid reaching the proc max
-        baseClientConf.setLedgerManagerFactoryClassName(ledgerManagerFactory);
-    }
-
-    @Before
-    @Override
-    public void setUp() throws Exception {
-        // Set up the configuration properties needed.
-        baseClientConf.setBookieRecoveryDigestType(digestType);
-        baseClientConf.setBookieRecoveryPasswd("".getBytes());
-        super.setUp();
-
-        sync = new SyncObject();
-        bookieRecoverCb = new BookieRecoverCallback();
-        ClientConfiguration adminConf = new ClientConfiguration(baseClientConf);
-        adminConf.setZkServers(zkUtil.getZooKeeperConnectString());
-        bkAdmin = new BookKeeperAdmin(adminConf);
-    }
-
-    @After
-    @Override
-    public void tearDown() throws Exception {
-        // Release any resources used by the BookKeeperTools instance.
-        if (bkAdmin != null) {
-            bkAdmin.close();
-        }
-        super.tearDown();
-    }
-
-    // Object used for synchronizing async method calls
-    class SyncLedgerMetaObject {
-        boolean value;
-        int rc;
-        LedgerMetadata meta;
-
-        public SyncLedgerMetaObject() {
-            value = false;
-            meta = null;
-        }
-    }
-
-    private LedgerMetadata getLedgerMetadata(LedgerHandle lh) throws Exception {
-        final SyncLedgerMetaObject syncObj = new SyncLedgerMetaObject();
-        bkc.getLedgerManager().readLedgerMetadata(lh.getId(), new GenericCallback<LedgerMetadata>() {
-
-            @Override
-            public void operationComplete(int rc, LedgerMetadata result) {
-                synchronized (syncObj) {
-                    syncObj.rc = rc;
-                    syncObj.meta = result;
-                    syncObj.value = true;
-                    syncObj.notify();
-                }
-            }
-
-        });
-
-        synchronized (syncObj) {
-            while (!syncObj.value) {
-                syncObj.wait();
-            }
-        }
-        assertEquals(BKException.Code.OK, syncObj.rc);
-        return syncObj.meta;
-    }
-
-    private static class ReplicationVerificationCallback implements ReadEntryCallback {
-        final CountDownLatch latch;
-        final AtomicLong numSuccess;
-
-        ReplicationVerificationCallback(int numRequests) {
-            latch = new CountDownLatch(numRequests);
-            numSuccess = new AtomicLong(0);
-        }
-
-        @Override
-        public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffer, Object ctx) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Got " + rc + " for ledger " + ledgerId + " entry " + entryId + " from " + ctx);
-            }
-            if (rc == BKException.Code.OK) {
-                numSuccess.incrementAndGet();
-            }
-            latch.countDown();
-        }
-
-        long await() throws InterruptedException {
-            if (!latch.await(60, TimeUnit.SECONDS)) {
-                LOG.warn("Didn't get all responses in verification");
-                return 0;
-            } else {
-                return numSuccess.get();
-            }
-        }
-    }
-
-    private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws Exception {
-        LedgerMetadata md = getLedgerMetadata(lh);
-
-        Map<Long, ArrayList<BookieSocketAddress>> ensembles = md.getEnsembles();
-
-        HashMap<Long, Long> ranges = new HashMap<Long, Long>();
-        ArrayList<Long> keyList = Collections.list(
-                Collections.enumeration(ensembles.keySet()));
-        Collections.sort(keyList);
-        for (int i = 0; i < keyList.size() - 1; i++) {
-            ranges.put(keyList.get(i), keyList.get(i + 1));
-        }
-        ranges.put(keyList.get(keyList.size() - 1), untilEntry);
-
-        for (Map.Entry<Long, ArrayList<BookieSocketAddress>> e : ensembles.entrySet()) {
-            int quorum = md.getAckQuorumSize();
-            long startEntryId = e.getKey();
-            long endEntryId = ranges.get(startEntryId);
-            long expectedSuccess = quorum * (endEntryId - startEntryId);
-            int numRequests = e.getValue().size() * ((int) (endEntryId - startEntryId));
-
-            ReplicationVerificationCallback cb = new ReplicationVerificationCallback(numRequests);
-            for (long i = startEntryId; i < endEntryId; i++) {
-                for (BookieSocketAddress addr : e.getValue()) {
-                    bkc.getBookieClient().readEntry(addr, lh.getId(), i, cb, addr);
-                }
-            }
-
-            long numSuccess = cb.await();
-            if (numSuccess < expectedSuccess) {
-                LOG.warn("Fragment not fully replicated ledgerId = " + lh.getId()
-                         + " startEntryId = " + startEntryId
-                         + " endEntryId = " + endEntryId
-                         + " expectedSuccess = " + expectedSuccess
-                         + " gotSuccess = " + numSuccess);
-                return false;
-            }
-        }
-        return true;
-    }
-
-    /**
-     * Test that when we try to recover a ledger which doesn't have
-     * the password stored in the configuration, we don't succeed.
-     */
-    @Test
-    public void ensurePasswordUsedForOldLedgers() throws Exception {
-        // This test bases on creating old ledgers in version 4.1.0, which only
-        // supports ZooKeeper based flat and hierarchical LedgerManagerFactory.
-        // So we ignore it for MSLedgerManagerFactory and LongHierarchicalLedgerManagerFactory.
-        if (MSLedgerManagerFactory.class.getName().equals(ledgerManagerFactory)) {
-            return;
-        }
-        if (LongHierarchicalLedgerManagerFactory.class.getName().equals(ledgerManagerFactory)) {
-            return;
-        }
-
-        // stop all bookies
-        // and wipe the ledger layout so we can use an old client
-        zkUtil.getZooKeeperClient().delete("/ledgers/LAYOUT", -1);
-
-        byte[] passwdCorrect = "AAAAAA".getBytes();
-        byte[] passwdBad = "BBBBBB".getBytes();
-        DigestType digestCorrect = digestType;
-        DigestType digestBad = digestCorrect == DigestType.MAC ? DigestType.CRC32 : DigestType.MAC;
-
-        org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper.DigestType digestCorrect410 =
-                org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper.DigestType.valueOf(digestType.toString());
-
-        org.apache.bk_v4_1_0.bookkeeper.conf.ClientConfiguration c =
-                new org.apache.bk_v4_1_0.bookkeeper.conf.ClientConfiguration();
-        c.setZkServers(zkUtil.getZooKeeperConnectString())
-            .setLedgerManagerType(
-                    ledgerManagerFactory.equals("org.apache.bookkeeper.meta.FlatLedgerManagerFactory")
-                            ? "flat" : "hierarchical");
-
-        // create client to set up layout, close it, restart bookies, and open a new client.
-        // the new client is necessary to ensure that it has all the restarted bookies in the
-        // its available bookie list
-        org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper bkc41 =
-                new org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper(c);
-        bkc41.close();
-        restartBookies();
-        bkc41 = new org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper(c);
-
-        org.apache.bk_v4_1_0.bookkeeper.client.LedgerHandle lh41 =
-                bkc41.createLedger(3, 2, digestCorrect410, passwdCorrect);
-        long ledgerId = lh41.getId();
-        for (int i = 0; i < 100; i++) {
-            lh41.addEntry("foobar".getBytes());
-        }
-        lh41.close();
-        bkc41.close();
-
-        // Startup a new bookie server
-        startNewBookie();
-        int removeIndex = 0;
-        BookieSocketAddress bookieSrc = bs.get(removeIndex).getLocalAddress();
-        bs.get(removeIndex).shutdown();
-        bs.remove(removeIndex);
-
-        // Check that entries are missing
-        LedgerHandle lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect);
-        assertFalse("Should be entries missing", verifyFullyReplicated(lh, 100));
-        lh.close();
-
-        // Try to recover with bad password in conf
-        // if the digest type is MAC
-        // for CRC32, the password is only checked
-        // when adding new entries, which recovery will
-        // never do
-        ClientConfiguration adminConf;
-        BookKeeperAdmin bka;
-        if (digestCorrect == DigestType.MAC) {
-            adminConf = new ClientConfiguration();
-            adminConf.setZkServers(zkUtil.getZooKeeperConnectString());
-            adminConf.setLedgerManagerFactoryClassName(ledgerManagerFactory);
-            adminConf.setBookieRecoveryDigestType(digestCorrect);
-            adminConf.setBookieRecoveryPasswd(passwdBad);
-
-            bka = new BookKeeperAdmin(adminConf);
-            try {
-                bka.recoverBookieData(bookieSrc);
-                fail("Shouldn't be able to recover with wrong password");
-            } catch (BKException bke) {
-                // correct behaviour
-            } finally {
-                bka.close();
-            }
-        }
-
-        // Try to recover with bad digest in conf
-        adminConf = new ClientConfiguration();
-        adminConf.setZkServers(zkUtil.getZooKeeperConnectString());
-        adminConf.setLedgerManagerFactoryClassName(ledgerManagerFactory);
-        adminConf.setBookieRecoveryDigestType(digestBad);
-        adminConf.setBookieRecoveryPasswd(passwdCorrect);
-
-        bka = new BookKeeperAdmin(adminConf);
-        try {
-            bka.recoverBookieData(bookieSrc);
-            fail("Shouldn't be able to recover with wrong digest");
-        } catch (BKException bke) {
-            // correct behaviour
-        } finally {
-            bka.close();
-        }
-
-        // Check that entries are still missing
-        lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect);
-        assertFalse("Should be entries missing", verifyFullyReplicated(lh, 100));
-        lh.close();
-
-        adminConf.setBookieRecoveryDigestType(digestCorrect);
-        adminConf.setBookieRecoveryPasswd(passwdCorrect);
-
-        bka = new BookKeeperAdmin(adminConf);
-        bka.recoverBookieData(bookieSrc);
-        bka.close();
-
-        lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect);
-        assertTrue("Should have recovered everything", verifyFullyReplicated(lh, 100));
-        lh.close();
-    }
-
-}
diff --git a/tests/pom.xml b/tests/pom.xml
index b242b51..8382eb2 100644
--- a/tests/pom.xml
+++ b/tests/pom.xml
@@ -43,6 +43,7 @@
     <module>backward-compat-upgrade</module>
     <module>backward-compat-upgrade-direct</module>
     <module>backward-compat-hierarchical-ledger-manager</module>
+    <module>backward-compat-recovery-no-password</module>
   </modules>
   <build>
     <plugins>

-- 
To stop receiving notification emails like this one, please contact
ivank@apache.org.